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 wa...@apache.org on 2017/12/10 21:54:34 UTC

[01/50] [abbrv] hadoop git commit: YARN-7438. Additional changes to make SchedulingPlacementSet agnostic to ResourceRequest / placement algorithm. Contributed by Wangda Tan [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-6592 9152b843f -> 0e66d31e2 (forced update)


YARN-7438. Additional changes to make SchedulingPlacementSet agnostic to ResourceRequest / placement algorithm. 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/a957f1c6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a957f1c6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a957f1c6

Branch: refs/heads/YARN-6592
Commit: a957f1c60e1308d1d70a1803381994f59949c5f8
Parents: 3150c01
Author: Sunil G <su...@apache.org>
Authored: Tue Dec 5 22:50:07 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Dec 5 22:50:07 2017 +0530

----------------------------------------------------------------------
 .../rmcontainer/RMContainer.java                |  5 +-
 .../rmcontainer/RMContainerImpl.java            | 20 +++---
 .../scheduler/AbstractYarnScheduler.java        |  7 ++-
 .../scheduler/AppSchedulingInfo.java            | 66 +++++++++-----------
 .../scheduler/ContainerUpdateContext.java       | 11 ++--
 .../scheduler/SchedulerApplicationAttempt.java  | 15 ++---
 .../allocator/RegularContainerAllocator.java    |  3 +-
 .../scheduler/common/ContainerRequest.java      | 54 ++++++++++++++++
 .../scheduler/common/fica/FiCaSchedulerApp.java | 15 ++---
 .../scheduler/fair/FSAppAttempt.java            |  5 +-
 .../scheduler/fifo/FifoAppAttempt.java          |  5 +-
 .../placement/AppPlacementAllocator.java        | 19 +++---
 .../LocalityAppPlacementAllocator.java          | 32 +++++-----
 .../placement/PendingAskUpdateResult.java       | 65 +++++++++++++++++++
 .../placement/ResourceRequestUpdateResult.java  | 43 -------------
 .../server/resourcemanager/Application.java     |  6 +-
 .../rmcontainer/TestRMContainerImpl.java        |  8 +--
 .../capacity/TestCapacityScheduler.java         |  3 +-
 18 files changed, 227 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.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/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
index 29680e5..f3cbf63 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
 
@@ -86,8 +87,8 @@ public interface RMContainer extends EventHandler<RMContainerEvent>,
   ContainerReport createContainerReport();
   
   boolean isAMContainer();
-  
-  List<ResourceRequest> getResourceRequests();
+
+  ContainerRequest getContainerRequest();
 
   String getNodeHttpAddress();
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.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/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index a43459c..e26689e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeUpdateContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -178,7 +179,7 @@ public class RMContainerImpl implements RMContainer {
   private long finishTime;
   private ContainerStatus finishedStatus;
   private boolean isAMContainer;
-  private List<ResourceRequest> resourceRequests;
+  private ContainerRequest containerRequestForRecovery;
 
   // Only used for container resource increase and decrease. This is the
   // resource to rollback to should container resource increase token expires.
@@ -233,7 +234,6 @@ public class RMContainerImpl implements RMContainer {
     this.eventHandler = rmContext.getDispatcher().getEventHandler();
     this.containerAllocationExpirer = rmContext.getContainerAllocationExpirer();
     this.isAMContainer = false;
-    this.resourceRequests = null;
     this.nodeLabelExpression = nodeLabelExpression;
     this.lastConfirmedResource = container.getResource();
     this.isExternallyAllocated = isExternallyAllocated;
@@ -412,21 +412,21 @@ public class RMContainerImpl implements RMContainer {
       readLock.unlock();
     }
   }
-  
+
   @Override
-  public List<ResourceRequest> getResourceRequests() {
+  public ContainerRequest getContainerRequest() {
     try {
       readLock.lock();
-      return resourceRequests;
+      return containerRequestForRecovery;
     } finally {
       readLock.unlock();
     }
   }
-  
-  public void setResourceRequests(List<ResourceRequest> requests) {
+
+  public void setContainerRequest(ContainerRequest request) {
+    writeLock.lock();
     try {
-      writeLock.lock();
-      this.resourceRequests = requests;
+      this.containerRequestForRecovery = request;
     } finally {
       writeLock.unlock();
     }
@@ -576,7 +576,7 @@ public class RMContainerImpl implements RMContainer {
     public void transition(RMContainerImpl container, RMContainerEvent event) {
       // Clear ResourceRequest stored in RMContainer, we don't need to remember
       // this anymore.
-      container.setResourceRequests(null);
+      container.setContainerRequest(null);
       
       // Register with containerAllocationExpirer.
       container.containerAllocationExpirer.register(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 4749c3d..d94efb1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -86,6 +86,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContai
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 
 
@@ -600,10 +601,10 @@ public abstract class AbstractYarnScheduler
    * @param rmContainer rmContainer
    */
   private void recoverResourceRequestForContainer(RMContainer rmContainer) {
-    List<ResourceRequest> requests = rmContainer.getResourceRequests();
+    ContainerRequest containerRequest = rmContainer.getContainerRequest();
 
     // If container state is moved to ACQUIRED, request will be empty.
-    if (requests == null) {
+    if (containerRequest == null) {
       return;
     }
 
@@ -618,7 +619,7 @@ public abstract class AbstractYarnScheduler
     SchedulerApplicationAttempt schedulerAttempt =
         getCurrentAttemptForContainer(rmContainer.getContainerId());
     if (schedulerAttempt != null) {
-      schedulerAttempt.recoverResourceRequestsForContainer(requests);
+      schedulerAttempt.recoverResourceRequestsForContainer(containerRequest);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 9f49880..e47f0c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -45,10 +45,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.LocalityAppPlacementAllocator;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceRequestUpdateResult;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PendingAskUpdateResult;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.util.resource.Resources;
 /**
@@ -220,16 +221,14 @@ public class AppSchedulingInfo {
       }
 
       // Update AppPlacementAllocator
-      ResourceRequestUpdateResult pendingAmountChanges =
+      PendingAskUpdateResult pendingAmountChanges =
           schedulerKeyToAppPlacementAllocator.get(schedulerRequestKey)
-              .updateResourceRequests(
-                  entry.getValue().values(),
+              .updatePendingAsk(entry.getValue().values(),
                   recoverPreemptedRequestForAContainer);
 
       if (null != pendingAmountChanges) {
         updatePendingResources(
-            pendingAmountChanges.getLastAnyResourceRequest(),
-            pendingAmountChanges.getNewResourceRequest(), schedulerRequestKey,
+            pendingAmountChanges, schedulerRequestKey,
             queue.getMetrics());
         offswitchResourcesUpdated = true;
       }
@@ -237,12 +236,17 @@ public class AppSchedulingInfo {
     return offswitchResourcesUpdated;
   }
 
-  private void updatePendingResources(ResourceRequest lastRequest,
-      ResourceRequest request, SchedulerRequestKey schedulerKey,
-      QueueMetrics metrics) {
+  private void updatePendingResources(PendingAskUpdateResult updateResult,
+      SchedulerRequestKey schedulerKey, QueueMetrics metrics) {
+
+    PendingAsk lastPendingAsk = updateResult.getLastPendingAsk();
+    PendingAsk newPendingAsk = updateResult.getNewPendingAsk();
+    String lastNodePartition = updateResult.getLastNodePartition();
+    String newNodePartition = updateResult.getNewNodePartition();
+
     int lastRequestContainers =
-        (lastRequest != null) ? lastRequest.getNumContainers() : 0;
-    if (request.getNumContainers() <= 0) {
+        (lastPendingAsk != null) ? lastPendingAsk.getCount() : 0;
+    if (newPendingAsk.getCount() <= 0) {
       if (lastRequestContainers >= 0) {
         schedulerKeys.remove(schedulerKey);
         schedulerKeyToAppPlacementAllocator.remove(schedulerKey);
@@ -258,31 +262,23 @@ public class AppSchedulingInfo {
       }
     }
 
-    Resource lastRequestCapability =
-        lastRequest != null ? lastRequest.getCapability() : Resources.none();
-    metrics.incrPendingResources(request.getNodeLabelExpression(), user,
-        request.getNumContainers(), request.getCapability());
-
-    if(lastRequest != null) {
-      metrics.decrPendingResources(lastRequest.getNodeLabelExpression(), user,
-          lastRequestContainers, lastRequestCapability);
+    if (lastPendingAsk != null) {
+      // Deduct resources from metrics / pending resources of queue/app.
+      metrics.decrPendingResources(lastNodePartition, user,
+          lastPendingAsk.getCount(), lastPendingAsk.getPerAllocationResource());
+      Resource decreasedResource = Resources.multiply(
+          lastPendingAsk.getPerAllocationResource(), lastRequestContainers);
+      queue.decPendingResource(lastNodePartition, decreasedResource);
+      appResourceUsage.decPending(lastNodePartition, decreasedResource);
     }
 
-    // update queue:
-    Resource increasedResource =
-        Resources.multiply(request.getCapability(), request.getNumContainers());
-    queue.incPendingResource(request.getNodeLabelExpression(),
-        increasedResource);
-    appResourceUsage.incPending(request.getNodeLabelExpression(),
-        increasedResource);
-    if (lastRequest != null) {
-      Resource decreasedResource =
-          Resources.multiply(lastRequestCapability, lastRequestContainers);
-      queue.decPendingResource(lastRequest.getNodeLabelExpression(),
-          decreasedResource);
-      appResourceUsage.decPending(lastRequest.getNodeLabelExpression(),
-          decreasedResource);
-    }
+    // Increase resources to metrics / pending resources of queue/app.
+    metrics.incrPendingResources(newNodePartition, user,
+        newPendingAsk.getCount(), newPendingAsk.getPerAllocationResource());
+    Resource increasedResource = Resources.multiply(
+        newPendingAsk.getPerAllocationResource(), newPendingAsk.getCount());
+    queue.incPendingResource(newNodePartition, increasedResource);
+    appResourceUsage.incPending(newNodePartition, increasedResource);
   }
 
   public void addRequestedPartition(String partition) {
@@ -417,7 +413,7 @@ public class AppSchedulingInfo {
     }
   }
 
-  public List<ResourceRequest> allocate(NodeType type,
+  public ContainerRequest allocate(NodeType type,
       SchedulerNode node, SchedulerRequestKey schedulerKey,
       Container containerAllocated) {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.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/ContainerUpdateContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.java
index 93995a1..f410db1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer
     .RMContainerImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -155,16 +156,16 @@ public class ContainerUpdateContext {
     AppPlacementAllocator<SchedulerNode> appPlacementAllocator =
         appSchedulingInfo.getAppPlacementAllocator(schedulerKey);
     if (appPlacementAllocator != null) {
-      Map<String, ResourceRequest> resourceRequests = appPlacementAllocator
-          .getResourceRequests();
-      ResourceRequest prevReq = resourceRequests.get(ResourceRequest.ANY);
+      PendingAsk pendingAsk = appPlacementAllocator.getPendingAsk(
+          ResourceRequest.ANY);
       // Decrement the pending using a dummy RR with
       // resource = prev update req capability
-      if (prevReq != null) {
+      if (pendingAsk != null && pendingAsk.getCount() > 0) {
         appSchedulingInfo.allocate(NodeType.OFF_SWITCH, schedulerNode,
             schedulerKey, Container.newInstance(UNDEFINED,
                 schedulerNode.getNodeID(), "host:port",
-                prevReq.getCapability(), schedulerKey.getPriority(), null));
+                pendingAsk.getPerAllocationResource(),
+                schedulerKey.getPriority(), null));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 65673c9..dfb0e67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerUpda
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeUpdateContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
@@ -449,11 +450,12 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   }
   
   public void recoverResourceRequestsForContainer(
-      List<ResourceRequest> requests) {
+      ContainerRequest containerRequest) {
     try {
       writeLock.lock();
       if (!isStopped) {
-        appSchedulingInfo.updateResourceRequests(requests, true);
+        appSchedulingInfo.updateResourceRequests(
+            containerRequest.getResourceRequests(), true);
       }
     } finally {
       writeLock.unlock();
@@ -913,7 +915,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
         RMContainer c = tempIter.next();
         // Mark container for release (set RRs to null, so RM does not think
         // it is a recoverable container)
-        ((RMContainerImpl) c).setResourceRequests(null);
+        ((RMContainerImpl) c).setContainerRequest(null);
 
         // Release this container async-ly so as to prevent
         // 'LeafQueue::completedContainer()' from trying to acquire a lock
@@ -1383,13 +1385,6 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       SchedulerRequestKey schedulerRequestKey) {
     return appSchedulingInfo.getAppPlacementAllocator(schedulerRequestKey);
   }
-
-  public Map<String, ResourceRequest> getResourceRequests(
-      SchedulerRequestKey schedulerRequestKey) {
-    return appSchedulingInfo.getAppPlacementAllocator(schedulerRequestKey)
-        .getResourceRequests();
-  }
-
   public void incUnconfirmedRes(Resource res) {
     unconfirmedAllocatedMem.addAndGet(res.getMemorySize());
     unconfirmedAllocatedVcores.addAndGet(res.getVirtualCores());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
index 69e90c6..2642532 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
@@ -299,7 +299,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
       }
       // If we have only ANY requests for this schedulerKey, we should not
       // delay its scheduling.
-      if (application.getResourceRequests(schedulerKey).size() == 1) {
+      if (application.getAppPlacementAllocator(schedulerKey)
+          .getUniqueLocationAsks() == 1) {
         return true;
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ContainerRequest.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/ContainerRequest.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/ContainerRequest.java
new file mode 100644
index 0000000..075db79
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ContainerRequest.java
@@ -0,0 +1,54 @@
+/**
+ * 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.scheduler.common;
+
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+
+import java.util.List;
+
+/**
+ * ContainerRequest is a class to capture resource requests associated with a
+ * Container, this will be used by scheduler to recover resource requests if the
+ * container preempted or cancelled before AM acquire the container.
+ *
+ * It should include deducted resource requests when the container allocated.
+ *
+ * Lifecycle of the ContainerRequest is:
+ *
+ * <pre>
+ * 1) It is instantiated when container created.
+ * 2) It will be set to ContainerImpl by scheduler.
+ * 3) When container preempted or cancelled because of whatever reason before
+ *    container acquired by AM. ContainerRequest will be added back to pending
+ *    request pool.
+ * 4) It will be cleared from ContainerImpl if the container already acquired by
+ *    AM.
+ * </pre>
+ */
+public class ContainerRequest {
+  private List<ResourceRequest> requests;
+
+  public ContainerRequest(List<ResourceRequest> requests) {
+    this.requests = requests;
+  }
+
+  public List<ResourceRequest> getResourceRequests() {
+    return requests;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/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 93d51d8..34594cf 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
@@ -73,6 +73,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Scheduli
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AbstractContainerAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer;
@@ -369,7 +370,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
   public boolean accept(Resource cluster,
       ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
-    List<ResourceRequest> resourceRequests = null;
+    ContainerRequest containerRequest = null;
     boolean reReservation = false;
 
     try {
@@ -397,8 +398,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
         if (schedulerContainer.isAllocated()) {
           // When allocate a new container
-          resourceRequests =
-              schedulerContainer.getRmContainer().getResourceRequests();
+          containerRequest =
+              schedulerContainer.getRmContainer().getContainerRequest();
 
           // Check pending resource request
           if (!appSchedulingInfo.checkAllocation(allocation.getAllocationLocalityType(),
@@ -471,8 +472,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
 
     // When rejected, recover resource requests for this app
-    if (!accepted && resourceRequests != null) {
-      recoverResourceRequestsForContainer(resourceRequests);
+    if (!accepted && containerRequest != null) {
+      recoverResourceRequestsForContainer(containerRequest);
     }
 
     return accepted;
@@ -524,12 +525,12 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
           liveContainers.put(containerId, rmContainer);
 
           // Deduct pending resource requests
-          List<ResourceRequest> requests = appSchedulingInfo.allocate(
+          ContainerRequest containerRequest = appSchedulingInfo.allocate(
               allocation.getAllocationLocalityType(),
               schedulerContainer.getSchedulerNode(),
               schedulerContainer.getSchedulerRequestKey(),
               schedulerContainer.getRmContainer().getContainer());
-          ((RMContainerImpl) rmContainer).setResourceRequests(requests);
+          ((RMContainerImpl) rmContainer).setContainerRequest(containerRequest);
 
           attemptResourceUsage.incUsed(schedulerContainer.getNodePartition(),
               allocation.getAllocatedOrReservedResource());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.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/FSAppAttempt.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/FSAppAttempt.java
index 43daace..e095a42 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/FSAppAttempt.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/FSAppAttempt.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@@ -460,13 +461,13 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       liveContainers.put(container.getId(), rmContainer);
 
       // Update consumption and track allocations
-      List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
+      ContainerRequest containerRequest = appSchedulingInfo.allocate(
           type, node, schedulerKey, container);
       this.attemptResourceUsage.incUsed(container.getResource());
       getQueue().incUsedResource(container.getResource());
 
       // Update resource requests related to "request" and store in RMContainer
-      ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
+      ((RMContainerImpl) rmContainer).setContainerRequest(containerRequest);
 
       // Inform the container
       rmContainer.handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java
index d932e0e..169b98a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 
@@ -80,14 +81,14 @@ public class FifoAppAttempt extends FiCaSchedulerApp {
       liveContainers.put(containerId, rmContainer);
 
       // Update consumption and track allocations
-      List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
+      ContainerRequest containerRequest = appSchedulingInfo.allocate(
           type, node, schedulerKey, container);
 
       attemptResourceUsage.incUsed(node.getPartition(),
           container.getResource());
 
       // Update resource requests related to "request" and store in RMContainer
-      ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
+      ((RMContainerImpl) rmContainer).setContainerRequest(containerRequest);
 
       // Inform the container
       rmContainer.handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.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/placement/AppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
index 63b22a3..dcb38aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
@@ -57,14 +58,14 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
   Iterator<N> getPreferredNodeIterator(CandidateNodeSet<N> candidateNodeSet);
 
   /**
-   * Replace existing ResourceRequest by the new requests
+   * Replace existing pending asks by the new requests
    *
-   * @param requests new ResourceRequests
+   * @param requests new asks
    * @param recoverPreemptedRequestForAContainer if we're recovering resource
    * requests for preempted container
    * @return true if total pending resource changed
    */
-  ResourceRequestUpdateResult updateResourceRequests(
+  PendingAskUpdateResult updatePendingAsk(
       Collection<ResourceRequest> requests,
       boolean recoverPreemptedRequestForAContainer);
 
@@ -97,18 +98,14 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    * @param schedulerKey SchedulerRequestKey for this ResourceRequest
    * @param type Type of the allocation
    * @param node Which node this container allocated on
-   * @return list of ResourceRequests deducted
+   * @return ContainerRequest which include resource requests associated with
+   *         the container. This will be used by scheduler to recover requests.
+   *         Please refer to {@link ContainerRequest} for more details.
    */
-  List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
+  ContainerRequest allocate(SchedulerRequestKey schedulerKey,
       NodeType type, SchedulerNode node);
 
   /**
-   * Returns list of accepted resourceNames.
-   * @return Iterator of accepted resourceNames
-   */
-  Iterator<String> getAcceptedResouceNames();
-
-  /**
    * We can still have pending requirement for a given NodeType and node
    * @param type Locality Type
    * @param node which node we will allocate on

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.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/placement/LocalityAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
index 7f89435..766827c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
@@ -44,7 +45,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
  * into account locality preferences (node, rack, any) when allocating
  * containers.
  */
-public class LocalityAppPlacementAllocator<N extends SchedulerNode>
+public class LocalityAppPlacementAllocator <N extends SchedulerNode>
     implements AppPlacementAllocator<N> {
   private static final Log LOG =
       LogFactory.getLog(LocalityAppPlacementAllocator.class);
@@ -122,13 +123,13 @@ public class LocalityAppPlacementAllocator<N extends SchedulerNode>
   }
 
   @Override
-  public ResourceRequestUpdateResult updateResourceRequests(
+  public PendingAskUpdateResult updatePendingAsk(
       Collection<ResourceRequest> requests,
       boolean recoverPreemptedRequestForAContainer) {
     try {
       this.writeLock.lock();
 
-      ResourceRequestUpdateResult updateResult = null;
+      PendingAskUpdateResult updateResult = null;
 
       // Update resource requests
       for (ResourceRequest request : requests) {
@@ -156,7 +157,16 @@ public class LocalityAppPlacementAllocator<N extends SchedulerNode>
           //update the applications requested labels set
           appSchedulingInfo.addRequestedPartition(partition);
 
-          updateResult = new ResourceRequestUpdateResult(lastRequest, request);
+          PendingAsk lastPendingAsk =
+              lastRequest == null ? null : new PendingAsk(
+                  lastRequest.getCapability(), lastRequest.getNumContainers());
+          String lastRequestedNodePartition =
+              lastRequest == null ? null : lastRequest.getNodeLabelExpression();
+
+          updateResult = new PendingAskUpdateResult(lastPendingAsk,
+              new PendingAsk(request.getCapability(),
+                  request.getNumContainers()), lastRequestedNodePartition,
+              request.getNodeLabelExpression());
         }
       }
       return updateResult;
@@ -380,7 +390,7 @@ public class LocalityAppPlacementAllocator<N extends SchedulerNode>
   }
 
   @Override
-  public List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
+  public ContainerRequest allocate(SchedulerRequestKey schedulerKey,
       NodeType type, SchedulerNode node) {
     try {
       writeLock.lock();
@@ -404,19 +414,9 @@ public class LocalityAppPlacementAllocator<N extends SchedulerNode>
         allocateOffSwitch(schedulerKey, request, resourceRequests);
       }
 
-      return resourceRequests;
+      return new ContainerRequest(resourceRequests);
     } finally {
       writeLock.unlock();
     }
   }
-
-  @Override
-  public Iterator<String> getAcceptedResouceNames() {
-    try {
-      readLock.lock();
-      return resourceRequestMap.keySet().iterator();
-    } finally {
-      readLock.unlock();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PendingAskUpdateResult.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/placement/PendingAskUpdateResult.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PendingAskUpdateResult.java
new file mode 100644
index 0000000..8765e86
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PendingAskUpdateResult.java
@@ -0,0 +1,65 @@
+/**
+ * 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.scheduler.placement;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
+
+/**
+ * Result of a resource-request update. This will be used by
+ * {@link org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo}
+ * to update queue metrics and application/queue's overall pending resources.
+ * And this is per-scheduler-key.
+ *
+ * Following fields will be set if pending ask changed for a given scheduler key
+ * - lastPendingAsk: how many resource asked before.
+ * - newPendingAsk: how many resource asked now.
+ * - lastNodePartition: what's the node partition before.
+ * - newNodePartition: what's the node partition now.
+ */
+public class PendingAskUpdateResult {
+  private final PendingAsk lastPendingAsk;
+  private final String lastNodePartition;
+  private final PendingAsk newPendingAsk;
+  private final String newNodePartition;
+
+  public PendingAskUpdateResult(PendingAsk lastPendingAsk,
+      PendingAsk newPendingAsk, String lastNodePartition,
+      String newNodePartition) {
+    this.lastPendingAsk = lastPendingAsk;
+    this.newPendingAsk = newPendingAsk;
+    this.lastNodePartition = lastNodePartition;
+    this.newNodePartition = newNodePartition;
+  }
+
+  public PendingAsk getLastPendingAsk() {
+    return lastPendingAsk;
+  }
+
+  public PendingAsk getNewPendingAsk() {
+    return newPendingAsk;
+  }
+
+  public String getLastNodePartition() {
+    return lastNodePartition;
+  }
+
+  public String getNewNodePartition() {
+    return newNodePartition;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceRequestUpdateResult.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/placement/ResourceRequestUpdateResult.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceRequestUpdateResult.java
deleted file mode 100644
index da356f5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceRequestUpdateResult.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
-
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-
-/**
- * Result of ResourceRequest update
- */
-public class ResourceRequestUpdateResult {
-  private final ResourceRequest lastAnyResourceRequest;
-  private final ResourceRequest newResourceRequest;
-
-  public ResourceRequestUpdateResult(ResourceRequest lastAnyResourceRequest,
-      ResourceRequest newResourceRequest) {
-    this.lastAnyResourceRequest = lastAnyResourceRequest;
-    this.newResourceRequest = newResourceRequest;
-  }
-
-  public ResourceRequest getLastAnyResourceRequest() {
-    return lastAnyResourceRequest;
-  }
-
-  public ResourceRequest getNewResourceRequest() {
-    return newResourceRequest;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.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/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
index e889de0..fbde681 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
@@ -431,7 +431,7 @@ public class Application {
     if (type == NodeType.NODE_LOCAL) {
       for (String host : task.getHosts()) {
         if(LOG.isDebugEnabled()) {
-          LOG.debug("updateResourceRequests:" + " application=" + applicationId
+          LOG.debug("updatePendingAsk:" + " application=" + applicationId
             + " type=" + type + " host=" + host
             + " request=" + ((requests == null) ? "null" : requests.get(host)));
         }
@@ -442,7 +442,7 @@ public class Application {
     if (type == NodeType.NODE_LOCAL || type == NodeType.RACK_LOCAL) {
       for (String rack : task.getRacks()) {
         if(LOG.isDebugEnabled()) {
-          LOG.debug("updateResourceRequests:" + " application=" + applicationId
+          LOG.debug("updatePendingAsk:" + " application=" + applicationId
             + " type=" + type + " rack=" + rack
             + " request=" + ((requests == null) ? "null" : requests.get(rack)));
         }
@@ -453,7 +453,7 @@ public class Application {
     updateResourceRequest(requests.get(ResourceRequest.ANY));
     
     if(LOG.isDebugEnabled()) {
-      LOG.debug("updateResourceRequests:" + " application=" + applicationId
+      LOG.debug("updatePendingAsk:" + " application=" + applicationId
         + " #asks=" + ask.size());
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.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/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
index db31448..6c189b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
@@ -278,8 +278,8 @@ public class TestRMContainerImpl {
 
     // Verify whether list of ResourceRequest is present in RMContainer
     // while moving to ALLOCATED state
-    Assert.assertNotNull(scheduler.getRMContainer(containerId2)
-        .getResourceRequests());
+    Assert.assertNotNull(
+        scheduler.getRMContainer(containerId2).getContainerRequest());
 
     // Allocate container
     am1.allocate(new ArrayList<ResourceRequest>(), new ArrayList<ContainerId>())
@@ -288,8 +288,8 @@ public class TestRMContainerImpl {
 
     // After RMContainer moving to ACQUIRED state, list of ResourceRequest will
     // be empty
-    Assert.assertNull(scheduler.getRMContainer(containerId2)
-        .getResourceRequests());
+    Assert.assertNull(
+        scheduler.getRMContainer(containerId2).getContainerRequest());
   }
 
   @Test (timeout = 180000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/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 1edb0da..e91f734 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
@@ -1696,7 +1696,8 @@ public class TestCapacityScheduler {
     rm1.waitForState(nm1, containerId1, RMContainerState.ALLOCATED);
 
     RMContainer rmContainer = cs.getRMContainer(containerId1);
-    List<ResourceRequest> requests = rmContainer.getResourceRequests();
+    List<ResourceRequest> requests =
+        rmContainer.getContainerRequest().getResourceRequests();
     FiCaSchedulerApp app = cs.getApplicationAttempt(am1
         .getApplicationAttemptId());
 


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


[46/50] [abbrv] hadoop git commit: YARN-6595. [API] Add Placement Constraints at the application level. (Arun Suresh via kkaranasos)

Posted by wa...@apache.org.
YARN-6595. [API] Add Placement Constraints at the application level. (Arun Suresh via kkaranasos)


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

Branch: refs/heads/YARN-6592
Commit: c519f0dc1804e77ddc8bcbb37e3d04a30fdc2e0f
Parents: ef1e90d
Author: Konstantinos Karanasos <kk...@apache.org>
Authored: Mon Nov 13 15:25:24 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Sun Dec 10 10:27:01 2017 -0800

----------------------------------------------------------------------
 .../RegisterApplicationMasterRequest.java       |  42 ++++-
 .../yarn/api/resource/PlacementConstraint.java  | 156 +++++++++++++++++++
 .../src/main/proto/yarn_protos.proto            |   6 +
 .../src/main/proto/yarn_service_protos.proto    |   1 +
 .../RegisterApplicationMasterRequestPBImpl.java | 106 ++++++++++++-
 .../hadoop/yarn/api/BasePBImplRecordsTest.java  |  11 ++
 6 files changed, 313 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c519f0dc/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 395e190..f2d537a 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
@@ -18,11 +18,16 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
 import org.apache.hadoop.yarn.util.Records;
-
 /**
  * The request sent by the {@code ApplicationMaster} to {@code ResourceManager}
  * on registration.
@@ -132,4 +137,39 @@ public abstract class RegisterApplicationMasterRequest {
   @Public
   @Stable
   public abstract void setTrackingUrl(String trackingUrl);
+
+  /**
+   * Return all Placement Constraints specified at the Application level. The
+   * mapping is from a set of allocation tags to a
+   * <code>PlacementConstraint</code> associated with the tags, i.e., each
+   * {@link org.apache.hadoop.yarn.api.records.SchedulingRequest} that has those
+   * tags will be placed taking into account the corresponding constraint.
+   *
+   * @return A map of Placement Constraints.
+   */
+  @Public
+  @Unstable
+  public Map<Set<String>, PlacementConstraint> getPlacementConstraints() {
+    return new HashMap<>();
+  }
+
+  /**
+   * Set Placement Constraints applicable to the
+   * {@link org.apache.hadoop.yarn.api.records.SchedulingRequest}s
+   * of this application.
+   * The mapping is from a set of allocation tags to a
+   * <code>PlacementConstraint</code> associated with the tags.
+   * For example:
+   *  Map &lt;
+   *   &lt;hb_regionserver&gt; -&gt; node_anti_affinity,
+   *   &lt;hb_regionserver, hb_master&gt; -&gt; rack_affinity,
+   *   ...
+   *  &gt;
+   * @param placementConstraints Placement Constraint Mapping.
+   */
+  @Public
+  @Unstable
+  public void setPlacementConstraints(
+      Map<Set<String>, PlacementConstraint> placementConstraints) {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c519f0dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
index f0e3982..b6e851a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
@@ -54,6 +54,26 @@ public class PlacementConstraint {
     return constraintExpr;
   }
 
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof PlacementConstraint)) {
+      return false;
+    }
+
+    PlacementConstraint that = (PlacementConstraint) o;
+
+    return getConstraintExpr() != null ? getConstraintExpr().equals(that
+        .getConstraintExpr()) : that.getConstraintExpr() == null;
+  }
+
+  @Override
+  public int hashCode() {
+    return getConstraintExpr() != null ? getConstraintExpr().hashCode() : 0;
+  }
+
   /**
    * Interface used to enable the elements of the constraint tree to be visited.
    */
@@ -174,6 +194,38 @@ public class PlacementConstraint {
     }
 
     @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof SingleConstraint)) {
+        return false;
+      }
+
+      SingleConstraint that = (SingleConstraint) o;
+
+      if (getMinCardinality() != that.getMinCardinality()) {
+        return false;
+      }
+      if (getMaxCardinality() != that.getMaxCardinality()) {
+        return false;
+      }
+      if (!getScope().equals(that.getScope())) {
+        return false;
+      }
+      return getTargetExpressions().equals(that.getTargetExpressions());
+    }
+
+    @Override
+    public int hashCode() {
+      int result = getScope().hashCode();
+      result = 31 * result + getMinCardinality();
+      result = 31 * result + getMaxCardinality();
+      result = 31 * result + getTargetExpressions().hashCode();
+      return result;
+    }
+
+    @Override
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
@@ -332,6 +384,34 @@ public class PlacementConstraint {
     }
 
     @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof TargetConstraint)) {
+        return false;
+      }
+
+      TargetConstraint that = (TargetConstraint) o;
+
+      if (getOp() != that.getOp()) {
+        return false;
+      }
+      if (!getScope().equals(that.getScope())) {
+        return false;
+      }
+      return getTargetExpressions().equals(that.getTargetExpressions());
+    }
+
+    @Override
+    public int hashCode() {
+      int result = getOp().hashCode();
+      result = 31 * result + getScope().hashCode();
+      result = 31 * result + getTargetExpressions().hashCode();
+      return result;
+    }
+
+    @Override
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
@@ -388,6 +468,34 @@ public class PlacementConstraint {
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      CardinalityConstraint that = (CardinalityConstraint) o;
+
+      if (minCardinality != that.minCardinality) {
+        return false;
+      }
+      if (maxCardinality != that.maxCardinality) {
+        return false;
+      }
+      return scope != null ? scope.equals(that.scope) : that.scope == null;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = scope != null ? scope.hashCode() : 0;
+      result = 31 * result + minCardinality;
+      result = 31 * result + maxCardinality;
+      return result;
+    }
   }
 
   /**
@@ -406,6 +514,25 @@ public class PlacementConstraint {
      * @return the children of the composite constraint
      */
     public abstract List<R> getChildren();
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      return getChildren() != null ? getChildren().equals(
+          ((CompositeConstraint)o).getChildren()) :
+          ((CompositeConstraint)o).getChildren() == null;
+    }
+
+    @Override
+    public int hashCode() {
+      return getChildren() != null ? getChildren().hashCode() : 0;
+    }
   }
 
   /**
@@ -563,5 +690,34 @@ public class PlacementConstraint {
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      TimedPlacementConstraint that = (TimedPlacementConstraint) o;
+
+      if (schedulingDelay != that.schedulingDelay) {
+        return false;
+      }
+      if (constraint != null ? !constraint.equals(that.constraint) :
+          that.constraint != null) {
+        return false;
+      }
+      return delayUnit == that.delayUnit;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = constraint != null ? constraint.hashCode() : 0;
+      result = 31 * result + (int) (schedulingDelay ^ (schedulingDelay >>> 32));
+      result = 31 * result + (delayUnit != null ? delayUnit.hashCode() : 0);
+      return result;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c519f0dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 2dbdefb..ac43d1b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -648,6 +648,12 @@ message CompositePlacementConstraintProto {
   repeated TimedPlacementConstraintProto timedChildConstraints = 3;
 }
 
+// This associates a set of allocation tags to a Placement Constraint.
+message PlacementConstraintMapEntryProto {
+  repeated string allocation_tags = 1;
+  optional PlacementConstraintProto placement_constraint = 2;
+}
+
 ////////////////////////////////////////////////////////////////////////
 ////// From reservation_protocol /////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c519f0dc/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 4e97c74..68e585d 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
@@ -38,6 +38,7 @@ message RegisterApplicationMasterRequestProto {
   optional string host = 1;
   optional int32 rpc_port = 2;
   optional string tracking_url = 3;
+  repeated PlacementConstraintMapEntryProto placement_constraints = 4;
 }
 
 message RegisterApplicationMasterResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c519f0dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java
index 037dfd9..64bee85 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java
@@ -21,24 +21,41 @@ package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintFromProtoConverter;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintToProtoConverter;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProtoOrBuilder;
 
 import com.google.protobuf.TextFormat;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 @Private
 @Unstable
-public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationMasterRequest {
-  RegisterApplicationMasterRequestProto proto = RegisterApplicationMasterRequestProto.getDefaultInstance();
-  RegisterApplicationMasterRequestProto.Builder builder = null;
+public class RegisterApplicationMasterRequestPBImpl
+    extends RegisterApplicationMasterRequest {
+  private RegisterApplicationMasterRequestProto proto =
+      RegisterApplicationMasterRequestProto.getDefaultInstance();
+  private RegisterApplicationMasterRequestProto.Builder builder = null;
+  private Map<Set<String>, PlacementConstraint> placementConstraints = null;
   boolean viaProto = false;
   
   public RegisterApplicationMasterRequestPBImpl() {
     builder = RegisterApplicationMasterRequestProto.newBuilder();
   }
 
-  public RegisterApplicationMasterRequestPBImpl(RegisterApplicationMasterRequestProto proto) {
+  public RegisterApplicationMasterRequestPBImpl(
+      RegisterApplicationMasterRequestProto proto) {
     this.proto = proto;
     viaProto = true;
   }
@@ -71,6 +88,30 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
   }
 
   private void mergeLocalToBuilder() {
+    if (this.placementConstraints != null) {
+      addPlacementConstraintMap();
+    }
+  }
+
+  private void addPlacementConstraintMap() {
+    maybeInitBuilder();
+    builder.clearPlacementConstraints();
+    if (this.placementConstraints == null) {
+      return;
+    }
+    List<YarnProtos.PlacementConstraintMapEntryProto> protoList =
+        new ArrayList<>();
+    for (Map.Entry<Set<String>, PlacementConstraint> entry :
+        this.placementConstraints.entrySet()) {
+      protoList.add(
+          YarnProtos.PlacementConstraintMapEntryProto.newBuilder()
+              .addAllAllocationTags(entry.getKey())
+              .setPlacementConstraint(
+                  new PlacementConstraintToProtoConverter(
+                      entry.getValue()).convert())
+              .build());
+    }
+    builder.addAllPlacementConstraints(protoList);
   }
 
   private void mergeLocalToProto() {
@@ -90,7 +131,8 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
 
   @Override
   public String getHost() {
-    RegisterApplicationMasterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    RegisterApplicationMasterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
     return p.getHost();
   }
 
@@ -106,7 +148,8 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
 
   @Override
   public int getRpcPort() {
-    RegisterApplicationMasterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    RegisterApplicationMasterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
     return p.getRpcPort();
   }
 
@@ -118,7 +161,8 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
 
   @Override
   public String getTrackingUrl() {
-    RegisterApplicationMasterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    RegisterApplicationMasterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
     return p.getTrackingUrl();
   }
 
@@ -131,4 +175,50 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
     }
     builder.setTrackingUrl(url);
   }
-}  
+
+  private void initPlacementConstraintMap() {
+    if (this.placementConstraints != null) {
+      return;
+    }
+    RegisterApplicationMasterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<YarnProtos.PlacementConstraintMapEntryProto> pcmList =
+        p.getPlacementConstraintsList();
+    this.placementConstraints = new HashMap<>();
+    for (YarnProtos.PlacementConstraintMapEntryProto e : pcmList) {
+      this.placementConstraints.put(
+          new HashSet<>(e.getAllocationTagsList()),
+          new PlacementConstraintFromProtoConverter(
+              e.getPlacementConstraint()).convert());
+    }
+  }
+
+  @Override
+  public Map<Set<String>, PlacementConstraint> getPlacementConstraints() {
+    initPlacementConstraintMap();
+    return this.placementConstraints;
+  }
+
+  @Override
+  public void setPlacementConstraints(
+      Map<Set<String>, PlacementConstraint> constraints) {
+    maybeInitBuilder();
+    if (constraints == null) {
+      builder.clearPlacementConstraints();
+    } else {
+      removeEmptyKeys(constraints);
+    }
+    this.placementConstraints = constraints;
+  }
+
+  private void removeEmptyKeys(
+      Map<Set<String>, PlacementConstraint> constraintMap) {
+    Iterator<Set<String>> iter = constraintMap.keySet().iterator();
+    while (iter.hasNext()) {
+      Set<String> aTags = iter.next();
+      if (aTags.size() == 0) {
+        iter.remove();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c519f0dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
index 8694651..ebd66af 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
@@ -22,12 +22,19 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.junit.Assert;
 
 import java.lang.reflect.*;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints
+    .PlacementTargets.allocationTag;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+
 /**
  * Generic helper class to validate protocol records.
  */
@@ -85,6 +92,10 @@ public class BasePBImplRecordsTest {
         ByteBuffer buff = ByteBuffer.allocate(4);
         rand.nextBytes(buff.array());
         return buff;
+      } else if (type.equals(PlacementConstraint.class)) {
+        PlacementConstraint.AbstractConstraint sConstraintExpr =
+            targetIn(NODE, allocationTag("foo"));
+        ret = PlacementConstraints.build(sConstraintExpr);
       }
     } else if (type instanceof ParameterizedType) {
       ParameterizedType pt = (ParameterizedType)type;


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


[21/50] [abbrv] hadoop git commit: YARN-7254. UI and metrics changes related to absolute resource configuration. (Sunil G via wangda)

Posted by wa...@apache.org.
YARN-7254. UI and metrics changes related to absolute resource configuration. (Sunil G via wangda)


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

Branch: refs/heads/YARN-6592
Commit: d52627a7cbddfd981db973e223aefffde1ebf82d
Parents: 5e798b1
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Oct 16 11:04:05 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../yarn/api/records/QueueConfigurations.java   |  76 +++++++++++
 .../yarn/api/records/ResourceInformation.java   |   6 +
 .../src/main/proto/yarn_protos.proto            |   4 +
 .../impl/pb/QueueConfigurationsPBImpl.java      | 136 ++++++++++++++++++-
 .../resource/DominantResourceCalculator.java    |  24 +++-
 .../scheduler/QueueResourceQuotas.java          |  38 ------
 .../scheduler/capacity/AbstractCSQueue.java     |  18 ++-
 .../scheduler/capacity/CSQueue.java             |   4 +-
 .../scheduler/capacity/LeafQueue.java           |   4 +-
 .../scheduler/capacity/ParentQueue.java         |  56 ++++++--
 .../scheduler/capacity/UsersManager.java        |   2 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  23 +++-
 .../webapp/CapacitySchedulerPage.java           |  49 +++++--
 .../webapp/dao/CapacitySchedulerInfo.java       |   3 +-
 .../dao/CapacitySchedulerLeafQueueInfo.java     |   6 +-
 .../webapp/dao/CapacitySchedulerQueueInfo.java  |  10 +-
 .../dao/PartitionQueueCapacitiesInfo.java       |  34 ++++-
 .../webapp/dao/QueueCapacitiesInfo.java         |  13 +-
 ...CapacitySchedulerWithMultiResourceTypes.java |  25 ++--
 .../scheduler/capacity/TestLeafQueue.java       |  24 +++-
 20 files changed, 443 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueConfigurations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueConfigurations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueConfigurations.java
index e25c8aa..30096bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueConfigurations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueConfigurations.java
@@ -147,4 +147,80 @@ public abstract class QueueConfigurations {
   @Private
   @Unstable
   public abstract void setMaxAMPercentage(float maxAMPercentage);
+
+  /**
+   * Get the effective minimum capacity of queue (from absolute resource).
+   *
+   * @return minimum resource capability
+   */
+  @Public
+  @Unstable
+  public abstract Resource getEffectiveMinCapacity();
+
+  /**
+   * Set the effective minimum capacity of queue (from absolute resource).
+   *
+   * @param capacity
+   *          minimum resource capability
+   */
+  @Private
+  @Unstable
+  public abstract void setEffectiveMinCapacity(Resource capacity);
+
+  /**
+   * Get the effective maximum capacity of queue (from absolute resource).
+   *
+   * @return maximum resource capability
+   */
+  @Public
+  @Unstable
+  public abstract Resource getEffectiveMaxCapacity();
+
+  /**
+   * Set the effective maximum capacity of queue (from absolute resource).
+   *
+   * @param capacity
+   *          maximum resource capability
+   */
+  @Private
+  @Unstable
+  public abstract void setEffectiveMaxCapacity(Resource capacity);
+
+  /**
+   * Get the configured minimum capacity of queue (from absolute resource).
+   *
+   * @return minimum resource capability
+   */
+  @Public
+  @Unstable
+  public abstract Resource getConfiguredMinCapacity();
+
+  /**
+   * Set the configured minimum capacity of queue (from absolute resource).
+   *
+   * @param configuredMinResource
+   *          minimum resource capability
+   */
+  @Public
+  @Unstable
+  public abstract void setConfiguredMinCapacity(Resource configuredMinResource);
+
+  /**
+   * Get the configured maximum capacity of queue (from absolute resource).
+   *
+   * @return maximum resource capability
+   */
+  @Public
+  @Unstable
+  public abstract Resource getConfiguredMaxCapacity();
+
+  /**
+   * Set the configured maximum capacity of queue (from absolute resource).
+   *
+   * @param configuredMaxResource
+   *          maximum resource capability
+   */
+  @Public
+  @Unstable
+  public abstract void setConfiguredMaxCapacity(Resource configuredMaxResource);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
index a8198d8..904ff4b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
@@ -228,6 +228,12 @@ public class ResourceInformation implements Comparable<ResourceInformation> {
         Long.MAX_VALUE);
   }
 
+  public static ResourceInformation newInstance(String name, String units,
+      long minRes, long maxRes) {
+    return ResourceInformation.newInstance(name, units, 0L,
+        ResourceTypes.COUNTABLE, minRes, maxRes);
+  }
+
   public static ResourceInformation newInstance(String name, long value) {
     return ResourceInformation
         .newInstance(name, "", value, ResourceTypes.COUNTABLE, 0L,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index fdbe2d4..3a9662b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -551,6 +551,10 @@ message QueueConfigurationsProto {
     optional float maxCapacity = 3;
     optional float absoluteMaxCapacity = 4;
     optional float maxAMPercentage = 5;
+    optional ResourceProto effectiveMinCapacity = 6;
+    optional ResourceProto effectiveMaxCapacity = 7;
+    optional ResourceProto configuredMinCapacity = 8;
+    optional ResourceProto configuredMaxCapacity = 9;
 }
 
 message QueueConfigurationsMapProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java
index f308bce..80ef4b2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java
@@ -19,16 +19,22 @@
 package org.apache.hadoop.yarn.api.records.impl.pb;
 
 import org.apache.hadoop.yarn.api.records.QueueConfigurations;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueConfigurationsProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueConfigurationsProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 
 import com.google.protobuf.TextFormat;
 
 public class QueueConfigurationsPBImpl extends QueueConfigurations {
 
-  QueueConfigurationsProto proto =
-      QueueConfigurationsProto.getDefaultInstance();
+  QueueConfigurationsProto proto = QueueConfigurationsProto
+      .getDefaultInstance();
   QueueConfigurationsProto.Builder builder = null;
+  Resource configuredMinResource = null;
+  Resource configuredMaxResource = null;
+  Resource effMinResource = null;
+  Resource effMaxResource = null;
   boolean viaProto = false;
 
   public QueueConfigurationsPBImpl() {
@@ -41,11 +47,40 @@ public class QueueConfigurationsPBImpl extends QueueConfigurations {
   }
 
   public QueueConfigurationsProto getProto() {
+    mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
   }
 
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.effMinResource != null) {
+      builder
+          .setEffectiveMinCapacity(convertToProtoFormat(this.effMinResource));
+    }
+    if (this.effMaxResource != null) {
+      builder
+          .setEffectiveMaxCapacity(convertToProtoFormat(this.effMaxResource));
+    }
+    if (this.configuredMinResource != null) {
+      builder.setEffectiveMinCapacity(
+          convertToProtoFormat(this.configuredMinResource));
+    }
+    if (this.configuredMaxResource != null) {
+      builder.setEffectiveMaxCapacity(
+          convertToProtoFormat(this.configuredMaxResource));
+    }
+  }
+
   @Override
   public float getCapacity() {
     QueueConfigurationsProtoOrBuilder p = viaProto ? proto : builder;
@@ -106,6 +141,58 @@ public class QueueConfigurationsPBImpl extends QueueConfigurations {
     builder.setMaxAMPercentage(maxAMPercentage);
   }
 
+  @Override
+  public Resource getEffectiveMinCapacity() {
+    QueueConfigurationsProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.effMinResource != null) {
+      return this.effMinResource;
+    }
+    if (!p.hasEffectiveMinCapacity()) {
+      return null;
+    }
+    this.effMinResource = convertFromProtoFormat(p.getEffectiveMinCapacity());
+    return this.effMinResource;
+  }
+
+  @Override
+  public void setEffectiveMinCapacity(Resource capacity) {
+    maybeInitBuilder();
+    if (capacity == null) {
+      builder.clearEffectiveMinCapacity();
+    }
+    this.effMinResource = capacity;
+  }
+
+  @Override
+  public Resource getEffectiveMaxCapacity() {
+    QueueConfigurationsProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.effMaxResource != null) {
+      return this.effMaxResource;
+    }
+    if (!p.hasEffectiveMaxCapacity()) {
+      return null;
+    }
+    this.effMaxResource = convertFromProtoFormat(p.getEffectiveMaxCapacity());
+    return this.effMaxResource;
+  }
+
+  @Override
+  public void setEffectiveMaxCapacity(Resource capacity) {
+    maybeInitBuilder();
+    if (capacity == null) {
+      builder.clearEffectiveMaxCapacity();
+    }
+    this.effMaxResource = capacity;
+  }
+
+  private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource t) {
+    return ProtoUtils.convertToProtoFormat(t);
+  }
+
   private void maybeInitBuilder() {
     if (viaProto || builder == null) {
       builder = QueueConfigurationsProto.newBuilder(proto);
@@ -134,4 +221,49 @@ public class QueueConfigurationsPBImpl extends QueueConfigurations {
     return false;
   }
 
+  @Override
+  public Resource getConfiguredMinCapacity() {
+    QueueConfigurationsProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.configuredMinResource != null) {
+      return this.configuredMinResource;
+    }
+    if (!p.hasConfiguredMinCapacity()) {
+      return null;
+    }
+    this.configuredMinResource = convertFromProtoFormat(
+        p.getConfiguredMinCapacity());
+    return this.configuredMinResource;
+  }
+
+  @Override
+  public void setConfiguredMinCapacity(Resource configuredMinResource) {
+    maybeInitBuilder();
+    if (configuredMinResource == null) {
+      builder.clearConfiguredMinCapacity();
+    }
+    this.configuredMinResource = configuredMinResource;
+  }
+
+  @Override
+  public Resource getConfiguredMaxCapacity() {
+    QueueConfigurationsProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.configuredMaxResource != null) {
+      return this.configuredMaxResource;
+    }
+    if (!p.hasConfiguredMaxCapacity()) {
+      return null;
+    }
+    this.configuredMaxResource = convertFromProtoFormat(
+        p.getConfiguredMaxCapacity());
+    return this.configuredMaxResource;
+  }
+
+  @Override
+  public void setConfiguredMaxCapacity(Resource configuredMaxResource) {
+    maybeInitBuilder();
+    if (configuredMaxResource == null) {
+      builder.clearConfiguredMaxCapacity();
+    }
+    this.configuredMaxResource = configuredMaxResource;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index 16e4527..cc4acf6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -570,8 +570,26 @@ public class DominantResourceCalculator extends ResourceCalculator {
 
   @Override
   public Resource normalizeDown(Resource r, Resource stepFactor) {
-    return Resources.createResource(
-        roundDown(r.getMemorySize(), stepFactor.getMemorySize()),
-        roundDown(r.getVirtualCores(), stepFactor.getVirtualCores()));
+    Resource ret = Resource.newInstance(r);
+    int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
+    for (int i = 0; i < maxLength; i++) {
+      ResourceInformation rResourceInformation = r.getResourceInformation(i);
+      ResourceInformation stepFactorResourceInformation = stepFactor
+          .getResourceInformation(i);
+      ResourceInformation tmp = ret.getResourceInformation(i);
+
+      long rValue = rResourceInformation.getValue();
+      long stepFactorValue = UnitsConversionUtil.convert(
+          stepFactorResourceInformation.getUnits(),
+          rResourceInformation.getUnits(),
+          stepFactorResourceInformation.getValue());
+
+      long value = rValue;
+      if (stepFactorValue != 0) {
+        value = roundDown(rValue, stepFactorValue);
+      }
+      tmp.setValue(value);
+    }
+    return ret;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueResourceQuotas.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/QueueResourceQuotas.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueResourceQuotas.java
index 2e653fc..08b4d04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueResourceQuotas.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueResourceQuotas.java
@@ -112,42 +112,4 @@ public class QueueResourceQuotas extends AbstractResourceUsage {
   public void setEffectiveMaxResource(String label, Resource res) {
     _set(label, ResourceType.EFF_MAX_RESOURCE, res);
   }
-
-  /*
-   * Effective Minimum Resource
-   */
-  public Resource getEffectiveMinResourceUp() {
-    return _get(NL, ResourceType.EFF_MIN_RESOURCE_UP);
-  }
-
-  public Resource getEffectiveMinResourceUp(String label) {
-    return _get(label, ResourceType.EFF_MIN_RESOURCE_UP);
-  }
-
-  public void setEffectiveMinResourceUp(String label, Resource res) {
-    _set(label, ResourceType.EFF_MIN_RESOURCE_UP, res);
-  }
-
-  public void setEffectiveMinResourceUp(Resource res) {
-    _set(NL, ResourceType.EFF_MIN_RESOURCE_UP, res);
-  }
-
-  /*
-   * Effective Maximum Resource
-   */
-  public Resource getEffectiveMaxResourceUp() {
-    return getEffectiveMaxResourceUp(NL);
-  }
-
-  public Resource getEffectiveMaxResourceUp(String label) {
-    return _get(label, ResourceType.EFF_MAX_RESOURCE_UP);
-  }
-
-  public void setEffectiveMaxResourceUp(Resource res) {
-    setEffectiveMaxResourceUp(NL, res);
-  }
-
-  public void setEffectiveMaxResourceUp(String label, Resource res) {
-    _set(label, ResourceType.EFF_MAX_RESOURCE_UP, res);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/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 dddac4d..9caf589 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
@@ -398,6 +398,10 @@ public abstract class AbstractCSQueue implements CSQueue {
       Resource maxResource = conf.getMaximumResourceRequirement(label,
           queuePath, resourceTypes);
 
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("capacityConfigType is '" + capacityConfigType
+            + "' for queue '" + getQueueName());
+      }
       if (this.capacityConfigType.equals(CapacityConfigType.NONE)) {
         this.capacityConfigType = (!minResource.equals(Resources.none())
             && queueCapacities.getAbsoluteCapacity(label) == 0f)
@@ -481,12 +485,6 @@ public abstract class AbstractCSQueue implements CSQueue {
   }
 
   @Override
-  public Resource getEffectiveCapacityUp(String label) {
-    return Resources
-        .clone(getQueueResourceQuotas().getEffectiveMinResourceUp(label));
-  }
-
-  @Override
   public Resource getEffectiveCapacityDown(String label, Resource factor) {
     return Resources.normalizeDown(resourceCalculator,
         getQueueResourceQuotas().getEffectiveMinResource(label),
@@ -621,6 +619,14 @@ public abstract class AbstractCSQueue implements CSQueue {
       queueConfiguration.setMaxCapacity(maxCapacity);
       queueConfiguration.setAbsoluteMaxCapacity(absMaxCapacity);
       queueConfiguration.setMaxAMPercentage(maxAMPercentage);
+      queueConfiguration.setConfiguredMinCapacity(
+          queueResourceQuotas.getConfiguredMinResource(nodeLabel));
+      queueConfiguration.setConfiguredMaxCapacity(
+          queueResourceQuotas.getConfiguredMaxResource(nodeLabel));
+      queueConfiguration.setEffectiveMinCapacity(
+          queueResourceQuotas.getEffectiveMinResource(nodeLabel));
+      queueConfiguration.setEffectiveMaxCapacity(
+          queueResourceQuotas.getEffectiveMaxResource(nodeLabel));
       queueConfigurations.put(nodeLabel, queueConfiguration);
     }
     return queueConfigurations;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/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 2e29a71..624fcc7 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
@@ -25,7 +25,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.security.AccessControlException;
@@ -52,6 +51,8 @@ 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.server.resourcemanager.scheduler.placement.CandidateNodeSet;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * <code>CSQueue</code> represents a node in the tree of 
  * hierarchical queues in the {@link CapacityScheduler}.
@@ -380,7 +381,6 @@ public interface CSQueue extends SchedulerQueue<CSQueue> {
    * @return effective queue capacity
    */
   Resource getEffectiveCapacity(String label);
-  Resource getEffectiveCapacityUp(String label);
   Resource getEffectiveCapacityDown(String label, Resource factor);
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/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 1460121..41ec4ba 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
@@ -671,7 +671,7 @@ public class LeafQueue extends AbstractCSQueue {
           1.0f / Math.max(getAbstractUsersManager().getNumActiveUsers(), 1));
       effectiveUserLimit = Math.min(effectiveUserLimit * userWeight, 1.0f);
 
-      Resource queuePartitionResource = getEffectiveCapacityUp(nodePartition);
+      Resource queuePartitionResource = getEffectiveCapacity(nodePartition);
 
       Resource userAMLimit = Resources.multiplyAndNormalizeUp(
           resourceCalculator, queuePartitionResource,
@@ -700,7 +700,7 @@ public class LeafQueue extends AbstractCSQueue {
        * non-labeled), * with per-partition am-resource-percent to get the max am
        * resource limit for this queue and partition.
        */
-      Resource queuePartitionResource = getEffectiveCapacityUp(nodePartition);
+      Resource queuePartitionResource = getEffectiveCapacity(nodePartition);
 
       Resource queueCurrentLimit = Resources.none();
       // For non-labeled partition, we need to consider the current queue

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/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 c770fac..5ab1494 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
@@ -914,9 +914,11 @@ public class ParentQueue extends AbstractCSQueue {
       queueResourceQuotas.setConfiguredMaxResource(label, resourceByLabel);
       queueResourceQuotas.setEffectiveMinResource(label, resourceByLabel);
       queueResourceQuotas.setEffectiveMaxResource(label, resourceByLabel);
+      queueCapacities.setAbsoluteCapacity(label, 1.0f);
     }
 
-    // Total configured min resources of direct children of queue
+    // Total configured min resources of direct children of this given parent
+    // queue.
     Resource configuredMinResources = Resource.newInstance(0L, 0);
     for (CSQueue childQueue : getChildQueues()) {
       Resources.addTo(configuredMinResources,
@@ -960,7 +962,7 @@ public class ParentQueue extends AbstractCSQueue {
         // present could also be taken from effective max resource of parent.
         Resource parentMaxRes = queueResourceQuotas
             .getConfiguredMaxResource(label);
-        if (parentMaxRes.equals(Resources.none())) {
+        if (parent != null && parentMaxRes.equals(Resources.none())) {
           parentMaxRes = parent.getQueueResourceQuotas()
               .getEffectiveMaxResource(label);
         }
@@ -976,6 +978,11 @@ public class ParentQueue extends AbstractCSQueue {
             parentMaxRes);
         childQueue.getQueueResourceQuotas().setEffectiveMaxResource(label,
             Resources.clone(effMaxResource));
+
+        // In cases where we still need to update some units based on
+        // percentage, we have to calculate percentage and update.
+        deriveCapacityFromAbsoluteConfigurations(label, clusterResource, rc,
+            childQueue);
       } else {
         childQueue.getQueueResourceQuotas().setEffectiveMinResource(label,
             Resources.multiply(resourceByLabel,
@@ -983,16 +990,6 @@ public class ParentQueue extends AbstractCSQueue {
         childQueue.getQueueResourceQuotas().setEffectiveMaxResource(label,
             Resources.multiply(resourceByLabel, childQueue.getQueueCapacities()
                 .getAbsoluteMaximumCapacity(label)));
-
-        childQueue.getQueueResourceQuotas().setEffectiveMinResourceUp(label,
-            Resources.multiplyAndNormalizeUp(rc, resourceByLabel,
-                childQueue.getQueueCapacities().getAbsoluteCapacity(label),
-                minimumAllocation));
-        childQueue.getQueueResourceQuotas().setEffectiveMaxResourceUp(label,
-            Resources.multiplyAndNormalizeUp(rc,
-                resourceByLabel, childQueue.getQueueCapacities()
-                    .getAbsoluteMaximumCapacity(label),
-                    minimumAllocation));
       }
 
       if (LOG.isDebugEnabled()) {
@@ -1006,6 +1003,41 @@ public class ParentQueue extends AbstractCSQueue {
     }
   }
 
+  private void deriveCapacityFromAbsoluteConfigurations(String label,
+      Resource clusterResource, ResourceCalculator rc, CSQueue childQueue) {
+
+    /*
+     * In case when queues are configured with absolute resources, it is better
+     * to update capacity/max-capacity etc w.r.t absolute resource as well. In
+     * case of computation, these values wont be used any more. However for
+     * metrics and UI, its better these values are pre-computed here itself.
+     */
+
+    // 1. Update capacity as a float based on parent's minResource
+    childQueue.getQueueCapacities().setCapacity(label,
+        rc.divide(clusterResource,
+            childQueue.getQueueResourceQuotas().getEffectiveMinResource(label),
+            getQueueResourceQuotas().getEffectiveMinResource(label)));
+
+    // 2. Update max-capacity as a float based on parent's maxResource
+    childQueue.getQueueCapacities().setMaximumCapacity(label,
+        rc.divide(clusterResource,
+            childQueue.getQueueResourceQuotas().getEffectiveMaxResource(label),
+            getQueueResourceQuotas().getEffectiveMaxResource(label)));
+
+    // 3. Update absolute capacity as a float based on parent's minResource and
+    // cluster resource.
+    childQueue.getQueueCapacities().setAbsoluteCapacity(label,
+        (float) childQueue.getQueueCapacities().getCapacity()
+            / getQueueCapacities().getAbsoluteCapacity(label));
+
+    // 4. Update absolute max-capacity as a float based on parent's maxResource
+    // and cluster resource.
+    childQueue.getQueueCapacities().setAbsoluteMaximumCapacity(label,
+        (float) childQueue.getQueueCapacities().getMaximumCapacity(label)
+            / getQueueCapacities().getAbsoluteMaximumCapacity(label));
+  }
+
   @Override
   public List<CSQueue> getChildQueues() {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.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/UsersManager.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/UsersManager.java
index efc20e9..7287c5b 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/UsersManager.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/UsersManager.java
@@ -686,7 +686,7 @@ public class UsersManager implements AbstractUsersManager {
      * * If we're running over capacity, then its (usedResources + required)
      * (which extra resources we are allocating)
      */
-    Resource queueCapacity = lQueue.getEffectiveCapacityUp(nodePartition);
+    Resource queueCapacity = lQueue.getEffectiveCapacity(nodePartition);
 
     /*
      * Assume we have required resource equals to minimumAllocation, this can

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/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 34594cf..a4e0096 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
@@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
@@ -909,6 +910,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       StringBuilder diagnosticMessage) {
     LeafQueue queue = getCSLeafQueue();
     QueueCapacities queueCapacities = queue.getQueueCapacities();
+    QueueResourceQuotas queueResourceQuotas = queue.getQueueResourceQuotas();
     diagnosticMessage.append(" Details : AM Partition = ");
     diagnosticMessage.append(appAMNodePartitionName.isEmpty()
         ? NodeLabel.DEFAULT_NODE_LABEL_PARTITION : appAMNodePartitionName);
@@ -930,6 +932,18 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
         queueCapacities.getAbsoluteMaximumCapacity(appAMNodePartitionName)
             * 100);
     diagnosticMessage.append(" % ; ");
+    diagnosticMessage.append("Queue's capacity (absolute resource) = ");
+    diagnosticMessage.append(
+        queueResourceQuotas.getEffectiveMinResource(appAMNodePartitionName));
+    diagnosticMessage.append(" ; ");
+    diagnosticMessage.append("Queue's used capacity (absolute resource) = ");
+    diagnosticMessage
+        .append(queue.getQueueResourceUsage().getUsed(appAMNodePartitionName));
+    diagnosticMessage.append(" ; ");
+    diagnosticMessage.append("Queue's max capacity (absolute resource) = ");
+    diagnosticMessage.append(
+        queueResourceQuotas.getEffectiveMaxResource(appAMNodePartitionName));
+    diagnosticMessage.append(" ; ");
   }
 
   /**
@@ -993,13 +1007,10 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       ResourceCalculator calc =
           rmContext.getScheduler().getResourceCalculator();
       if (!calc.isInvalidDivisor(totalPartitionRes)) {
-        float queueAbsMaxCapPerPartition =
-            ((AbstractCSQueue) getQueue()).getQueueCapacities()
-                .getAbsoluteCapacity(getAppAMNodePartitionName());
+        Resource effCap = ((AbstractCSQueue) getQueue())
+            .getEffectiveCapacity(getAppAMNodePartitionName());
         float queueUsagePerc = calc.divide(totalPartitionRes,
-            report.getUsedResources(),
-            Resources.multiply(totalPartitionRes, queueAbsMaxCapPerPartition))
-            * 100;
+            report.getUsedResources(), effCap) * 100;
         report.setQueueUsagePercentage(queueUsagePerc);
       }
       return report;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 74b4e79..3f68e07 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -155,21 +155,36 @@ class CapacitySchedulerPage extends RmView {
           ? new ResourceInfo(Resources.none())
           : resourceUsages.getAmUsed();
       ri.
-          __("Used Capacity:", percent(capacities.getUsedCapacity() / 100)).
-          __("Configured Capacity:", percent(capacities.getCapacity() / 100)).
-          __("Configured Max Capacity:", percent(capacities.getMaxCapacity() / 100)).
-          __("Absolute Used Capacity:", percent(capacities.getAbsoluteUsedCapacity() / 100)).
-          __("Absolute Configured Capacity:", percent(capacities.getAbsoluteCapacity() / 100)).
-          __("Absolute Configured Max Capacity:", percent(capacities.getAbsoluteMaxCapacity() / 100)).
-          __("Used Resources:", resourceUsages.getUsed().toString()).
-          __("Configured Max Application Master Limit:", StringUtils.format("%.1f",
-          capacities.getMaxAMLimitPercentage())).
-          __("Max Application Master Resources:",
-          resourceUsages.getAMLimit().toString()).
-          __("Used Application Master Resources:",
-          amUsed.toString()).
-          __("Max Application Master Resources Per User:",
-          userAMResourceLimit.toString());
+          __("Used Capacity:",
+              appendPercent(resourceUsages.getUsed().toString(),
+                  capacities.getUsedCapacity() / 100))
+          .__("Configured Capacity:",
+              capacities.getConfiguredMinResource().toString())
+          .__("Configured Max Capacity:",
+              capacities.getConfiguredMaxResource().getResource()
+                  .equals(Resources.none())
+                      ? "unlimited"
+                      : capacities.getConfiguredMaxResource().toString())
+          .__("Effective Capacity:",
+              appendPercent(capacities.getEffectiveMinResource().toString(),
+                  capacities.getCapacity() / 100))
+          .__("Effective Max Capacity:",
+              appendPercent(capacities.getEffectiveMaxResource().toString(),
+                  capacities.getMaxCapacity() / 100))
+          .__("Absolute Used Capacity:",
+              percent(capacities.getAbsoluteUsedCapacity() / 100))
+          .__("Absolute Configured Capacity:",
+              percent(capacities.getAbsoluteCapacity() / 100))
+          .__("Absolute Configured Max Capacity:",
+              percent(capacities.getAbsoluteMaxCapacity() / 100))
+          .__("Used Resources:", resourceUsages.getUsed().toString())
+          .__("Configured Max Application Master Limit:",
+              StringUtils.format("%.1f", capacities.getMaxAMLimitPercentage()))
+          .__("Max Application Master Resources:",
+              resourceUsages.getAMLimit().toString())
+          .__("Used Application Master Resources:", amUsed.toString())
+          .__("Max Application Master Resources Per User:",
+              userAMResourceLimit.toString());
     }
 
     private void renderCommonLeafQueueInfo(ResponseInfo ri) {
@@ -615,6 +630,10 @@ class CapacitySchedulerPage extends RmView {
     return QueuesBlock.class;
   }
 
+  static String appendPercent(String message, float f) {
+    return message + " (" + StringUtils.formatPercent(f, 1) + ")";
+  }
+
   static String percent(float f) {
     return StringUtils.formatPercent(f, 1);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
index 32e4ac5..ca42041 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
@@ -59,7 +59,8 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
       max = 1f;
     this.maxCapacity = max * 100;
 
-    capacities = new QueueCapacitiesInfo(parent.getQueueCapacities(), false);
+    capacities = new QueueCapacitiesInfo(parent.getQueueCapacities(),
+        parent.getQueueResourceQuotas(), false);
     queues = getQueues(parent);
     health = new CapacitySchedulerHealthInfo(cs);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
index 7dcdf58..343c78e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
@@ -25,6 +25,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
@@ -89,8 +90,9 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
   }
 
   @Override
-  protected void populateQueueCapacities(QueueCapacities qCapacities) {
-    capacities = new QueueCapacitiesInfo(qCapacities);
+  protected void populateQueueCapacities(QueueCapacities qCapacities,
+      QueueResourceQuotas qResQuotas) {
+    capacities = new QueueCapacitiesInfo(qCapacities, qResQuotas);
   }
 
   public int getNumActiveApplications() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
index 86b2fea..d4de9ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
@@ -28,6 +28,7 @@ import javax.xml.bind.annotation.XmlSeeAlso;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
@@ -103,7 +104,8 @@ public class CapacitySchedulerQueueInfo {
       Collections.sort(nodeLabels);
     }
     QueueCapacities qCapacities = q.getQueueCapacities();
-    populateQueueCapacities(qCapacities);
+    QueueResourceQuotas qResQuotas = q.getQueueResourceQuotas();
+    populateQueueCapacities(qCapacities, qResQuotas);
 
     ResourceUsage queueResourceUsage = q.getQueueResourceUsage();
     populateQueueResourceUsage(queueResourceUsage);
@@ -118,8 +120,10 @@ public class CapacitySchedulerQueueInfo {
     resources = new ResourcesInfo(queueResourceUsage, false);
   }
 
-  protected void populateQueueCapacities(QueueCapacities qCapacities) {
-    capacities = new QueueCapacitiesInfo(qCapacities, false);
+  protected void populateQueueCapacities(QueueCapacities qCapacities,
+      QueueResourceQuotas qResQuotas) {
+    capacities = new QueueCapacitiesInfo(qCapacities, qResQuotas,
+        false);
   }
 
   public float getCapacity() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/PartitionQueueCapacitiesInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/PartitionQueueCapacitiesInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/PartitionQueueCapacitiesInfo.java
index 5e298f9..2a15502 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/PartitionQueueCapacitiesInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/PartitionQueueCapacitiesInfo.java
@@ -21,6 +21,9 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
 /**
  * This class represents queue capacities for a given partition
  */
@@ -36,13 +39,19 @@ public class PartitionQueueCapacitiesInfo {
   private float absoluteUsedCapacity;
   private float absoluteMaxCapacity  = 100;
   private float maxAMLimitPercentage;
+  private ResourceInfo configuredMinResource;
+  private ResourceInfo configuredMaxResource;
+  private ResourceInfo effectiveMinResource;
+  private ResourceInfo effectiveMaxResource;
 
   public PartitionQueueCapacitiesInfo() {
   }
 
   public PartitionQueueCapacitiesInfo(String partitionName, float capacity,
       float usedCapacity, float maxCapacity, float absCapacity,
-      float absUsedCapacity, float absMaxCapacity, float maxAMLimitPercentage) {
+      float absUsedCapacity, float absMaxCapacity, float maxAMLimitPercentage,
+      Resource confMinRes, Resource confMaxRes, Resource effMinRes,
+      Resource effMaxRes) {
     super();
     this.partitionName = partitionName;
     this.capacity = capacity;
@@ -52,6 +61,10 @@ public class PartitionQueueCapacitiesInfo {
     this.absoluteUsedCapacity = absUsedCapacity;
     this.absoluteMaxCapacity = absMaxCapacity;
     this.maxAMLimitPercentage = maxAMLimitPercentage;
+    this.configuredMinResource = new ResourceInfo(confMinRes);
+    this.configuredMaxResource = new ResourceInfo(confMaxRes);
+    this.effectiveMinResource = new ResourceInfo(effMinRes);
+    this.effectiveMaxResource = new ResourceInfo(effMaxRes);
   }
 
   public float getCapacity() {
@@ -117,4 +130,23 @@ public class PartitionQueueCapacitiesInfo {
   public void setMaxAMLimitPercentage(float maxAMLimitPercentage) {
     this.maxAMLimitPercentage = maxAMLimitPercentage;
   }
+
+  public ResourceInfo getConfiguredMinResource() {
+    return configuredMinResource;
+  }
+
+  public ResourceInfo getConfiguredMaxResource() {
+    if (configuredMaxResource.getResource().equals(Resources.none())) {
+      return null;
+    }
+    return configuredMaxResource;
+  }
+
+  public ResourceInfo getEffectiveMinResource() {
+    return effectiveMinResource;
+  }
+
+  public ResourceInfo getEffectiveMaxResource() {
+    return effectiveMaxResource;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacitiesInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacitiesInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacitiesInfo.java
index 9a3e439..35c80d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacitiesInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacitiesInfo.java
@@ -24,6 +24,7 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 
 /**
@@ -39,6 +40,7 @@ public class QueueCapacitiesInfo {
   }
 
   public QueueCapacitiesInfo(QueueCapacities capacities,
+      QueueResourceQuotas resourceQuotas,
       boolean considerAMUsage) {
     if (capacities == null) {
       return;
@@ -68,12 +70,17 @@ public class QueueCapacitiesInfo {
       queueCapacitiesByPartition.add(new PartitionQueueCapacitiesInfo(
           partitionName, capacity, usedCapacity, maxCapacity, absCapacity,
           absUsedCapacity, absMaxCapacity,
-          considerAMUsage ? maxAMLimitPercentage : 0f));
+          considerAMUsage ? maxAMLimitPercentage : 0f,
+          resourceQuotas.getConfiguredMinResource(partitionName),
+          resourceQuotas.getConfiguredMaxResource(partitionName),
+          resourceQuotas.getEffectiveMinResource(partitionName),
+          resourceQuotas.getEffectiveMaxResource(partitionName)));
     }
   }
 
-  public QueueCapacitiesInfo(QueueCapacities capacities) {
-    this(capacities, true);
+  public QueueCapacitiesInfo(QueueCapacities capacities,
+      QueueResourceQuotas resourceQuotas) {
+    this(capacities, resourceQuotas, true);
   }
 
   public void add(PartitionQueueCapacitiesInfo partitionQueueCapacitiesInfo) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerWithMultiResourceTypes.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/TestCapacitySchedulerWithMultiResourceTypes.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/TestCapacitySchedulerWithMultiResourceTypes.java
index b386c18..38768e5 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/TestCapacitySchedulerWithMultiResourceTypes.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/TestCapacitySchedulerWithMultiResourceTypes.java
@@ -61,11 +61,20 @@ public class TestCapacitySchedulerWithMultiResourceTypes {
     Map<String, ResourceInformation> riMap = new HashMap<>();
 
     // Initialize mandatory resources
-    riMap.put(ResourceInformation.MEMORY_URI, ResourceInformation.MEMORY_MB);
-    riMap.put(ResourceInformation.VCORES_URI, ResourceInformation.VCORES);
-    riMap.put(RESOURCE_1, ResourceInformation
-        .newInstance(RESOURCE_1, "", 0, ResourceTypes.COUNTABLE, 0,
-            Integer.MAX_VALUE));
+    ResourceInformation memory = ResourceInformation.newInstance(
+        ResourceInformation.MEMORY_MB.getName(),
+        ResourceInformation.MEMORY_MB.getUnits(),
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
+    ResourceInformation vcores = ResourceInformation.newInstance(
+        ResourceInformation.VCORES.getName(),
+        ResourceInformation.VCORES.getUnits(),
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+    riMap.put(ResourceInformation.MEMORY_URI, memory);
+    riMap.put(ResourceInformation.VCORES_URI, vcores);
+    riMap.put(RESOURCE_1, ResourceInformation.newInstance(RESOURCE_1, "", 0,
+        ResourceTypes.COUNTABLE, 0, Integer.MAX_VALUE));
 
     ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
 
@@ -107,7 +116,7 @@ public class TestCapacitySchedulerWithMultiResourceTypes {
     RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
 
-    Assert.assertEquals(Resource.newInstance(1 * GB, 0),
+    Assert.assertEquals(Resource.newInstance(1 * GB, 1),
         leafQueue.getUsedResources());
 
     RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
@@ -123,9 +132,9 @@ public class TestCapacitySchedulerWithMultiResourceTypes {
     // Do node heartbeats 1 time and check container allocated.
     cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
 
-    // Now used resource = <mem=1GB, vcore=0> + <mem=2GB,vcore=2,res_1=2>
+    // Now used resource = <mem=1GB, vcore=1> + <mem=2GB,vcore=2,res_1=2>
     Assert.assertEquals(
-        TestUtils.createResource(3 * GB, 2, ImmutableMap.of(RESOURCE_1, 2)),
+        TestUtils.createResource(3 * GB, 3, ImmutableMap.of(RESOURCE_1, 2)),
         leafQueue.getUsedResources());
 
     // Acquire container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52627a7/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 96f8970..30bff78 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
@@ -74,6 +74,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueStateManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
@@ -1355,7 +1356,7 @@ public class TestLeafQueue {
     // TODO, fix headroom in the future patch
     assertEquals(1*GB, app_0.getHeadroom().getMemorySize());
       // User limit = 2G, 2 in use
-    assertEquals(0*GB, app_1.getHeadroom().getMemorySize());
+    assertEquals(1*GB, app_1.getHeadroom().getMemorySize());
       // the application is not yet active
 
     // Again one to user_0 since he hasn't exceeded user limit yet
@@ -1366,8 +1367,8 @@ public class TestLeafQueue {
     assertEquals(3*GB, a.getUsedResources().getMemorySize());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
-    assertEquals(1*GB, app_0.getHeadroom().getMemorySize()); // 4G - 3G
-    assertEquals(1*GB, app_1.getHeadroom().getMemorySize()); // 4G - 3G
+    assertEquals(0*GB, app_0.getHeadroom().getMemorySize());
+    assertEquals(0*GB, app_1.getHeadroom().getMemorySize()); // 4G - 3G
     
     // Submit requests for app_1 and set max-cap
     a.setMaxCapacity(.1f);
@@ -3874,7 +3875,7 @@ public class TestLeafQueue {
 
     // Queue "test" consumes 100% of the cluster, so its capacity and absolute
     // capacity are both 1.0f.
-    Queue queue = createQueue("test", null, 1.0f, 1.0f);
+    Queue queue = createQueue("test", null, 1.0f, 1.0f, res);
     final String user = "user1";
     FiCaSchedulerApp app =
         new FiCaSchedulerApp(appAttId, user, queue,
@@ -3891,7 +3892,8 @@ public class TestLeafQueue {
 
     // Queue "test2" is a child of root and its capacity is 50% of root. As a
     // child of root, its absolute capaicty is also 50%.
-    queue = createQueue("test2", null, 0.5f, 0.5f);
+    queue = createQueue("test2", null, 0.5f, 0.5f,
+        Resources.divideAndCeil(dominantResourceCalculator, res, 2));
     app = new FiCaSchedulerApp(appAttId, user, queue,
         queue.getAbstractUsersManager(), rmContext);
     app.getAppAttemptResourceUsage().incUsed(requestedResource);
@@ -3903,7 +3905,8 @@ public class TestLeafQueue {
 
     // Queue "test2.1" is 50% of queue "test2", which is 50% of the cluster.
     // Therefore, "test2.1" capacity is 50% and absolute capacity is 25%.
-    AbstractCSQueue qChild = createQueue("test2.1", queue, 0.5f, 0.25f);
+    AbstractCSQueue qChild = createQueue("test2.1", queue, 0.5f, 0.25f,
+        Resources.divideAndCeil(dominantResourceCalculator, res, 4));
     app = new FiCaSchedulerApp(appAttId, user, qChild,
         qChild.getAbstractUsersManager(), rmContext);
     app.getAppAttemptResourceUsage().incUsed(requestedResource);
@@ -3922,7 +3925,7 @@ public class TestLeafQueue {
   }
 
   private AbstractCSQueue createQueue(String name, Queue parent, float capacity,
-      float absCap) {
+      float absCap, Resource res) {
     CSQueueMetrics metrics = CSQueueMetrics.forQueue(name, parent, false, cs.getConf());
     QueueInfo queueInfo = QueueInfo.newInstance(name, capacity, 1.0f, 0, null,
         null, QueueState.RUNNING, null, "", null, false);
@@ -3934,6 +3937,13 @@ public class TestLeafQueue {
     QueueCapacities qCaps = mock(QueueCapacities.class);
     when(qCaps.getAbsoluteCapacity(any())).thenReturn(absCap);
     when(queue.getQueueCapacities()).thenReturn(qCaps);
+    QueueResourceQuotas qQuotas = mock(QueueResourceQuotas.class);
+    when(qQuotas.getConfiguredMinResource(any())).thenReturn(res);
+    when(qQuotas.getConfiguredMaxResource(any())).thenReturn(res);
+    when(qQuotas.getEffectiveMinResource(any())).thenReturn(res);
+    when(qQuotas.getEffectiveMaxResource(any())).thenReturn(res);
+    when(queue.getQueueResourceQuotas()).thenReturn(qQuotas);
+    when(queue.getEffectiveCapacity(any())).thenReturn(res);
     return queue;
   }
 


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


[41/50] [abbrv] hadoop git commit: YARN-7420. YARN UI changes to depict auto created queues. (Suma Shivaprasad via wangda)

Posted by wa...@apache.org.
YARN-7420. YARN UI changes to depict auto created queues. (Suma Shivaprasad via wangda)

Change-Id: I8039d3772a191ddede132cd1f8b08a8ca2e275b7


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

Branch: refs/heads/YARN-6592
Commit: f548bfffbdcd426811352d6920ee5fe50cd0182c
Parents: b38643c
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Dec 8 15:10:47 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Dec 8 15:10:47 2017 -0800

----------------------------------------------------------------------
 .../resourcemanager/webapp/CapacitySchedulerPage.java | 14 ++++++++++++--
 .../webapp/dao/CapacitySchedulerLeafQueueInfo.java    | 11 +++++++++++
 .../webapp/dao/CapacitySchedulerQueueInfo.java        |  4 ++++
 3 files changed, 27 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f548bfff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 4933d34..de85590 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -66,6 +66,7 @@ class CapacitySchedulerPage extends RmView {
   static final String Q_END = "left:101%";
   static final String Q_GIVEN =
       "left:0%;background:none;border:1px dashed #BFBFBF";
+  static final String Q_AUTO_CREATED = "background:#F4F0CB";
   static final String Q_OVER = "background:#FFA333";
   static final String Q_UNDER = "background:#5BD75B";
   static final String ACTIVE_USER = "background:#FFFF00"; // Yellow highlight
@@ -299,9 +300,16 @@ class CapacitySchedulerPage extends RmView {
         absMaxCap = partitionQueueCapsInfo.getAbsoluteMaxCapacity() / 100;
         absUsedCap = partitionQueueCapsInfo.getAbsoluteUsedCapacity() / 100;
 
+        boolean isAutoCreatedLeafQueue = info.isLeafQueue() ?
+            ((CapacitySchedulerLeafQueueInfo) info).isAutoCreatedLeafQueue()
+            : false;
+
+        String Q_WIDTH = width(absMaxCap * Q_MAX_WIDTH);
         LI<UL<Hamlet>> li = ul.
           li().
-            a(_Q).$style(width(absMaxCap * Q_MAX_WIDTH)).
+            a(_Q).$style(isAutoCreatedLeafQueue? join( Q_AUTO_CREATED, ";",
+            Q_WIDTH)
+            :  Q_WIDTH).
               $title(join("Absolute Capacity:", percent(absCap))).
               span().$style(join(Q_GIVEN, ";font-size:1px;", width(absCap/absMaxCap))).
             __('.').__().
@@ -313,7 +321,7 @@ class CapacitySchedulerPage extends RmView {
             __(join(percent(used), " used")).__();
 
         csqinfo.qinfo = info;
-        if (info.getQueues() == null) {
+        if (info.isLeafQueue()) {
           li.ul("#lq").li().__(LeafQueueInfoBlock.class).__().__();
           li.ul("#lq").li().__(QueueUsersInfoBlock.class).__().__();
         } else {
@@ -422,6 +430,8 @@ class CapacitySchedulerPage extends RmView {
               __("Max Capacity").__().
             span().$class("qlegend ui-corner-all").$style(ACTIVE_USER).
             __("Users Requesting Resources").__().
+            span().$class("qlegend ui-corner-all").$style(Q_AUTO_CREATED).
+            __("Auto Created Queues").__().
           __();
 
         float used = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f548bfff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
index 343c78e..b5f4e79 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
@@ -27,6 +27,8 @@ import javax.xml.bind.annotation.XmlTransient;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .AutoCreatedLeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UserInfo;
@@ -49,6 +51,7 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
   protected boolean preemptionDisabled;
   protected String defaultNodeLabelExpression;
   protected int defaultPriority;
+  protected boolean isAutoCreatedLeafQueue;
 
   @XmlTransient
   protected String orderingPolicyInfo;
@@ -82,6 +85,10 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
           .getPartitionResourceUsageInfo(RMNodeLabelsManager.NO_LABEL)
           .getAMLimit();
     }
+
+    if ( q instanceof AutoCreatedLeafQueue) {
+      isAutoCreatedLeafQueue = true;
+    }
   }
 
   @Override
@@ -155,4 +162,8 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
   public int getDefaultApplicationPriority() {
     return defaultPriority;
   }
+
+  public boolean isAutoCreatedLeafQueue() {
+    return isAutoCreatedLeafQueue;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f548bfff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
index d4de9ae..fd256b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
@@ -219,4 +219,8 @@ public class CapacitySchedulerQueueInfo {
   public ResourceInfo getMaxEffectiveCapacity(){
     return maxEffectiveCapacity;
   }
+
+  public boolean isLeafQueue() {
+    return getQueues() == null;
+  }
 }


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


[25/50] [abbrv] hadoop git commit: YARN-7332. Compute effectiveCapacity per each resource vector. (Sunil G via wangda)

Posted by wa...@apache.org.
YARN-7332. Compute effectiveCapacity per each resource vector. (Sunil G via wangda)


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

Branch: refs/heads/YARN-6592
Commit: aa3f62740f71e6e5b2a424a9f4654c1a4ba1dbe6
Parents: d52627a
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Oct 27 10:16:33 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../scheduler/capacity/ParentQueue.java         | 66 ++++++++++++--
 .../scheduler/capacity/TestParentQueue.java     | 94 ++++++++++++++++++++
 2 files changed, 153 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa3f6274/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 5ab1494..940637e 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
@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AccessType;
@@ -68,7 +69,9 @@ 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.server.resourcemanager.scheduler.placement.CandidateNodeSet;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.CandidateNodeSetUtils;
+import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 @Private
@@ -928,24 +931,25 @@ public class ParentQueue extends AbstractCSQueue {
     // Factor to scale down effective resource: When cluster has sufficient
     // resources, effective_min_resources will be same as configured
     // min_resources.
-    float effectiveMinRatio = 1;
+    Resource numeratorForMinRatio = null;
     ResourceCalculator rc = this.csContext.getResourceCalculator();
     if (getQueueName().equals("root")) {
       if (!resourceByLabel.equals(Resources.none()) && Resources.lessThan(rc,
           clusterResource, resourceByLabel, configuredMinResources)) {
-        effectiveMinRatio = Resources.divide(rc, clusterResource,
-            resourceByLabel, configuredMinResources);
+        numeratorForMinRatio = resourceByLabel;
       }
     } else {
       if (Resources.lessThan(rc, clusterResource,
           queueResourceQuotas.getEffectiveMinResource(label),
           configuredMinResources)) {
-        effectiveMinRatio = Resources.divide(rc, clusterResource,
-            queueResourceQuotas.getEffectiveMinResource(label),
-            configuredMinResources);
+        numeratorForMinRatio = queueResourceQuotas
+            .getEffectiveMinResource(label);
       }
     }
 
+    Map<String, Float> effectiveMinRatioPerResource = getEffectiveMinRatioPerResource(
+        configuredMinResources, numeratorForMinRatio);
+
     // loop and do this for all child queues
     for (CSQueue childQueue : getChildQueues()) {
       Resource minResource = childQueue.getQueueResourceQuotas()
@@ -955,7 +959,8 @@ public class ParentQueue extends AbstractCSQueue {
       if (childQueue.getCapacityConfigType()
           .equals(CapacityConfigType.ABSOLUTE_RESOURCE)) {
         childQueue.getQueueResourceQuotas().setEffectiveMinResource(label,
-            Resources.multiply(minResource, effectiveMinRatio));
+            getMinResourceNormalized(childQueue.getQueueName(), effectiveMinRatioPerResource,
+                minResource));
 
         // Max resource of a queue should be a minimum of {configuredMaxRes,
         // parentMaxRes}. parentMaxRes could be configured value. But if not
@@ -1003,6 +1008,53 @@ public class ParentQueue extends AbstractCSQueue {
     }
   }
 
+  private Resource getMinResourceNormalized(String name, Map<String, Float> effectiveMinRatio,
+      Resource minResource) {
+    Resource ret = Resource.newInstance(minResource);
+    int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
+    for (int i = 0; i < maxLength; i++) {
+      ResourceInformation nResourceInformation = minResource
+          .getResourceInformation(i);
+
+      Float ratio = effectiveMinRatio.get(nResourceInformation.getName());
+      if (ratio != null) {
+        ret.setResourceValue(i,
+            (long) (nResourceInformation.getValue() * ratio.floatValue()));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Updating min resource for Queue: " + name + " as "
+              + ret.getResourceInformation(i) + ", Actual resource: "
+              + nResourceInformation.getValue() + ", ratio: "
+              + ratio.floatValue());
+        }
+      }
+    }
+    return ret;
+  }
+
+  private Map<String, Float> getEffectiveMinRatioPerResource(
+      Resource configuredMinResources, Resource numeratorForMinRatio) {
+    Map<String, Float> effectiveMinRatioPerResource = new HashMap<>();
+    if (numeratorForMinRatio != null) {
+      int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
+      for (int i = 0; i < maxLength; i++) {
+        ResourceInformation nResourceInformation = numeratorForMinRatio
+            .getResourceInformation(i);
+        ResourceInformation dResourceInformation = configuredMinResources
+            .getResourceInformation(i);
+
+        long nValue = nResourceInformation.getValue();
+        long dValue = UnitsConversionUtil.convert(
+            dResourceInformation.getUnits(), nResourceInformation.getUnits(),
+            dResourceInformation.getValue());
+        if (dValue != 0) {
+          effectiveMinRatioPerResource.put(nResourceInformation.getName(),
+              (float) nValue / dValue);
+        }
+      }
+    }
+    return effectiveMinRatioPerResource;
+  }
+
   private void deriveCapacityFromAbsoluteConfigurations(String label,
       Resource clusterResource, ResourceCalculator rc, CSQueue childQueue) {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa3f6274/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 25a9774..fe66aba 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
@@ -68,6 +68,11 @@ import org.mockito.stubbing.Answer;
 
 public class TestParentQueue {
 
+  private static final Resource QUEUE_B_RESOURCE = Resource
+      .newInstance(14 * 1024, 22);
+  private static final Resource QUEUE_A_RESOURCE = Resource
+      .newInstance(6 * 1024, 10);
+
   private static final Log LOG = LogFactory.getLog(TestParentQueue.class);
   
   RMContext rmContext;
@@ -118,6 +123,23 @@ public class TestParentQueue {
     LOG.info("Setup top-level queues a and b");
   }
 
+  private void setupSingleLevelQueuesWithAbsoluteResource(
+      CapacitySchedulerConfiguration conf) {
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{A, B});
+
+    final String Q_A = CapacitySchedulerConfiguration.ROOT + "." + A;
+    conf.setMinimumResourceRequirement("", Q_A,
+        QUEUE_A_RESOURCE);
+
+    final String Q_B = CapacitySchedulerConfiguration.ROOT + "." + B;
+    conf.setMinimumResourceRequirement("", Q_B,
+        QUEUE_B_RESOURCE);
+
+    LOG.info("Setup top-level queues a and b with absolute resource");
+  }
+
   private FiCaSchedulerApp getMockApplication(int appId, String user) {
     FiCaSchedulerApp application = mock(FiCaSchedulerApp.class);
     doReturn(user).when(application).getUser();
@@ -931,6 +953,78 @@ public class TestParentQueue {
     reset(c);
   }
 
+  @Test
+  public void testAbsoluteResourceWithChangeInClusterResource()
+      throws Exception {
+    // Setup queue configs
+    setupSingleLevelQueuesWithAbsoluteResource(csConf);
+
+    Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
+    CSQueue root = CapacitySchedulerQueueManager.parseQueue(csContext, csConf,
+        null, CapacitySchedulerConfiguration.ROOT, queues, queues,
+        TestUtils.spyHook);
+
+    // Setup some nodes
+    final int memoryPerNode = 10;
+    int coresPerNode = 16;
+    int numNodes = 2;
+
+    Resource clusterResource = Resources.createResource(
+        numNodes * (memoryPerNode * GB), numNodes * coresPerNode);
+    when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
+
+    // Start testing
+    LeafQueue a = (LeafQueue) queues.get(A);
+    LeafQueue b = (LeafQueue) queues.get(B);
+
+    assertEquals(a.getQueueResourceQuotas().getConfiguredMinResource(),
+        QUEUE_A_RESOURCE);
+    assertEquals(b.getQueueResourceQuotas().getConfiguredMinResource(),
+        QUEUE_B_RESOURCE);
+    assertEquals(a.getQueueResourceQuotas().getEffectiveMinResource(),
+        QUEUE_A_RESOURCE);
+    assertEquals(b.getQueueResourceQuotas().getEffectiveMinResource(),
+        QUEUE_B_RESOURCE);
+
+    numNodes = 1;
+    clusterResource = Resources.createResource(numNodes * (memoryPerNode * GB),
+        numNodes * coresPerNode);
+    when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
+
+    Resource QUEUE_B_RESOURCE_HALF = Resource.newInstance(7 * 1024, 11);
+    Resource QUEUE_A_RESOURCE_HALF = Resource.newInstance(3 * 1024, 5);
+    assertEquals(a.getQueueResourceQuotas().getConfiguredMinResource(),
+        QUEUE_A_RESOURCE);
+    assertEquals(b.getQueueResourceQuotas().getConfiguredMinResource(),
+        QUEUE_B_RESOURCE);
+    assertEquals(a.getQueueResourceQuotas().getEffectiveMinResource(),
+        QUEUE_A_RESOURCE_HALF);
+    assertEquals(b.getQueueResourceQuotas().getEffectiveMinResource(),
+        QUEUE_B_RESOURCE_HALF);
+
+    coresPerNode = 40;
+    clusterResource = Resources.createResource(numNodes * (memoryPerNode * GB),
+        numNodes * coresPerNode);
+    when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
+
+    Resource QUEUE_B_RESOURCE_70PERC = Resource.newInstance(7 * 1024, 27);
+    Resource QUEUE_A_RESOURCE_30PERC = Resource.newInstance(3 * 1024, 12);
+    assertEquals(a.getQueueResourceQuotas().getConfiguredMinResource(),
+        QUEUE_A_RESOURCE);
+    assertEquals(b.getQueueResourceQuotas().getConfiguredMinResource(),
+        QUEUE_B_RESOURCE);
+    assertEquals(a.getQueueResourceQuotas().getEffectiveMinResource(),
+        QUEUE_A_RESOURCE_30PERC);
+    assertEquals(b.getQueueResourceQuotas().getEffectiveMinResource(),
+        QUEUE_B_RESOURCE_70PERC);
+  }
+
   @After
   public void tearDown() throws Exception {
   }


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


[38/50] [abbrv] hadoop git commit: YARN-7473. Implement Framework and policy for capacity management of auto created queues. (Suma Shivaprasad via wangda)

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueueBase.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/TestCapacitySchedulerAutoCreatedQueueBase.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/TestCapacitySchedulerAutoCreatedQueueBase.java
new file mode 100644
index 0000000..4e77339
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueueBase.java
@@ -0,0 +1,579 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scheduler.capacity;
+
+import org.apache.commons.lang.math.RandomUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .ApplicationPlacementContext;
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .UserGroupMappingPlacementRule;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .ResourceScheduler;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common
+    .QueueEntitlement;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
+    .AppAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
+    .AppAttemptAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
+    .SchedulerEvent;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
+    .NO_LABEL;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.CSQueueUtils.EPSILON;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.CapacitySchedulerConfiguration.DOT;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.CapacitySchedulerConfiguration.FAIR_APP_ORDERING_POLICY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+public class TestCapacitySchedulerAutoCreatedQueueBase {
+
+  private static final Log LOG = LogFactory.getLog(
+      TestCapacitySchedulerAutoCreatedQueueBase.class);
+  public final int GB = 1024;
+  public final static ContainerUpdates NULL_UPDATE_REQUESTS =
+      new ContainerUpdates();
+
+  public static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+  public static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+  public static final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+  public static final String D = CapacitySchedulerConfiguration.ROOT + ".d";
+  public static final String A1 = A + ".a1";
+  public static final String A2 = A + ".a2";
+  public static final String B1 = B + ".b1";
+  public static final String B2 = B + ".b2";
+  public static final String B3 = B + ".b3";
+  public static final String C1 = C + ".c1";
+  public static final String C2 = C + ".c2";
+  public static final String C3 = C + ".c3";
+  public static final float A_CAPACITY = 20f;
+  public static final float B_CAPACITY = 40f;
+  public static final float C_CAPACITY = 20f;
+  public static final float D_CAPACITY = 20f;
+  public static final float A1_CAPACITY = 30;
+  public static final float A2_CAPACITY = 70;
+  public static final float B1_CAPACITY = 60f;
+  public static final float B2_CAPACITY = 20f;
+  public static final float B3_CAPACITY = 20f;
+  public static final float C1_CAPACITY = 20f;
+  public static final float C2_CAPACITY = 20f;
+
+  public static final String USER = "user_";
+  public static final String USER0 = USER + 0;
+  public static final String USER1 = USER + 1;
+  public static final String USER3 = USER + 3;
+  public static final String USER2 = USER + 2;
+  public static final String PARENT_QUEUE = "c";
+
+  public static final Set<String> accessibleNodeLabelsOnC = new HashSet<>();
+
+  public static final String NODEL_LABEL_GPU = "GPU";
+  public static final String NODEL_LABEL_SSD = "SSD";
+
+  protected MockRM mockRM = null;
+  protected CapacityScheduler cs;
+  private final TestCapacityScheduler tcs = new TestCapacityScheduler();
+  protected SpyDispatcher dispatcher;
+  private static EventHandler<Event> rmAppEventEventHandler;
+
+  public static class SpyDispatcher extends AsyncDispatcher {
+
+    public static BlockingQueue<Event> eventQueue = new LinkedBlockingQueue<>();
+
+    public static class SpyRMAppEventHandler implements EventHandler<Event> {
+      public void handle(Event event) {
+        eventQueue.add(event);
+      }
+    }
+
+    @Override
+    protected void dispatch(Event event) {
+      eventQueue.add(event);
+    }
+
+    @Override
+    public EventHandler<Event> getEventHandler() {
+      return rmAppEventEventHandler;
+    }
+
+    void spyOnNextEvent(Event expectedEvent, long timeout)
+        throws InterruptedException {
+
+      Event event = eventQueue.poll(timeout, TimeUnit.MILLISECONDS);
+      assertEquals(expectedEvent.getType(), event.getType());
+      assertEquals(expectedEvent.getClass(), event.getClass());
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    setupQueueConfiguration(conf);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    setupQueueMappings(conf);
+
+    mockRM = new MockRM(conf);
+    cs = (CapacityScheduler) mockRM.getResourceScheduler();
+
+    dispatcher = new SpyDispatcher();
+    rmAppEventEventHandler = new SpyDispatcher.SpyRMAppEventHandler();
+    dispatcher.register(RMAppEventType.class, rmAppEventEventHandler);
+    cs.updatePlacementRules();
+    mockRM.start();
+
+    cs.start();
+  }
+
+  public static CapacitySchedulerConfiguration setupQueueMappings(
+      CapacitySchedulerConfiguration conf) {
+
+    List<String> queuePlacementRules = new ArrayList<>();
+    queuePlacementRules.add(YarnConfiguration.USER_GROUP_PLACEMENT_RULE);
+    conf.setQueuePlacementRules(queuePlacementRules);
+
+    //set queue mapping
+    List<UserGroupMappingPlacementRule.QueueMapping> queueMappings =
+        new ArrayList<>();
+    for (int i = 0; i <= 3; i++) {
+      //Set C as parent queue name for auto queue creation
+      UserGroupMappingPlacementRule.QueueMapping userQueueMapping =
+          new UserGroupMappingPlacementRule.QueueMapping(
+              UserGroupMappingPlacementRule.QueueMapping.MappingType.USER,
+              USER + i, getQueueMapping(PARENT_QUEUE, USER + i));
+      queueMappings.add(userQueueMapping);
+    }
+
+    conf.setQueueMappings(queueMappings);
+    //override with queue mappings
+    conf.setOverrideWithQueueMappings(true);
+    return conf;
+  }
+
+  /**
+   * @param conf, to be modified
+   * @return, CS configuration which has C as an auto creation enabled parent
+   * queue
+   * <p>
+   * root /     \      \       \ a        b      c    d / \    /  |  \ a1  a2 b1
+   * b2  b3
+   */
+  public static CapacitySchedulerConfiguration setupQueueConfiguration(
+      CapacitySchedulerConfiguration conf) {
+
+    //setup new queues with one of them auto enabled
+    // Define top-level queues
+    // Set childQueue for root
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] { "a", "b", "c", "d" });
+
+    conf.setCapacity(A, A_CAPACITY);
+    conf.setCapacity(B, B_CAPACITY);
+    conf.setCapacity(C, C_CAPACITY);
+    conf.setCapacity(D, D_CAPACITY);
+
+    // Define 2nd-level queues
+    conf.setQueues(A, new String[] { "a1", "a2" });
+    conf.setCapacity(A1, A1_CAPACITY);
+    conf.setUserLimitFactor(A1, 100.0f);
+    conf.setCapacity(A2, A2_CAPACITY);
+    conf.setUserLimitFactor(A2, 100.0f);
+
+    conf.setQueues(B, new String[] { "b1", "b2", "b3" });
+    conf.setCapacity(B1, B1_CAPACITY);
+    conf.setUserLimitFactor(B1, 100.0f);
+    conf.setCapacity(B2, B2_CAPACITY);
+    conf.setUserLimitFactor(B2, 100.0f);
+    conf.setCapacity(B3, B3_CAPACITY);
+    conf.setUserLimitFactor(B3, 100.0f);
+
+    conf.setUserLimitFactor(C, 1.0f);
+    conf.setAutoCreateChildQueueEnabled(C, true);
+
+    //Setup leaf queue template configs
+    conf.setAutoCreatedLeafQueueConfigCapacity(C, 50.0f);
+    conf.setAutoCreatedLeafQueueConfigMaxCapacity(C, 100.0f);
+    conf.setAutoCreatedLeafQueueConfigUserLimit(C, 100);
+    conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f);
+
+    LOG.info("Setup " + C + " as an auto leaf creation enabled parent queue");
+
+    conf.setUserLimitFactor(D, 1.0f);
+    conf.setAutoCreateChildQueueEnabled(D, true);
+    conf.setUserLimit(D, 100);
+    conf.setUserLimitFactor(D, 3.0f);
+
+    //Setup leaf queue template configs
+    conf.setAutoCreatedLeafQueueConfigCapacity(D, 10.0f);
+    conf.setAutoCreatedLeafQueueConfigMaxCapacity(D, 100.0f);
+    conf.setAutoCreatedLeafQueueConfigUserLimit(D, 3);
+    conf.setAutoCreatedLeafQueueConfigUserLimitFactor(D, 100);
+
+    conf.set(CapacitySchedulerConfiguration.PREFIX + C + DOT
+            + CapacitySchedulerConfiguration
+            .AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX
+            + DOT + CapacitySchedulerConfiguration.ORDERING_POLICY,
+        FAIR_APP_ORDERING_POLICY);
+
+    accessibleNodeLabelsOnC.add(NODEL_LABEL_GPU);
+    accessibleNodeLabelsOnC.add(NODEL_LABEL_SSD);
+    accessibleNodeLabelsOnC.add(NO_LABEL);
+
+    conf.setAccessibleNodeLabels(C, accessibleNodeLabelsOnC);
+    conf.setCapacityByLabel(C, NODEL_LABEL_GPU, 50);
+    conf.setCapacityByLabel(C, NODEL_LABEL_SSD, 50);
+
+    LOG.info("Setup " + D + " as an auto leaf creation enabled parent queue");
+
+    return conf;
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (mockRM != null) {
+      mockRM.stop();
+    }
+  }
+
+  protected void validateCapacities(AutoCreatedLeafQueue autoCreatedLeafQueue,
+      float capacity, float absCapacity, float maxCapacity,
+      float absMaxCapacity) {
+    assertEquals(capacity, autoCreatedLeafQueue.getCapacity(), EPSILON);
+    assertEquals(absCapacity, autoCreatedLeafQueue.getAbsoluteCapacity(),
+        EPSILON);
+    assertEquals(maxCapacity, autoCreatedLeafQueue.getMaximumCapacity(),
+        EPSILON);
+    assertEquals(absMaxCapacity,
+        autoCreatedLeafQueue.getAbsoluteMaximumCapacity(), EPSILON);
+  }
+
+  protected void cleanupQueue(String queueName) throws YarnException {
+    AutoCreatedLeafQueue queue = (AutoCreatedLeafQueue) cs.getQueue(queueName);
+    if (queue != null) {
+      setEntitlement(queue, new QueueEntitlement(0.0f, 0.0f));
+      ((ManagedParentQueue) queue.getParent()).removeChildQueue(
+          queue.getQueueName());
+      cs.getCapacitySchedulerQueueManager().removeQueue(queue.getQueueName());
+    }
+  }
+
+  protected ApplicationId submitApp(MockRM rm, CSQueue parentQueue,
+      String leafQueueName, String user, int expectedNumAppsInParentQueue,
+      int expectedNumAppsInLeafQueue) throws Exception {
+
+    CapacityScheduler capacityScheduler =
+        (CapacityScheduler) rm.getResourceScheduler();
+    // submit an app
+    RMApp rmApp = rm.submitApp(GB, "test-auto-queue-activation", user, null,
+        leafQueueName);
+
+    // check preconditions
+    List<ApplicationAttemptId> appsInParentQueue =
+        capacityScheduler.getAppsInQueue(parentQueue.getQueueName());
+    assertEquals(expectedNumAppsInParentQueue, appsInParentQueue.size());
+
+    List<ApplicationAttemptId> appsInLeafQueue =
+        capacityScheduler.getAppsInQueue(leafQueueName);
+    assertEquals(expectedNumAppsInLeafQueue, appsInLeafQueue.size());
+
+    return rmApp.getApplicationId();
+  }
+
+  protected List<UserGroupMappingPlacementRule.QueueMapping> setupQueueMapping(
+      CapacityScheduler newCS, String user, String parentQueue, String queue) {
+    List<UserGroupMappingPlacementRule.QueueMapping> queueMappings =
+        new ArrayList<>();
+    queueMappings.add(new UserGroupMappingPlacementRule.QueueMapping(
+        UserGroupMappingPlacementRule.QueueMapping.MappingType.USER, user,
+        getQueueMapping(parentQueue, queue)));
+    newCS.getConfiguration().setQueueMappings(queueMappings);
+    return queueMappings;
+  }
+
+  protected MockRM setupSchedulerInstance() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    setupQueueConfiguration(conf);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    List<String> queuePlacementRules = new ArrayList<String>();
+    queuePlacementRules.add(YarnConfiguration.USER_GROUP_PLACEMENT_RULE);
+    conf.setQueuePlacementRules(queuePlacementRules);
+
+    setupQueueMappings(conf);
+
+    MockRM newMockRM = new MockRM(conf);
+    newMockRM.start();
+    ((CapacityScheduler) newMockRM.getResourceScheduler()).start();
+    return newMockRM;
+  }
+
+  protected void checkQueueCapacities(CapacityScheduler newCS, float capacityC,
+      float capacityD) {
+    CSQueue rootQueue = newCS.getRootQueue();
+    CSQueue queueC = tcs.findQueue(rootQueue, C);
+    CSQueue queueD = tcs.findQueue(rootQueue, D);
+    CSQueue queueC1 = tcs.findQueue(queueC, C1);
+    CSQueue queueC2 = tcs.findQueue(queueC, C2);
+    CSQueue queueC3 = tcs.findQueue(queueC, C3);
+
+    float capC = capacityC / 100.0f;
+    float capD = capacityD / 100.0f;
+
+    tcs.checkQueueCapacity(queueC, capC, capC, 1.0f, 1.0f);
+    tcs.checkQueueCapacity(queueD, capD, capD, 1.0f, 1.0f);
+    tcs.checkQueueCapacity(queueC1, C1_CAPACITY / 100.0f,
+        (C1_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
+    tcs.checkQueueCapacity(queueC2, C2_CAPACITY / 100.0f,
+        (C2_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
+
+    if (queueC3 != null) {
+      ManagedParentQueue parentQueue = (ManagedParentQueue) queueC;
+      QueueCapacities cap =
+          parentQueue.getLeafQueueTemplate().getQueueCapacities();
+      tcs.checkQueueCapacity(queueC3, cap.getCapacity(),
+          (cap.getCapacity()) * capC, 1.0f, 1.0f);
+    }
+  }
+
+  static String getQueueMapping(String parentQueue, String leafQueue) {
+    return parentQueue + DOT + leafQueue;
+  }
+
+  protected ApplicationAttemptId submitApp(CapacityScheduler newCS, String user,
+      String queue, String parentQueue) {
+    ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
+    SchedulerEvent addAppEvent = new AppAddedSchedulerEvent(appId, queue, user,
+        new ApplicationPlacementContext(queue, parentQueue));
+    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
+        appId, 1);
+    SchedulerEvent addAttemptEvent = new AppAttemptAddedSchedulerEvent(
+        appAttemptId, false);
+    newCS.handle(addAppEvent);
+    newCS.handle(addAttemptEvent);
+    return appAttemptId;
+  }
+
+  protected RMApp submitApp(String user, String queue, String nodeLabel)
+      throws Exception {
+    RMApp app = mockRM.submitApp(GB,
+        "test-auto-queue-creation" + RandomUtils.nextInt(100), user, null,
+        queue, nodeLabel);
+    Assert.assertEquals(app.getAmNodeLabelExpression(), nodeLabel);
+    // check preconditions
+    List<ApplicationAttemptId> appsInC = cs.getAppsInQueue(PARENT_QUEUE);
+    assertEquals(1, appsInC.size());
+    assertNotNull(cs.getQueue(queue));
+
+    return app;
+  }
+
+  void setEntitlement(AutoCreatedLeafQueue queue,
+      QueueEntitlement entitlement) {
+    queue.setCapacity(entitlement.getCapacity());
+    queue.setAbsoluteCapacity(
+        queue.getParent().getAbsoluteCapacity() * entitlement.getCapacity());
+    // note: we currently set maxCapacity to capacity
+    // this might be revised later
+    queue.setMaxCapacity(entitlement.getMaxCapacity());
+  }
+
+  protected void validateUserAndAppLimits(
+      AutoCreatedLeafQueue autoCreatedLeafQueue, int maxApps,
+      int maxAppsPerUser) {
+    assertEquals(maxApps, autoCreatedLeafQueue.getMaxApplications());
+    assertEquals(maxAppsPerUser,
+        autoCreatedLeafQueue.getMaxApplicationsPerUser());
+  }
+
+  protected void validateInitialQueueEntitlement(CSQueue parentQueue,
+      String leafQueueName, float expectedTotalChildQueueAbsCapacity)
+      throws SchedulerDynamicEditException {
+    validateInitialQueueEntitlement(cs, parentQueue, leafQueueName,
+        expectedTotalChildQueueAbsCapacity);
+  }
+
+  protected void validateInitialQueueEntitlement(
+      CapacityScheduler capacityScheduler, CSQueue parentQueue,
+      String leafQueueName, float expectedTotalChildQueueAbsCapacity)
+      throws SchedulerDynamicEditException {
+    ManagedParentQueue autoCreateEnabledParentQueue =
+        (ManagedParentQueue) parentQueue;
+
+    GuaranteedOrZeroCapacityOverTimePolicy policy =
+        (GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue
+            .getAutoCreatedQueueManagementPolicy();
+
+    assertEquals(expectedTotalChildQueueAbsCapacity,
+        policy.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
+
+    AutoCreatedLeafQueue leafQueue =
+        (AutoCreatedLeafQueue) capacityScheduler.getQueue(leafQueueName);
+
+    for (String label : accessibleNodeLabelsOnC) {
+      validateCapacitiesByLabel(autoCreateEnabledParentQueue, leafQueue, label);
+    }
+
+    assertEquals(true, policy.isActive(leafQueue));
+  }
+
+  protected void validateCapacitiesByLabel(
+      ManagedParentQueue autoCreateEnabledParentQueue,
+      AutoCreatedLeafQueue leafQueue, String label) {
+    assertEquals(
+        autoCreateEnabledParentQueue.getLeafQueueTemplate().getQueueCapacities()
+            .getCapacity(), leafQueue.getQueueCapacities().getCapacity(label),
+        EPSILON);
+    assertEquals(
+        autoCreateEnabledParentQueue.getLeafQueueTemplate().getQueueCapacities()
+            .getMaximumCapacity(),
+        leafQueue.getQueueCapacities().getMaximumCapacity(label), EPSILON);
+  }
+
+  protected void validateActivatedQueueEntitlement(CSQueue parentQueue,
+      String leafQueueName, float expectedTotalChildQueueAbsCapacity,
+      List<QueueManagementChange> queueManagementChanges)
+      throws SchedulerDynamicEditException {
+    ManagedParentQueue autoCreateEnabledParentQueue =
+        (ManagedParentQueue) parentQueue;
+
+    GuaranteedOrZeroCapacityOverTimePolicy policy =
+        (GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue
+            .getAutoCreatedQueueManagementPolicy();
+
+    QueueCapacities cap = autoCreateEnabledParentQueue.getLeafQueueTemplate()
+        .getQueueCapacities();
+    QueueEntitlement expectedEntitlement = new QueueEntitlement(
+        cap.getCapacity(), cap.getMaximumCapacity());
+
+    //validate capacity
+    validateQueueEntitlements(leafQueueName, expectedEntitlement,
+        queueManagementChanges);
+
+    //validate parent queue state
+    assertEquals(expectedTotalChildQueueAbsCapacity,
+        policy.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
+
+    AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
+        leafQueueName);
+
+    //validate leaf queue state
+    assertEquals(true, policy.isActive(leafQueue));
+  }
+
+  protected void validateDeactivatedQueueEntitlement(CSQueue parentQueue,
+      String leafQueueName, float expectedTotalChildQueueAbsCapacity,
+      List<QueueManagementChange> queueManagementChanges)
+      throws SchedulerDynamicEditException {
+    QueueEntitlement expectedEntitlement = new QueueEntitlement(0.0f, 1.0f);
+
+    ManagedParentQueue autoCreateEnabledParentQueue =
+        (ManagedParentQueue) parentQueue;
+
+    AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
+        leafQueueName);
+
+    GuaranteedOrZeroCapacityOverTimePolicy policy =
+        (GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue
+            .getAutoCreatedQueueManagementPolicy();
+
+    //validate parent queue state
+    assertEquals(expectedTotalChildQueueAbsCapacity,
+        policy.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
+
+    //validate leaf queue state
+    assertEquals(false, policy.isActive(leafQueue));
+
+    //validate capacity
+    validateQueueEntitlements(leafQueueName, expectedEntitlement,
+        queueManagementChanges);
+  }
+
+  private void validateQueueEntitlements(String leafQueueName,
+      QueueEntitlement expectedEntitlement,
+      List<QueueManagementChange> queueEntitlementChanges) {
+    AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
+        leafQueueName);
+    validateQueueEntitlementChangesForLeafQueue(leafQueue, expectedEntitlement,
+        queueEntitlementChanges);
+  }
+
+  private void validateQueueEntitlementChangesForLeafQueue(CSQueue leafQueue,
+      QueueEntitlement expectedQueueEntitlement,
+      final List<QueueManagementChange> queueEntitlementChanges) {
+    boolean found = false;
+    for (QueueManagementChange entitlementChange : queueEntitlementChanges) {
+      if (leafQueue.getQueueName().equals(
+          entitlementChange.getQueue().getQueueName())) {
+
+        AutoCreatedLeafQueueConfig updatedQueueTemplate =
+            entitlementChange.getUpdatedQueueTemplate();
+
+        for (String label : accessibleNodeLabelsOnC) {
+          QueueEntitlement newEntitlement = new QueueEntitlement(
+              updatedQueueTemplate.getQueueCapacities().getCapacity(label),
+              updatedQueueTemplate.getQueueCapacities()
+                  .getMaximumCapacity(label));
+          assertEquals(expectedQueueEntitlement, newEntitlement);
+        }
+        found = true;
+        break;
+      }
+    }
+    if (!found) {
+      fail("Could not find the specified leaf queue in entitlement changes : "
+          + leafQueue.getQueueName());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoQueueCreation.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/TestCapacitySchedulerAutoQueueCreation.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/TestCapacitySchedulerAutoQueueCreation.java
index 7090bc9..049a932 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/TestCapacitySchedulerAutoQueueCreation.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/TestCapacitySchedulerAutoQueueCreation.java
@@ -26,51 +26,54 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.Event;
-import org.apache.hadoop.yarn.event.EventHandler;
-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.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
-import org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule;
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .ApplicationPlacementContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt
+    .RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
-import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
-import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .SchedulerDynamicEditException;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common
+    .QueueEntitlement;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
+    .AppAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
+    .NodeAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
+    .SchedulerEvent;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy
+    .FairOrderingPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.security
+    .ClientToAMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security
+    .NMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security
+    .RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule.CURRENT_USER_MAPPING;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
+import static org.apache.hadoop.yarn.server.resourcemanager.placement
+    .UserGroupMappingPlacementRule.CURRENT_USER_MAPPING;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.CSQueueUtils.EPSILON;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -79,198 +82,14 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 /**
- * Tests for creation and reinitilization of auto created leaf queues
+ * Tests for creation and reinitialization of auto created leaf queues
  * under a ManagedParentQueue.
  */
-public class TestCapacitySchedulerAutoQueueCreation {
-
-  private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
-  private final int GB = 1024;
-  private final static ContainerUpdates NULL_UPDATE_REQUESTS =
-      new ContainerUpdates();
-
-  private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
-  private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-  private static final String C = CapacitySchedulerConfiguration.ROOT + ".c";
-  private static final String D = CapacitySchedulerConfiguration.ROOT + ".d";
-  private static final String A1 = A + ".a1";
-  private static final String A2 = A + ".a2";
-  private static final String B1 = B + ".b1";
-  private static final String B2 = B + ".b2";
-  private static final String B3 = B + ".b3";
-  private static final String C1 = C + ".c1";
-  private static final String C2 = C + ".c2";
-  private static final String C3 = C + ".c3";
-  private static float A_CAPACITY = 20f;
-  private static float B_CAPACITY = 40f;
-  private static float C_CAPACITY = 20f;
-  private static float D_CAPACITY = 20f;
-  private static float A1_CAPACITY = 30;
-  private static float A2_CAPACITY = 70;
-  private static float B1_CAPACITY = 60f;
-  private static float B2_CAPACITY = 20f;
-  private static float B3_CAPACITY = 20f;
-  private static float C1_CAPACITY = 20f;
-  private static float C2_CAPACITY = 20f;
-
-  private static String USER = "user_";
-  private static String USER0 = USER + 0;
-  private static String USER2 = USER + 2;
-  private static String PARENT_QUEUE = "c";
-
-  private MockRM mockRM = null;
-
-  private CapacityScheduler cs;
-
-  private final TestCapacityScheduler tcs = new TestCapacityScheduler();
-
-  private static SpyDispatcher dispatcher;
-
-  private static EventHandler<Event> rmAppEventEventHandler;
-
-  private static class SpyDispatcher extends AsyncDispatcher {
-
-    private static BlockingQueue<Event> eventQueue =
-        new LinkedBlockingQueue<>();
-
-    private static class SpyRMAppEventHandler implements EventHandler<Event> {
-      public void handle(Event event) {
-        eventQueue.add(event);
-      }
-    }
-
-    @Override
-    protected void dispatch(Event event) {
-      eventQueue.add(event);
-    }
-
-    @Override
-    public EventHandler<Event> getEventHandler() {
-      return rmAppEventEventHandler;
-    }
-
-    void spyOnNextEvent(Event expectedEvent, long timeout)
-        throws InterruptedException {
-
-      Event event = eventQueue.poll(timeout, TimeUnit.MILLISECONDS);
-      assertEquals(expectedEvent.getType(), event.getType());
-      assertEquals(expectedEvent.getClass(), event.getClass());
-    }
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
-    setupQueueConfiguration(conf);
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-
-    List<String> queuePlacementRules = new ArrayList<>();
-    queuePlacementRules.add(YarnConfiguration.USER_GROUP_PLACEMENT_RULE);
-    conf.setQueuePlacementRules(queuePlacementRules);
-
-    setupQueueMappings(conf);
-
-    mockRM = new MockRM(conf);
-    cs = (CapacityScheduler) mockRM.getResourceScheduler();
-
-    dispatcher = new SpyDispatcher();
-    rmAppEventEventHandler = new SpyDispatcher.SpyRMAppEventHandler();
-    dispatcher.register(RMAppEventType.class, rmAppEventEventHandler);
-    cs.updatePlacementRules();
-    mockRM.start();
-
-    cs.start();
-  }
-
-  private CapacitySchedulerConfiguration setupQueueMappings(
-      CapacitySchedulerConfiguration conf) {
-
-    //set queue mapping
-    List<UserGroupMappingPlacementRule.QueueMapping> queueMappings =
-        new ArrayList<>();
-    for (int i = 0; i <= 3; i++) {
-      //Set C as parent queue name for auto queue creation
-      UserGroupMappingPlacementRule.QueueMapping userQueueMapping =
-          new UserGroupMappingPlacementRule.QueueMapping(
-              UserGroupMappingPlacementRule.QueueMapping.MappingType.USER,
-              USER + i, getQueueMapping(PARENT_QUEUE, USER + i));
-      queueMappings.add(userQueueMapping);
-    }
+public class TestCapacitySchedulerAutoQueueCreation
+    extends TestCapacitySchedulerAutoCreatedQueueBase {
 
-    conf.setQueueMappings(queueMappings);
-    //override with queue mappings
-    conf.setOverrideWithQueueMappings(true);
-    return conf;
-  }
-
-  /**
-   * @param conf, to be modified
-   * @return, CS configuration which has C
-   * as an auto creation enabled parent queue
-   * <p>
-   * root
-   * /     \      \       \
-   * a        b      c    d
-   * / \    /  |  \
-   * a1  a2 b1  b2  b3
-   */
-  private CapacitySchedulerConfiguration setupQueueConfiguration(
-      CapacitySchedulerConfiguration conf) {
-
-    //setup new queues with one of them auto enabled
-    // Define top-level queues
-    // Set childQueue for root
-    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
-        new String[] { "a", "b", "c", "d" });
-
-    conf.setCapacity(A, A_CAPACITY);
-    conf.setCapacity(B, B_CAPACITY);
-    conf.setCapacity(C, C_CAPACITY);
-    conf.setCapacity(D, D_CAPACITY);
-
-    // Define 2nd-level queues
-    conf.setQueues(A, new String[] { "a1", "a2" });
-    conf.setCapacity(A1, A1_CAPACITY);
-    conf.setUserLimitFactor(A1, 100.0f);
-    conf.setCapacity(A2, A2_CAPACITY);
-    conf.setUserLimitFactor(A2, 100.0f);
-
-    conf.setQueues(B, new String[] { "b1", "b2", "b3" });
-    conf.setCapacity(B1, B1_CAPACITY);
-    conf.setUserLimitFactor(B1, 100.0f);
-    conf.setCapacity(B2, B2_CAPACITY);
-    conf.setUserLimitFactor(B2, 100.0f);
-    conf.setCapacity(B3, B3_CAPACITY);
-    conf.setUserLimitFactor(B3, 100.0f);
-
-    conf.setUserLimitFactor(C, 1.0f);
-    conf.setAutoCreateChildQueueEnabled(C, true);
-
-    //Setup leaf queue template configs
-    conf.setAutoCreatedLeafQueueTemplateCapacity(C, 50.0f);
-    conf.setAutoCreatedLeafQueueTemplateMaxCapacity(C, 100.0f);
-
-    LOG.info("Setup " + C + " as an auto leaf creation enabled parent queue");
-
-    conf.setUserLimitFactor(D, 1.0f);
-    conf.setAutoCreateChildQueueEnabled(D, true);
-
-    //Setup leaf queue template configs
-    conf.setAutoCreatedLeafQueueTemplateCapacity(D, 10.0f);
-    conf.setAutoCreatedLeafQueueTemplateMaxCapacity(D, 100.0f);
-
-    LOG.info("Setup " + D + " as an auto leaf creation enabled parent queue");
-
-    return conf;
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (mockRM != null) {
-      mockRM.stop();
-    }
-  }
+  private static final Log LOG = LogFactory.getLog(
+      TestCapacitySchedulerAutoQueueCreation.class);
 
   @Test(timeout = 10000)
   public void testAutoCreateLeafQueueCreation() throws Exception {
@@ -289,7 +108,11 @@ public class TestCapacitySchedulerAutoQueueCreation {
       ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue(
           PARENT_QUEUE);
       assertEquals(parentQueue, autoCreatedLeafQueue.getParent());
-      validateCapacities(autoCreatedLeafQueue);
+      validateInitialQueueEntitlement(parentQueue, USER0, 0.1f);
+      validateUserAndAppLimits(autoCreatedLeafQueue, 1000, 1000);
+
+      assertTrue(autoCreatedLeafQueue
+          .getOrderingPolicy() instanceof FairOrderingPolicy);
     } finally {
       cleanupQueue(USER0);
     }
@@ -297,7 +120,6 @@ public class TestCapacitySchedulerAutoQueueCreation {
 
   @Test
   public void testReinitializeStoppedAutoCreatedLeafQueue() throws Exception {
-
     try {
       String host = "127.0.0.1";
       RMNode node = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1,
@@ -306,20 +128,28 @@ public class TestCapacitySchedulerAutoQueueCreation {
 
       // submit an app
 
-      RMApp app = mockRM.submitApp(GB, "test-auto-queue-creation-1", USER0,
+      RMApp app1 = mockRM.submitApp(GB, "test-auto-queue-creation-1", USER0,
           null, USER0);
+
+      RMApp app2 = mockRM.submitApp(GB, "test-auto-queue-creation-2", USER1,
+          null, USER1);
       // check preconditions
       List<ApplicationAttemptId> appsInC = cs.getAppsInQueue(PARENT_QUEUE);
-      assertEquals(1, appsInC.size());
+      assertEquals(2, appsInC.size());
 
       assertNotNull(cs.getQueue(USER0));
+      assertNotNull(cs.getQueue(USER1));
 
-      AutoCreatedLeafQueue autoCreatedLeafQueue =
-          (AutoCreatedLeafQueue) cs.getQueue(USER0);
+      AutoCreatedLeafQueue user0Queue = (AutoCreatedLeafQueue) cs.getQueue(
+          USER0);
+      AutoCreatedLeafQueue user1Queue = (AutoCreatedLeafQueue) cs.getQueue(
+          USER0);
       ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue(
           PARENT_QUEUE);
-      assertEquals(parentQueue, autoCreatedLeafQueue.getParent());
-      validateCapacities(autoCreatedLeafQueue);
+      assertEquals(parentQueue, user0Queue.getParent());
+      assertEquals(parentQueue, user1Queue.getParent());
+      validateInitialQueueEntitlement(parentQueue, USER0, 0.2f);
+      validateInitialQueueEntitlement(parentQueue, USER1, 0.2f);
 
       ApplicationAttemptId appAttemptId = appsInC.get(0);
 
@@ -337,7 +167,7 @@ public class TestCapacitySchedulerAutoQueueCreation {
       CapacityScheduler.schedule(cs);
 
       //change state to draining
-      autoCreatedLeafQueue.stopQueue();
+      user0Queue.stopQueue();
 
       cs.killAllAppsInQueue(USER0);
 
@@ -346,80 +176,24 @@ public class TestCapacitySchedulerAutoQueueCreation {
       mockRM.waitForState(appAttemptId.getApplicationId(), RMAppState.KILLED);
 
       //change state to stopped
-      autoCreatedLeafQueue.stopQueue();
+      user0Queue.stopQueue();
       assertEquals(QueueState.STOPPED,
-          autoCreatedLeafQueue.getQueueInfo().getQueueState());
+          user0Queue.getQueueInfo().getQueueState());
 
       cs.reinitialize(cs.getConf(), mockRM.getRMContext());
 
-      AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
-          USER0);
-      validateCapacities(leafQueue);
-
-    } finally {
-      cleanupQueue(USER0);
-    }
-  }
-
-  @Test
-  public void testRefreshQueuesWithAutoCreatedLeafQueues() throws Exception {
-
-    MockRM newMockRM = setupSchedulerInstance();
-    try {
-      CapacityScheduler newCS =
-          (CapacityScheduler) newMockRM.getResourceScheduler();
-      CapacitySchedulerConfiguration conf = newCS.getConfiguration();
-
-      // Test add one auto created queue dynamically and manually modify
-      // capacity
-      ManagedParentQueue parentQueue = (ManagedParentQueue) newCS.getQueue("c");
-      AutoCreatedLeafQueue c1 = new AutoCreatedLeafQueue(newCS, "c1",
-          parentQueue);
-      newCS.addQueue(c1);
-      c1.setEntitlement(new QueueEntitlement(C1_CAPACITY / 100, 1f));
-
-      // Test add another auto created queue and use setEntitlement to modify
-      // capacity
-      AutoCreatedLeafQueue c2 = new AutoCreatedLeafQueue(newCS, "c2",
-          (ManagedParentQueue) newCS.getQueue("c"));
-      newCS.addQueue(c2);
-      newCS.setEntitlement("c2", new QueueEntitlement(C2_CAPACITY / 100, 1f));
-
-      // Verify all allocations match
-      checkQueueCapacities(newCS, C_CAPACITY, D_CAPACITY);
-
-      // Reinitialize and verify all dynamic queued survived
-
-      conf.setCapacity(A, 20f);
-      conf.setCapacity(B, 20f);
-      conf.setCapacity(C, 40f);
-      conf.setCapacity(D, 20f);
-      newCS.reinitialize(conf, newMockRM.getRMContext());
-
-      checkQueueCapacities(newCS, 40f, 20f);
-
-      //chnage parent template configs and reinitialize
-      conf.setAutoCreatedLeafQueueTemplateCapacity(C, 30.0f);
-      conf.setAutoCreatedLeafQueueTemplateMaxCapacity(C, 100.0f);
-      newCS.reinitialize(conf, newMockRM.getRMContext());
+      AutoCreatedLeafQueue user0QueueReinited =
+          (AutoCreatedLeafQueue) cs.getQueue(USER0);
 
-      ManagedParentQueue c = (ManagedParentQueue) newCS.getQueue("c");
-      AutoCreatedLeafQueue c3 = new AutoCreatedLeafQueue(newCS, "c3", c);
-      newCS.addQueue(c3);
+      validateCapacities(user0QueueReinited, 0.0f, 0.0f, 1.0f, 1.0f);
 
-      AbstractManagedParentQueue.AutoCreatedLeafQueueTemplate
-          leafQueueTemplate = parentQueue.getLeafQueueTemplate();
-      QueueCapacities cap = leafQueueTemplate.getQueueCapacities();
-      c3.setEntitlement(
-          new QueueEntitlement(cap.getCapacity(), cap.getMaximumCapacity()));
-      newCS.reinitialize(conf, newMockRM.getRMContext());
+      AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
+          USER1);
+      validateInitialQueueEntitlement(parentQueue, leafQueue.getQueueName(),
+          0.1f);
 
-      checkQueueCapacities(newCS, 40f, 20f);
     } finally {
-      if (newMockRM != null) {
-        ((CapacityScheduler) newMockRM.getResourceScheduler()).stop();
-        newMockRM.stop();
-      }
+      cleanupQueue(USER0);
     }
   }
 
@@ -460,7 +234,7 @@ public class TestCapacitySchedulerAutoQueueCreation {
       CapacitySchedulerConfiguration newConf =
           new CapacitySchedulerConfiguration();
       setupQueueConfiguration(newConf);
-      newConf.setAutoCreatedLeafQueueTemplateCapacity(A1, A1_CAPACITY / 10);
+      newConf.setAutoCreatedLeafQueueConfigCapacity(A1, A1_CAPACITY / 10);
       newConf.setAutoCreateChildQueueEnabled(A1, true);
 
       newCS.setConf(new YarnConfiguration());
@@ -490,7 +264,7 @@ public class TestCapacitySchedulerAutoQueueCreation {
       CapacitySchedulerConfiguration newConf =
           new CapacitySchedulerConfiguration();
       setupQueueConfiguration(newConf);
-      newConf.setAutoCreatedLeafQueueTemplateCapacity(A, A_CAPACITY / 10);
+      newConf.setAutoCreatedLeafQueueConfigCapacity(A, A_CAPACITY / 10);
       newConf.setAutoCreateChildQueueEnabled(A, true);
 
       newCS.setConf(new YarnConfiguration());
@@ -531,39 +305,6 @@ public class TestCapacitySchedulerAutoQueueCreation {
     assertEquals(RMAppState.FAILED, app.getState());
   }
 
-  private void validateCapacities(AutoCreatedLeafQueue autoCreatedLeafQueue) {
-    assertEquals(autoCreatedLeafQueue.getCapacity(), 0.0f, EPSILON);
-    assertEquals(autoCreatedLeafQueue.getAbsoluteCapacity(), 0.0f, EPSILON);
-    assertEquals(autoCreatedLeafQueue.getMaximumCapacity(), 0.0f, EPSILON);
-    assertEquals(autoCreatedLeafQueue.getAbsoluteMaximumCapacity(), 0.0f,
-        EPSILON);
-    int maxAppsForAutoCreatedQueues = (int) (
-        CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS
-            * autoCreatedLeafQueue.getParent().getAbsoluteCapacity());
-    assertEquals(autoCreatedLeafQueue.getMaxApplicationsPerUser(),
-        maxAppsForAutoCreatedQueues);
-    assertEquals(autoCreatedLeafQueue.getMaxApplicationsPerUser(),
-        (int) (maxAppsForAutoCreatedQueues * (cs.getConfiguration()
-            .getUserLimitFactor(
-                autoCreatedLeafQueue.getParent().getQueuePath()))));
-  }
-
-  private void cleanupQueue(String queueName) throws YarnException {
-    AutoCreatedLeafQueue queue = (AutoCreatedLeafQueue) cs.getQueue(queueName);
-    if (queue != null) {
-      queue.setEntitlement(new QueueEntitlement(0.0f, 0.0f));
-      ((ManagedParentQueue) queue.getParent()).removeChildQueue(
-          queue.getQueueName());
-      cs.getCapacitySchedulerQueueManager().removeQueue(queue.getQueueName());
-    } else{
-      throw new YarnException("Queue does not exist " + queueName);
-    }
-  }
-
-  String getQueueMapping(String parentQueue, String leafQueue) {
-    return parentQueue + DOT + leafQueue;
-  }
-
   @Test(timeout = 10000)
   public void testQueueMappingValidationFailsWithInvalidParentQueueInMapping()
       throws Exception {
@@ -586,8 +327,7 @@ public class TestCapacitySchedulerAutoQueueCreation {
         //expected exception
         assertTrue(e.getMessage().contains(
             "invalid parent queue which does not have auto creation of leaf "
-                + "queues enabled ["
-                + "a" + "]"));
+                + "queues enabled [" + "a" + "]"));
       }
 
       //"a" is not auto create enabled and app_user does not exist as a leaf
@@ -650,9 +390,6 @@ public class TestCapacitySchedulerAutoQueueCreation {
         (CapacityScheduler) newMockRM.getResourceScheduler();
 
     try {
-      newMockRM.start();
-      newCS.start();
-
       submitApp(newCS, USER0, USER0, PARENT_QUEUE);
 
       assertNotNull(newCS.getQueue(USER0));
@@ -700,12 +437,16 @@ public class TestCapacitySchedulerAutoQueueCreation {
       AutoCreatedLeafQueue c1 = new AutoCreatedLeafQueue(newCS, "c1",
           parentQueue);
       newCS.addQueue(c1);
-      c1.setEntitlement(new QueueEntitlement(0.5f, 1f));
+      c1.setCapacity(0.5f);
+      c1.setAbsoluteCapacity(c1.getParent().getAbsoluteCapacity() * 1f);
+      c1.setMaxCapacity(1f);
+
+      setEntitlement(c1, new QueueEntitlement(0.5f, 1f));
 
       AutoCreatedLeafQueue c2 = new AutoCreatedLeafQueue(newCS, "c2",
           parentQueue);
       newCS.addQueue(c2);
-      c2.setEntitlement(new QueueEntitlement(0.5f, 1f));
+      setEntitlement(c2, new QueueEntitlement(0.5f, 1f));
 
       try {
         AutoCreatedLeafQueue c3 = new AutoCreatedLeafQueue(newCS, "c3",
@@ -723,72 +464,160 @@ public class TestCapacitySchedulerAutoQueueCreation {
     }
   }
 
-  private List<UserGroupMappingPlacementRule.QueueMapping> setupQueueMapping(
-      CapacityScheduler newCS, String user, String parentQueue, String queue) {
-    List<UserGroupMappingPlacementRule.QueueMapping> queueMappings =
-        new ArrayList<>();
-    queueMappings.add(new UserGroupMappingPlacementRule.QueueMapping(
-        UserGroupMappingPlacementRule.QueueMapping.MappingType.USER, user,
-        getQueueMapping(parentQueue, queue)));
-    newCS.getConfiguration().setQueueMappings(queueMappings);
-    return queueMappings;
-  }
+  @Test
+  public void testAutoCreatedQueueActivationDeactivation() throws Exception {
+
+    try {
+      String host = "127.0.0.1";
+      RMNode node = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1,
+          host);
+      cs.handle(new NodeAddedSchedulerEvent(node));
+
+      CSQueue parentQueue = cs.getQueue(PARENT_QUEUE);
+
+      //submit app1 as USER1
+      submitApp(mockRM, parentQueue, USER1, USER1, 1, 1);
+      validateInitialQueueEntitlement(parentQueue, USER1, 0.1f);
+
+      //submit another app2 as USER2
+      ApplicationId user2AppId = submitApp(mockRM, parentQueue, USER2, USER2, 2,
+          1);
+      validateInitialQueueEntitlement(parentQueue, USER2, 0.2f);
+
+      //submit another app3 as USER1
+      submitApp(mockRM, parentQueue, USER1, USER1, 3, 2);
+
+      //validate total activated abs capacity remains the same
+      GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy =
+          (GuaranteedOrZeroCapacityOverTimePolicy) ((ManagedParentQueue)
+              parentQueue)
+              .getAutoCreatedQueueManagementPolicy();
+      assertEquals(autoCreatedQueueManagementPolicy
+          .getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON);
+
+      //submit user_3 app. This cant be scheduled since there is no capacity
+      submitApp(mockRM, parentQueue, USER3, USER3, 4, 1);
+      final CSQueue user3LeafQueue = cs.getQueue(USER3);
+      validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.0f, 0.0f,
+          1.0f, 1.0f);
 
-  private MockRM setupSchedulerInstance() {
-    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
-    setupQueueConfiguration(conf);
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
+      assertEquals(autoCreatedQueueManagementPolicy
+          .getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON);
 
-    List<String> queuePlacementRules = new ArrayList<String>();
-    queuePlacementRules.add(YarnConfiguration.USER_GROUP_PLACEMENT_RULE);
-    conf.setQueuePlacementRules(queuePlacementRules);
+      //deactivate USER2 queue
+      cs.killAllAppsInQueue(USER2);
+      mockRM.waitForState(user2AppId, RMAppState.KILLED);
 
-    setupQueueMappings(conf);
+      //Verify if USER_2 can be deactivated since it has no pending appsA
+      List<QueueManagementChange> queueManagementChanges =
+          autoCreatedQueueManagementPolicy.computeQueueManagementChanges();
 
-    MockRM newMockRM = new MockRM(conf);
-    return newMockRM;
+      ManagedParentQueue managedParentQueue = (ManagedParentQueue) parentQueue;
+      managedParentQueue.validateAndApplyQueueManagementChanges(
+          queueManagementChanges);
+
+      validateDeactivatedQueueEntitlement(parentQueue, USER2, 0.2f,
+          queueManagementChanges);
+
+      //USER_3 should now get activated
+      validateActivatedQueueEntitlement(parentQueue, USER3, 0.2f,
+          queueManagementChanges);
+
+    } finally {
+      cleanupQueue(USER1);
+      cleanupQueue(USER2);
+      cleanupQueue(USER3);
+    }
   }
 
-  void checkQueueCapacities(CapacityScheduler newCS, float capacityC,
-      float capacityD) {
-    CSQueue rootQueue = newCS.getRootQueue();
-    CSQueue queueC = tcs.findQueue(rootQueue, C);
-    CSQueue queueD = tcs.findQueue(rootQueue, D);
-    CSQueue queueC1 = tcs.findQueue(queueC, C1);
-    CSQueue queueC2 = tcs.findQueue(queueC, C2);
-    CSQueue queueC3 = tcs.findQueue(queueC, C3);
-
-    float capC = capacityC / 100.0f;
-    float capD = capacityD / 100.0f;
-
-    tcs.checkQueueCapacity(queueC, capC, capC, 1.0f, 1.0f);
-    tcs.checkQueueCapacity(queueD, capD, capD, 1.0f, 1.0f);
-    tcs.checkQueueCapacity(queueC1, C1_CAPACITY / 100.0f,
-        (C1_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
-    tcs.checkQueueCapacity(queueC2, C2_CAPACITY / 100.0f,
-        (C2_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
-
-    if (queueC3 != null) {
-      ManagedParentQueue parentQueue = (ManagedParentQueue) queueC;
-      QueueCapacities cap =
-          parentQueue.getLeafQueueTemplate().getQueueCapacities();
-      tcs.checkQueueCapacity(queueC3, cap.getCapacity(),
-          (cap.getCapacity()) * capC, 1.0f, 1.0f);
+  @Test
+  public void testAutoCreatedQueueInheritsNodeLabels() throws Exception {
+
+    try {
+      String host = "127.0.0.1";
+      RMNode node = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1,
+          host);
+      cs.handle(new NodeAddedSchedulerEvent(node));
+
+      CSQueue parentQueue = cs.getQueue(PARENT_QUEUE);
+
+      submitApp(USER1, USER1, NODEL_LABEL_GPU);
+      //submit app1 as USER1
+      validateInitialQueueEntitlement(parentQueue, USER1, 0.1f);
+    } finally {
+      cleanupQueue(USER1);
     }
   }
 
-  ApplicationAttemptId submitApp(CapacityScheduler newCS, String user,
-      String queue, String parentQueue) {
-    ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
-    SchedulerEvent addAppEvent = new AppAddedSchedulerEvent(appId, queue, user,
-        new ApplicationPlacementContext(queue, parentQueue));
-    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
-        appId, 1);
-    SchedulerEvent addAttemptEvent = new AppAttemptAddedSchedulerEvent(
-        appAttemptId, false);
-    newCS.handle(addAppEvent);
-    newCS.handle(addAttemptEvent);
-    return appAttemptId;
+  @Test
+  public void testReinitializeQueuesWithAutoCreatedLeafQueues()
+      throws Exception {
+
+    MockRM newMockRM = setupSchedulerInstance();
+    try {
+      CapacityScheduler newCS =
+          (CapacityScheduler) newMockRM.getResourceScheduler();
+      CapacitySchedulerConfiguration conf = newCS.getConfiguration();
+
+      String host = "127.0.0.1";
+      RMNode node = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1,
+          host);
+      newCS.handle(new NodeAddedSchedulerEvent(node));
+
+      CSQueue parentQueue = newCS.getQueue(PARENT_QUEUE);
+
+      //submit app1 as USER1
+      submitApp(newMockRM, parentQueue, USER1, USER1, 1, 1);
+      validateInitialQueueEntitlement(newCS, parentQueue, USER1, 0.1f);
+
+      //submit another app2 as USER2
+      ApplicationId user2AppId = submitApp(newMockRM, parentQueue, USER2, USER2,
+          2, 1);
+      validateInitialQueueEntitlement(newCS, parentQueue, USER2, 0.2f);
+
+      //update parent queue capacity
+      conf.setCapacity(C, 30f);
+      conf.setCapacity(D, 10f);
+      conf.setMaximumCapacity(C, 50f);
+
+      newCS.reinitialize(conf, newMockRM.getRMContext());
+
+      // validate that leaf queues abs capacity is now changed
+      AutoCreatedLeafQueue user0Queue = (AutoCreatedLeafQueue) newCS.getQueue(
+          USER1);
+      validateCapacities(user0Queue, 0.5f, 0.15f, 1.0f, 0.5f);
+      validateUserAndAppLimits(user0Queue, 1500, 1500);
+
+      //update leaf queue template capacities
+      conf.setAutoCreatedLeafQueueConfigCapacity(C, 30f);
+      conf.setAutoCreatedLeafQueueConfigMaxCapacity(C, 40f);
+
+      newCS.reinitialize(conf, newMockRM.getRMContext());
+      validateCapacities(user0Queue, 0.3f, 0.09f, 0.4f, 0.2f);
+      validateUserAndAppLimits(user0Queue, 900, 900);
+
+      //submit app1 as USER3
+      submitApp(newMockRM, parentQueue, USER3, USER3, 3, 1);
+      validateInitialQueueEntitlement(newCS, parentQueue, USER3, 0.27f);
+      AutoCreatedLeafQueue user3Queue = (AutoCreatedLeafQueue) newCS.getQueue(
+          USER1);
+      validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f, 0.2f);
+      validateUserAndAppLimits(user3Queue, 900, 900);
+
+      //submit app1 as USER1 - is already activated. there should be no diff
+      // in capacities
+      submitApp(newMockRM, parentQueue, USER3, USER3, 4, 2);
+      validateInitialQueueEntitlement(newCS, parentQueue, USER3, 0.27f);
+      validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f, 0.2f);
+      validateUserAndAppLimits(user3Queue, 900, 900);
+
+    } finally {
+      cleanupQueue(USER1);
+      cleanupQueue(USER2);
+      if (newMockRM != null) {
+        ((CapacityScheduler) newMockRM.getResourceScheduler()).stop();
+        newMockRM.stop();
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.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/TestCapacitySchedulerDynamicBehavior.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/TestCapacitySchedulerDynamicBehavior.java
index 9425d5e..9aba30c 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/TestCapacitySchedulerDynamicBehavior.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/TestCapacitySchedulerDynamicBehavior.java
@@ -77,21 +77,21 @@ public class TestCapacitySchedulerDynamicBehavior {
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
 
     //set default queue capacity to zero
-    ((AutoCreatedLeafQueue) cs
+    ((ReservationQueue) cs
             .getQueue("a" + ReservationConstants.DEFAULT_QUEUE_SUFFIX))
             .setEntitlement(
                     new QueueEntitlement(0f, 1f));
 
     // Test add one reservation dynamically and manually modify capacity
-    AutoCreatedLeafQueue a1 =
-        new AutoCreatedLeafQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
+    ReservationQueue a1 =
+        new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
     cs.addQueue(a1);
     a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
 
     // Test add another reservation queue and use setEntitlement to modify
     // capacity
-    AutoCreatedLeafQueue a2 =
-        new AutoCreatedLeafQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
+    ReservationQueue a2 =
+        new ReservationQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
     cs.addQueue(a2);
     cs.setEntitlement("a2", new QueueEntitlement(A2_CAPACITY / 100, 1.0f));
 
@@ -113,8 +113,8 @@ public class TestCapacitySchedulerDynamicBehavior {
 
     try {
       // Test invalid addition (adding non-zero size queue)
-      AutoCreatedLeafQueue a1 =
-          new AutoCreatedLeafQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
+      ReservationQueue a1 =
+          new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
       a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
       cs.addQueue(a1);
       fail();
@@ -123,11 +123,11 @@ public class TestCapacitySchedulerDynamicBehavior {
     }
 
     // Test add one reservation dynamically and manually modify capacity
-    AutoCreatedLeafQueue a1 =
-        new AutoCreatedLeafQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
+    ReservationQueue a1 =
+        new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
     cs.addQueue(a1);
     //set default queue capacity to zero
-    ((AutoCreatedLeafQueue) cs
+    ((ReservationQueue) cs
         .getQueue("a" + ReservationConstants.DEFAULT_QUEUE_SUFFIX))
             .setEntitlement(
                 new QueueEntitlement(0f, 1f));
@@ -135,8 +135,8 @@ public class TestCapacitySchedulerDynamicBehavior {
 
     // Test add another reservation queue and use setEntitlement to modify
     // capacity
-    AutoCreatedLeafQueue a2 =
-        new AutoCreatedLeafQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
+    ReservationQueue a2 =
+        new ReservationQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
 
     cs.addQueue(a2);
 
@@ -162,8 +162,8 @@ public class TestCapacitySchedulerDynamicBehavior {
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
 
     // Test add one reservation dynamically and manually modify capacity
-    AutoCreatedLeafQueue a1 =
-        new AutoCreatedLeafQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
+    ReservationQueue a1 =
+        new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
     cs.addQueue(a1);
     a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
 
@@ -230,8 +230,8 @@ public class TestCapacitySchedulerDynamicBehavior {
 
     // create the default reservation queue
     String defQName = "a" + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
-    AutoCreatedLeafQueue defQ =
-        new AutoCreatedLeafQueue(scheduler, defQName,
+    ReservationQueue defQ =
+        new ReservationQueue(scheduler, defQName,
             (PlanQueue) scheduler.getQueue("a"));
     scheduler.addQueue(defQ);
     defQ.setEntitlement(new QueueEntitlement(1f, 1f));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestGuaranteedOrZeroCapacityOverTimePolicy.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/TestGuaranteedOrZeroCapacityOverTimePolicy.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/TestGuaranteedOrZeroCapacityOverTimePolicy.java
new file mode 100644
index 0000000..c71d2bf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestGuaranteedOrZeroCapacityOverTimePolicy.java
@@ -0,0 +1,40 @@
+/**
+ * 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.scheduler.capacity;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestGuaranteedOrZeroCapacityOverTimePolicy {
+
+  @Test
+  public void testGetMaxLeavesToBeActivated()
+      throws SchedulerDynamicEditException {
+    GuaranteedOrZeroCapacityOverTimePolicy policy =
+        new GuaranteedOrZeroCapacityOverTimePolicy();
+
+    assertEquals(1, policy.getMaxLeavesToBeActivated(0.17f, 0.03f, 1));
+    assertEquals(5, policy.getMaxLeavesToBeActivated(0.17f, 0.03f, 7));
+    assertEquals(0, policy.getMaxLeavesToBeActivated(0, 0.03f, 10));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/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 1426e88..c45bdb4 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
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
+    .NO_LABEL;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.CSQueueUtils.EPSILON;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.CapacitySchedulerConfiguration.DOT;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.CapacitySchedulerConfiguration.ROOT;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -44,6 +52,7 @@ import java.util.concurrent.CyclicBarrier;
 import com.google.common.collect.ImmutableMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -189,7 +198,7 @@ public class TestLeafQueue {
 
     root = 
         CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
-            CapacitySchedulerConfiguration.ROOT, 
+            ROOT,
             queues, queues, 
             TestUtils.spyHook);
     root.updateClusterResource(Resources.createResource(100 * 16 * GB, 100 * 32),
@@ -222,12 +231,12 @@ public class TestLeafQueue {
       final String newRoot) {
     
     // Define top-level queues
-    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {newRoot});
-    conf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT, 100);
-    conf.setAcl(CapacitySchedulerConfiguration.ROOT,
+    conf.setQueues(ROOT, new String[] {newRoot});
+    conf.setMaximumCapacity(ROOT, 100);
+    conf.setAcl(ROOT,
       QueueACL.SUBMIT_APPLICATIONS, " ");
     
-    final String Q_newRoot = CapacitySchedulerConfiguration.ROOT + "." + newRoot;
+    final String Q_newRoot = ROOT + "." + newRoot;
     conf.setQueues(Q_newRoot, new String[] {A, B, C, D, E});
     conf.setCapacity(Q_newRoot, 100);
     conf.setMaximumCapacity(Q_newRoot, 100);
@@ -410,7 +419,7 @@ public class TestLeafQueue {
     CapacitySchedulerConfiguration testConf = 
         new CapacitySchedulerConfiguration();
     
-    String tproot = CapacitySchedulerConfiguration.ROOT + "." + 
+    String tproot = ROOT + "." +
       "testPolicyRoot" + System.currentTimeMillis();
 
     OrderingPolicy<FiCaSchedulerApp> comPol =    
@@ -485,7 +494,7 @@ public class TestLeafQueue {
     CapacitySchedulerConfiguration testConf =
         new CapacitySchedulerConfiguration();
 
-    String tproot = CapacitySchedulerConfiguration.ROOT + "." +
+    String tproot = ROOT + "." +
       "testPolicyRoot" + System.currentTimeMillis();
 
     OrderingPolicy<FiCaSchedulerApp> schedOrder =
@@ -722,12 +731,12 @@ public class TestLeafQueue {
     Priority priority = TestUtils.createMockPriority(1);
     app0.updateResourceRequests(Collections.singletonList(TestUtils
         .createResourceRequest(ResourceRequest.ANY, 20 * GB, 29, 1, true,
-            priority, recordFactory, RMNodeLabelsManager.NO_LABEL)));
+            priority, recordFactory, NO_LABEL)));
     assign = b.assignContainers(clusterResource, node0, new ResourceLimits(
         clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     app0.updateResourceRequests(Collections.singletonList(TestUtils
         .createResourceRequest(ResourceRequest.ANY, 10 * GB, 29, 2, true,
-            priority, recordFactory, RMNodeLabelsManager.NO_LABEL)));
+            priority, recordFactory, NO_LABEL)));
     assign = b.assignContainers(clusterResource, node0, new ResourceLimits(
         clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     assertTrue("Still within limits, should assign",
@@ -796,11 +805,11 @@ public class TestLeafQueue {
     Priority priority = TestUtils.createMockPriority(1);
     app0.updateResourceRequests(Collections.singletonList(TestUtils
         .createResourceRequest(ResourceRequest.ANY, 1 * GB, 40, 10, true,
-            priority, recordFactory, RMNodeLabelsManager.NO_LABEL)));
+            priority, recordFactory, NO_LABEL)));
 
     app2.updateResourceRequests(Collections.singletonList(TestUtils
         .createResourceRequest(ResourceRequest.ANY, 2 * GB, 10, 10, true,
-            priority, recordFactory, RMNodeLabelsManager.NO_LABEL)));
+            priority, recordFactory, NO_LABEL)));
 
     /**
      * Start testing...
@@ -2277,7 +2286,7 @@ public class TestLeafQueue {
         CapacitySchedulerConfiguration.RACK_LOCALITY_ADDITIONAL_DELAY, 1);
     Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
     CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext,
-        csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues,
+        csConf, null, ROOT, newQueues, queues,
         TestUtils.spyHook);
     root.reinitialize(newRoot, cs.getClusterResource());
 
@@ -2712,9 +2721,12 @@ public class TestLeafQueue {
         CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT
             * 2);
     Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
-    CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext,
-        csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues,
-        TestUtils.spyHook);
+    CSQueue newRoot =
+        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
+            ROOT,
+            newQueues, queues,
+            TestUtils.spyHook);
+    queues = newQueues;
     root.reinitialize(newRoot, csContext.getClusterResource());
 
     // after reinitialization
@@ -2738,7 +2750,7 @@ public class TestLeafQueue {
     Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
     CSQueue newRoot =
         CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
-            CapacitySchedulerConfiguration.ROOT,
+            ROOT,
             newQueues, queues,
             TestUtils.spyHook);
     root.reinitialize(newRoot, cs.getClusterResource());
@@ -3621,7 +3633,7 @@ public class TestLeafQueue {
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(0 * GB,
         e.getTotalPendingResourcesConsideringUserLimit(clusterResource,
-            RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+            NO_LABEL, false).getMemorySize());
 
     // Assign 2nd container of 1GB
     applyCSAssignment(clusterResource,
@@ -3635,7 +3647,7 @@ public class TestLeafQueue {
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
 
     // Can't allocate 3rd container due to user-limit. Headroom still 0.
     applyCSAssignment(clusterResource,
@@ -3645,7 +3657,7 @@ public class TestLeafQueue {
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
 
     // Increase user-limit-factor from 1GB to 10GB (1% * 10 * 100GB = 10GB).
     // Pending for both app_0 and app_1 are still 3GB, so user-limit-factor
@@ -3653,7 +3665,7 @@ public class TestLeafQueue {
     // getTotalPendingResourcesConsideringUserLimit()
     e.setUserLimitFactor(10.0f);
     assertEquals(3*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
 
     applyCSAssignment(clusterResource,
         e.assignContainers(clusterResource, node_0,
@@ -3663,7 +3675,7 @@ public class TestLeafQueue {
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(2*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
 
     // Get the last 2 containers for app_1, no more pending requests.
     applyCSAssignment(clusterResource,
@@ -3677,7 +3689,7 @@ public class TestLeafQueue {
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(2*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
 
     // Release each container from app_0
     for (RMContainer rmContainer : app_0.getLiveContainers()) {
@@ -3788,7 +3800,7 @@ public class TestLeafQueue {
     // With queue capacity set at 1% of 100GB and user-limit-factor set to 1.0,
     // queue 'e' should be able to consume 1GB per user.
     assertEquals(2*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
     // None of the apps have assigned resources
     // user_0's apps:
     assertEquals(0*GB, app_0.getCurrentConsumption().getMemorySize());
@@ -3805,7 +3817,7 @@ public class TestLeafQueue {
    // The first container was assigned to user_0's app_0. Queues total headroom
     // has 1GB left for user_1.
     assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
     // user_0's apps:
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
@@ -3823,7 +3835,7 @@ public class TestLeafQueue {
     // this container went to user_0's app_1. so, headroom for queue 'e'e is
     // still 1GB for user_1
     assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
     // user_0's apps:
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
@@ -3839,7 +3851,7 @@ public class TestLeafQueue {
     // Container was allocated to user_1's app_2 since user_1, Now, no headroom
     // is left.
     assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
     // user_0's apps:
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
@@ -3855,7 +3867,7 @@ public class TestLeafQueue {
     // Allocated to user_1's app_2 since scheduler allocates 1 container
     // above user resource limit. Available headroom still 0.
     assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
     // user_0's apps:
     long app_0_consumption = app_0.getCurrentConsumption().getMemorySize();
     assertEquals(1*GB, app_0_consumption);
@@ -3875,7 +3887,7 @@ public class TestLeafQueue {
     // Cannot allocate 5th container because both users are above their allowed
     // user resource limit. Values should be the same as previously.
     assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
     // user_0's apps:
     assertEquals(app_0_consumption, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(app_1_consumption, app_1.getCurrentConsumption().getMemorySize());
@@ -3894,7 +3906,7 @@ public class TestLeafQueue {
         SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps);
     // Next container goes to user_0's app_1, since it still wanted 1GB.
     assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
     // user_0's apps:
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(2*GB, app_1.getCurrentConsumption().getMemorySize());
@@ -3909,7 +3921,7 @@ public class TestLeafQueue {
     // Last container goes to user_1's app_3, since it still wanted 1GB.
     // user_0's apps:
     assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
-        clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
+        clusterResource, NO_LABEL, false).getMemorySize());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(2*GB, app_1.getCurrentConsumption().getMemorySize());
     // user_1's apps:
@@ -4027,6 +4039,59 @@ public class TestLeafQueue {
         app.getResourceUsageReport().getClusterUsagePercentage(), 0.01f);
   }
 
+  @Test
+  public void testSetupQueueConfigsWithSpecifiedConfiguration()
+      throws IOException {
+
+    try {
+      CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(
+          new Configuration(false), false);
+
+      final String leafQueueName =
+          "testSetupQueueConfigsWithSpecifiedConfiguration";
+
+      assertEquals(0, conf.size());
+      conf.setNodeLocalityDelay(60);
+      conf.setCapacity(ROOT + DOT + leafQueueName, 10);
+      conf.setMaximumCapacity(ROOT + DOT + leafQueueName, 100);
+      conf.setUserLimitFactor(ROOT + DOT +leafQueueName, 0.1f);
+
+      csConf.setNodeLocalityDelay(30);
+      csConf.setGlobalMaximumApplicationsPerQueue(20);
+
+      LeafQueue leafQueue = new LeafQueue(csContext, conf,
+          leafQueueName, cs.getRootQueue(),
+          null);
+
+      assertEquals(30, leafQueue.getNodeLocalityDelay());
+      assertEquals(20, leafQueue.getMaxApplications());
+      assertEquals(2, leafQueue.getMaxApplicationsPerUser());
+
+      //check queue configs
+      conf.setMaximumAMResourcePercentPerPartition(leafQueue.getQueueName(),
+          NO_LABEL, 10);
+      conf.setMaximumCapacity(leafQueue.getQueueName(), 10);
+
+      assertEquals(0.1, leafQueue.getMaxAMResourcePerQueuePercent(),
+          EPSILON);
+      assertEquals(1, leafQueue.getMaximumCapacity(),
+          EPSILON);
+      assertEquals(0.1, leafQueue.getCapacity(),
+          EPSILON);
+      assertEquals(0.1, leafQueue.getAbsoluteCapacity(),
+          EPSILON);
+      assertEquals(1.0, leafQueue.getAbsoluteMaximumCapacity(),
+          EPSILON);
+
+     } finally {
+        //revert config changes
+        csConf.setNodeLocalityDelay(
+            CapacitySchedulerConfiguration.DEFAULT_NODE_LOCALITY_DELAY);
+        csConf.setGlobalMaximumApplicationsPerQueue(
+            (int) CapacitySchedulerConfiguration.UNDEFINED);
+     }
+  }
+
   private ApplicationAttemptId createAppAttemptId(int appId, int attemptId) {
     ApplicationId appIdImpl = ApplicationId.newInstance(0, appId);
     ApplicationAttemptId attId =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueManagementDynamicEditPolicy.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/TestQueueManagementDynamicEditPolicy.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/TestQueueManagementDynamicEditPolicy.java
new file mode 100644
index 0000000..4dc56fb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueManagementDynamicEditPolicy.java
@@ -0,0 +1,121 @@
+/**
+ * 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.scheduler.capacity;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
+import org.junit.Before;
+import org.junit.Test;
+
+
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.CSQueueUtils.EPSILON;
+import static org.junit.Assert.assertEquals;
+
+public class TestQueueManagementDynamicEditPolicy extends
+    TestCapacitySchedulerAutoCreatedQueueBase {
+   private QueueManagementDynamicEditPolicy policy = new
+       QueueManagementDynamicEditPolicy();
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    policy.init(cs.getConfiguration(), cs.getRMContext(), cs);
+  }
+
+  @Test
+  public void testEditSchedule() throws Exception {
+
+    try {
+      policy.editSchedule();
+      assertEquals(2, policy.getManagedParentQueues().size());
+
+      CSQueue parentQueue = cs.getQueue(PARENT_QUEUE);
+
+      GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy =
+          (GuaranteedOrZeroCapacityOverTimePolicy) ((ManagedParentQueue)
+              parentQueue)
+              .getAutoCreatedQueueManagementPolicy();
+      assertEquals(0f, autoCreatedQueueManagementPolicy
+          .getAbsoluteActivatedChildQueueCapacity(), EPSILON);
+
+      //submit app1 as USER1
+      ApplicationId user1AppId = submitApp(mockRM, parentQueue, USER1, USER1, 1,
+          1);
+      validateInitialQueueEntitlement(parentQueue, USER1, 0.1f);
+
+      //submit another app2 as USER2
+      ApplicationId user2AppId = submitApp(mockRM, parentQueue, USER2, USER2, 2,
+          1);
+      validateInitialQueueEntitlement(parentQueue, USER2, 0.2f);
+
+      //validate total activated abs capacity
+      assertEquals(0.2f, autoCreatedQueueManagementPolicy
+          .getAbsoluteActivatedChildQueueCapacity(), EPSILON);
+
+      //submit user_3 app. This cant be scheduled since there is no capacity
+      submitApp(mockRM, parentQueue, USER3, USER3, 3, 1);
+      final CSQueue user3LeafQueue = cs.getQueue(USER3);
+      validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.0f, 0.0f,
+          1.0f, 1.0f);
+
+      assertEquals(autoCreatedQueueManagementPolicy
+          .getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON);
+
+      //deactivate USER2 queue
+      cs.killAllAppsInQueue(USER2);
+      mockRM.waitForState(user2AppId, RMAppState.KILLED);
+
+      //deactivate USER1 queue
+      cs.killAllAppsInQueue(USER1);
+      mockRM.waitForState(user1AppId, RMAppState.KILLED);
+
+      policy.editSchedule();
+
+      waitForPolicyState(0.1f, autoCreatedQueueManagementPolicy, 1000);
+
+      validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.5f, 0.1f,
+          1.0f, 1.0f);
+
+      validateCapacitiesByLabel((ManagedParentQueue) parentQueue, (AutoCreatedLeafQueue) user3LeafQueue,
+          NODEL_LABEL_GPU);
+
+    } finally {
+      cleanupQueue(USER1);
+      cleanupQueue(USER2);
+      cleanupQueue(USER3);
+    }
+  }
+
+  private void waitForPolicyState(float expectedVal,
+      GuaranteedOrZeroCapacityOverTimePolicy queueManagementPolicy, int
+      timesec) throws
+      InterruptedException {
+    long start = System.currentTimeMillis();
+    while (System.currentTimeMillis() - start < timesec * 1000) {
+      if (Float.compare(expectedVal, queueManagementPolicy
+          .getAbsoluteActivatedChildQueueCapacity()) != 0) {
+        Thread.sleep(100);
+      } else {
+        break;
+      }
+    }
+  }
+}


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


[19/50] [abbrv] hadoop git commit: YARN-7533. Documentation for absolute resource support in Capacity Scheduler. Contributed by Sunil G.

Posted by wa...@apache.org.
YARN-7533. Documentation for absolute resource support in Capacity Scheduler. Contributed by Sunil G.


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

Branch: refs/heads/YARN-6592
Commit: 532bbf4602e418276961cfc4f6b5e3f70e9cedc0
Parents: 4db4a4a
Author: Sunil G <su...@apache.org>
Authored: Wed Nov 29 22:57:54 2017 +0530
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../src/site/markdown/CapacityScheduler.md                | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/532bbf46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
index 0449e88..86d9167 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
@@ -64,6 +64,8 @@ The `CapacityScheduler` supports the following features:
 
 * **Priority Scheduling** - This feature allows applications to be submitted and scheduled with different priorities. Higher integer value indicates higher priority for an application. Currently Application priority is supported only for FIFO ordering policy.
 
+* **Absolute Resource Configuration** - Administrators could specify absolute resources to a queue instead of providing percentage based values. This provides better control for admins to configure required amount of resources for a given queue.
+
 Configuration
 -------------
 
@@ -118,14 +120,18 @@ Configuration
 
 | Property | Description |
 |:---- |:---- |
-| `yarn.scheduler.capacity.<queue-path>.capacity` | Queue *capacity* in percentage (%) as a float (e.g. 12.5). The sum of capacities for all queues, at each level, must be equal to 100. Applications in the queue may consume more resources than the queue's capacity if there are free resources, providing elasticity. |
-| `yarn.scheduler.capacity.<queue-path>.maximum-capacity` | Maximum queue capacity in percentage (%) as a float. This limits the *elasticity* for applications in the queue. Defaults to -1 which disables it. |
+| `yarn.scheduler.capacity.<queue-path>.capacity` | Queue *capacity* in percentage (%) as a float (e.g. 12.5) OR as absolute resource queue minimum capacity. The sum of capacities for all queues, at each level, must be equal to 100. However if absolute resource is configured, sum of absolute resources of child queues could be less than it's parent absolute resource capacity. Applications in the queue may consume more resources than the queue's capacity if there are free resources, providing elasticity. |
+| `yarn.scheduler.capacity.<queue-path>.maximum-capacity` | Maximum queue capacity in percentage (%) as a float OR as absolute resource queue maximum capacity. This limits the *elasticity* for applications in the queue. Defaults to -1 which disables it. |
 | `yarn.scheduler.capacity.<queue-path>.minimum-user-limit-percent` | Each queue enforces a limit on the percentage of resources allocated to a user at any given time, if there is demand for resources. The user limit can vary between a minimum and maximum value. The former (the minimum value) is set to this property value and the latter (the maximum value) depends on the number of users who have submitted applications. For e.g., suppose the value of this property is 25. If two users have submitted applications to a queue, no single user can use more than 50% of the queue resources. If a third user submits an application, no single user can use more than 33% of the queue resources. With 4 or more users, no user can use more than 25% of the queues resources. A value of 100 implies no user limits are imposed. The default is 100. Value is specified as a integer. |
 | `yarn.scheduler.capacity.<queue-path>.user-limit-factor` | The multiple of the queue capacity which can be configured to allow a single user to acquire more resources. By default this is set to 1 which ensures that a single user can never take more than the queue's configured capacity irrespective of how idle the cluster is. Value is specified as a float. |
 | `yarn.scheduler.capacity.<queue-path>.maximum-allocation-mb` | The per queue maximum limit of memory to allocate to each container request at the Resource Manager. This setting overrides the cluster configuration `yarn.scheduler.maximum-allocation-mb`. This value must be smaller than or equal to the cluster maximum. |
 | `yarn.scheduler.capacity.<queue-path>.maximum-allocation-vcores` | The per queue maximum limit of virtual cores to allocate to each container request at the Resource Manager. This setting overrides the cluster configuration `yarn.scheduler.maximum-allocation-vcores`. This value must be smaller than or equal to the cluster maximum. |
 | `yarn.scheduler.capacity.<queue-path>.user-settings.<user-name>.weight` | This floating point value is used when calculating the user limit resource values for users in a queue. This value will weight each user more or less than the other users in the queue. For example, if user A should receive 50% more resources in a queue than users B and C, this property will be set to 1.5 for user A.  Users B and C will default to 1.0. |
 
+  * Resource Allocation using Absolute Resources configuration
+
+ `CapacityScheduler` supports configuration of absolute resources instead of providing Queue *capacity* in percentage. As mentioned in above configuration section for `yarn.scheduler.capacity.<queue-path>.capacity` and `yarn.scheduler.capacity.<queue-path>.max-capacity`, administrator could specify an absolute resource value like `[memory=10240,vcores=12]`. This is a valid configuration which indicates 10GB Memory and 12 VCores.
+
   * Running and Pending Application Limits
   
   The `CapacityScheduler` supports the following parameters to control the running and pending applications:


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


[32/50] [abbrv] hadoop git commit: HADOOP-15056. Fix TestUnbuffer#testUnbufferException failure. Contributed by Jack Bearden.

Posted by wa...@apache.org.
HADOOP-15056. Fix TestUnbuffer#testUnbufferException failure. Contributed by Jack Bearden.


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

Branch: refs/heads/YARN-6592
Commit: 19e089420999dd9d97d981dcd0abd64b6166152d
Parents: 532bbf4
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Dec 7 21:04:05 2017 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Dec 7 21:05:55 2017 -0800

----------------------------------------------------------------------
 .../hadoop/fs/StreamCapabilitiesPolicy.java     | 15 ++++++++++++---
 .../java/org/apache/hadoop/fs/TestUnbuffer.java | 20 +++++++++++++++-----
 2 files changed, 27 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e08942/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilitiesPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilitiesPolicy.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilitiesPolicy.java
index 3080780..8b63d4b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilitiesPolicy.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilitiesPolicy.java
@@ -22,6 +22,8 @@ import java.io.InputStream;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Static methods to implement policies for {@link StreamCapabilities}.
@@ -29,6 +31,10 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class StreamCapabilitiesPolicy {
+  public static final String CAN_UNBUFFER_NOT_IMPLEMENTED_MESSAGE =
+          "claims unbuffer capabilty but does not implement CanUnbuffer";
+  static final Logger LOG = LoggerFactory.getLogger(
+          StreamCapabilitiesPolicy.class);
   /**
    * Implement the policy for {@link CanUnbuffer#unbuffer()}.
    *
@@ -40,11 +46,14 @@ public class StreamCapabilitiesPolicy {
           && ((StreamCapabilities) in).hasCapability(
           StreamCapabilities.UNBUFFER)) {
         ((CanUnbuffer) in).unbuffer();
+      } else {
+        LOG.debug(in.getClass().getName() + ":"
+                + " does not implement StreamCapabilities"
+                + " and the unbuffer capability");
       }
     } catch (ClassCastException e) {
-      throw new UnsupportedOperationException("this stream " +
-          in.getClass().getName() +
-          " claims to unbuffer but forgets to implement CanUnbuffer");
+      throw new UnsupportedOperationException(in.getClass().getName() + ": "
+              + CAN_UNBUFFER_NOT_IMPLEMENTED_MESSAGE);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19e08942/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
index b112e30..655d453 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
@@ -131,17 +131,27 @@ public class TestUnbuffer {
   }
 
   /**
-   * Test unbuffer method which throws an Exception with class name included.
+   * Test that a InputStream should throw an exception when not implementing
+   * CanUnbuffer
+   *
+   * This should throw an exception when the stream claims to have the
+   * unbuffer capability, but actually does not implement CanUnbuffer.
    */
   @Test
   public void testUnbufferException() {
-    FSInputStream in = Mockito.mock(FSInputStream.class);
-    FSDataInputStream fs = new FSDataInputStream(in);
+    abstract class BuggyStream
+            extends FSInputStream
+            implements StreamCapabilities {
+    }
+
+    BuggyStream bs = Mockito.mock(BuggyStream.class);
+    Mockito.when(bs.hasCapability(Mockito.anyString())).thenReturn(true);
 
     exception.expect(UnsupportedOperationException.class);
-    exception.expectMessage("this stream " + in.getClass().getName()
-        + " does not support unbuffering");
+    exception.expectMessage(
+            StreamCapabilitiesPolicy.CAN_UNBUFFER_NOT_IMPLEMENTED_MESSAGE);
 
+    FSDataInputStream fs = new FSDataInputStream(bs);
     fs.unbuffer();
   }
 }


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


[05/50] [abbrv] hadoop git commit: HADOOP-15039. Move SemaphoredDelegatingExecutor to hadoop-common. Contributed by Genmao Yu

Posted by wa...@apache.org.
HADOOP-15039. Move SemaphoredDelegatingExecutor to hadoop-common. Contributed by Genmao Yu


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

Branch: refs/heads/YARN-6592
Commit: 73b86979d661f4ad56fcfc3a05a403dfcb2a860e
Parents: 44b06d3
Author: Kai Zheng <zh...@alibaba-inc.com>
Authored: Wed Dec 6 12:01:36 2017 +0800
Committer: Kai Zheng <zh...@alibaba-inc.com>
Committed: Wed Dec 6 12:01:36 2017 +0800

----------------------------------------------------------------------
 .../fs/s3a/BlockingThreadPoolExecutorService.java    |  7 +++----
 .../java/org/apache/hadoop/fs/s3a/S3AFileSystem.java |  2 ++
 .../hadoop/fs/s3a/SemaphoredDelegatingExecutor.java  | 15 ++++++---------
 .../s3a/ITestBlockingThreadPoolExecutorService.java  |  2 ++
 4 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/73b86979/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
index f13942d..404eea9 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
@@ -15,8 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.hadoop.fs.s3a;
+package org.apache.hadoop.util;
 
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -42,7 +41,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
  * this s4 threadpool</a>
  */
 @InterfaceAudience.Private
-final class BlockingThreadPoolExecutorService
+public final class BlockingThreadPoolExecutorService
     extends SemaphoredDelegatingExecutor {
 
   private static final Logger LOG = LoggerFactory
@@ -86,7 +85,7 @@ final class BlockingThreadPoolExecutorService
    * @return a thread factory that creates named, daemon threads with
    * the supplied exception handler and normal priority
    */
-  static ThreadFactory newDaemonThreadFactory(final String prefix) {
+  public static ThreadFactory newDaemonThreadFactory(final String prefix) {
     final ThreadFactory namedFactory = getNamedThreadFactory(prefix);
     return new ThreadFactory() {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73b86979/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 63a4349..e927758 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -112,8 +112,10 @@ import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Invoker.*;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73b86979/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
index 6b21912..bcc19e3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a;
+package org.apache.hadoop.util;
 
 import com.google.common.util.concurrent.ForwardingListeningExecutorService;
 import com.google.common.util.concurrent.Futures;
@@ -42,17 +42,13 @@ import java.util.concurrent.TimeoutException;
  * This is a refactoring of {@link BlockingThreadPoolExecutorService}; that code
  * contains the thread pool logic, whereas this isolates the semaphore
  * and submit logic for use with other thread pools and delegation models.
- * In particular, it <i>permits multiple per stream executors to share a
- * single per-FS-instance executor; the latter to throttle overall
- * load from the the FS, the others to limit the amount of load which
- * a single output stream can generate.</i>
  * <p>
  * This is inspired by <a href="https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java">
  * this s4 threadpool</a>
  */
 @SuppressWarnings("NullableProblems")
 @InterfaceAudience.Private
-class SemaphoredDelegatingExecutor extends
+public class SemaphoredDelegatingExecutor extends
     ForwardingListeningExecutorService {
 
   private final Semaphore queueingPermits;
@@ -65,7 +61,8 @@ class SemaphoredDelegatingExecutor extends
    * @param permitCount number of permits into the queue permitted
    * @param fair should the semaphore be "fair"
    */
-  SemaphoredDelegatingExecutor(ListeningExecutorService executorDelegatee,
+  public SemaphoredDelegatingExecutor(
+      ListeningExecutorService executorDelegatee,
       int permitCount,
       boolean fair) {
     this.permitCount = permitCount;
@@ -190,7 +187,7 @@ class SemaphoredDelegatingExecutor extends
 
     private Runnable delegatee;
 
-    public RunnableWithPermitRelease(Runnable delegatee) {
+    RunnableWithPermitRelease(Runnable delegatee) {
       this.delegatee = delegatee;
     }
 
@@ -212,7 +209,7 @@ class SemaphoredDelegatingExecutor extends
 
     private Callable<T> delegatee;
 
-    public CallableWithPermitRelease(Callable<T> delegatee) {
+    CallableWithPermitRelease(Callable<T> delegatee) {
       this.delegatee = delegatee;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73b86979/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
index b1b8240..3dfe286 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.fs.s3a;
 
 import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
+import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 import org.apache.hadoop.util.StopWatch;
 
 import org.junit.AfterClass;


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


[49/50] [abbrv] hadoop git commit: YARN-6594. [API] Introduce SchedulingRequest object. (Konstantinos Karanasos via wangda)

Posted by wa...@apache.org.
YARN-6594. [API] Introduce SchedulingRequest object. (Konstantinos Karanasos via wangda)


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

Branch: refs/heads/YARN-6592
Commit: ef1e90d993c569695a58f659e345b9d673841257
Parents: ccd57dc
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Oct 30 16:54:02 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Sun Dec 10 10:27:01 2017 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/ResourceSizing.java |  64 +++++
 .../yarn/api/records/SchedulingRequest.java     | 205 ++++++++++++++
 .../src/main/proto/yarn_protos.proto            |  14 +
 .../records/impl/pb/ResourceSizingPBImpl.java   | 117 ++++++++
 .../impl/pb/SchedulingRequestPBImpl.java        | 266 +++++++++++++++++++
 5 files changed, 666 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef1e90d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
new file mode 100644
index 0000000..d82be11
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@code ResourceSizing} contains information for the size of a
+ * {@link SchedulingRequest}, such as the number of requested allocations and
+ * the resources for each allocation.
+ */
+@Public
+@Unstable
+public abstract class ResourceSizing {
+
+  @Public
+  @Unstable
+  public static ResourceSizing newInstance(Resource resources) {
+    return ResourceSizing.newInstance(1, resources);
+  }
+
+  @Public
+  @Unstable
+  public static ResourceSizing newInstance(int numAllocations, Resource resources) {
+    ResourceSizing resourceSizing = Records.newRecord(ResourceSizing.class);
+    resourceSizing.setNumAllocations(numAllocations);
+    resourceSizing.setResources(resources);
+    return resourceSizing;
+  }
+
+  @Public
+  @Unstable
+  public abstract int getNumAllocations();
+
+  @Public
+  @Unstable
+  public abstract void setNumAllocations(int numAllocations);
+
+  @Public
+  @Unstable
+  public abstract Resource getResources();
+
+  @Public
+  @Unstable
+  public abstract void setResources(Resource resources);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef1e90d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
new file mode 100644
index 0000000..47a0697
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
@@ -0,0 +1,205 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records;
+
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@code SchedulingRequest} represents a request made by an application to the
+ * {@code ResourceManager} to obtain an allocation. It is similar to the
+ * {@link ResourceRequest}. However, it is more complete than the latter, as it
+ * allows applications to specify allocation tags (e.g., to express that an
+ * allocation belongs to {@code Spark} or is an {@code HBase-master}), as well
+ * as involved {@link PlacementConstraint}s (e.g., anti-affinity between Spark
+ * and HBase allocations).
+ *
+ * The size specification of the allocation is in {@code ResourceSizing}.
+ */
+@Public
+@Unstable
+public abstract class SchedulingRequest {
+
+  @Public
+  @Unstable
+  public static SchedulingRequest newInstance(long allocationRequestId,
+      Priority priority, ExecutionTypeRequest executionType,
+      Set<String> allocationTags, ResourceSizing resourceSizing,
+      PlacementConstraint placementConstraintExpression) {
+    return SchedulingRequest.newBuilder()
+        .allocationRequestId(allocationRequestId).priority(priority)
+        .executionType(executionType).allocationTags(allocationTags)
+        .placementConstraintExpression(placementConstraintExpression).build();
+  }
+
+  @Public
+  @Unstable
+  public static SchedulingRequestBuilder newBuilder() {
+    return new SchedulingRequestBuilder();
+  }
+
+  /**
+   * Class to construct instances of {@link SchedulingRequest} with specific
+   * options.
+   */
+  @Public
+  @Unstable
+  public static final class SchedulingRequestBuilder {
+    private SchedulingRequest schedulingRequest =
+            Records.newRecord(SchedulingRequest.class);
+
+    private SchedulingRequestBuilder() {
+      schedulingRequest.setAllocationRequestId(0);
+      schedulingRequest.setPriority(Priority.newInstance(0));
+      schedulingRequest.setExecutionType(ExecutionTypeRequest.newInstance());
+    }
+
+    /**
+     * Set the <code>allocationRequestId</code> of the request.
+     * 
+     * @see SchedulingRequest#setAllocationRequestId(long)
+     * @param allocationRequestId <code>allocationRequestId</code> of the
+     *          request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder allocationRequestId(
+            long allocationRequestId) {
+      schedulingRequest.setAllocationRequestId(allocationRequestId);
+      return this;
+    }
+
+    /**
+     * Set the <code>priority</code> of the request.
+     *
+     * @param priority <code>priority</code> of the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     * @see SchedulingRequest#setPriority(Priority)
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder priority(Priority priority) {
+      schedulingRequest.setPriority(priority);
+      return this;
+    }
+
+    /**
+     * Set the <code>executionType</code> of the request.
+     * 
+     * @see SchedulingRequest#setExecutionType(ExecutionTypeRequest)
+     * @param executionType <code>executionType</code> of the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder executionType(
+        ExecutionTypeRequest executionType) {
+      schedulingRequest.setExecutionType(executionType);
+      return this;
+    }
+    
+    /**
+     * Set the <code>allocationTags</code> of the request.
+     *
+     * @see SchedulingRequest#setAllocationTags(Set)
+     * @param allocationTags <code>allocationsTags</code> of the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder allocationTags(Set<String> allocationTags) {
+      schedulingRequest.setAllocationTags(allocationTags);
+      return this;
+    }
+
+    /**
+     * Set the <code>executionType</code> of the request.
+     *
+     * @see SchedulingRequest#setResourceSizing(ResourceSizing)
+     * @param resourceSizing <code>resourceSizing</code> of the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder resourceSizing(
+        ResourceSizing resourceSizing) {
+      schedulingRequest.setResourceSizing(resourceSizing);
+      return this;
+    }
+
+    /**
+     * Set the <code>placementConstraintExpression</code> of the request.
+     *
+     * @see SchedulingRequest#setPlacementConstraint(
+     *      PlacementConstraint)
+     * @param placementConstraintExpression <code>placementConstraints</code> of
+     *          the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder placementConstraintExpression(
+        PlacementConstraint placementConstraintExpression) {
+      schedulingRequest
+          .setPlacementConstraint(placementConstraintExpression);
+      return this;
+    }
+
+    /**
+     * Return generated {@link SchedulingRequest} object.
+     * 
+     * @return {@link SchedulingRequest}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequest build() {
+      return schedulingRequest;
+    }
+  }
+
+  public abstract long getAllocationRequestId();
+
+  public abstract void setAllocationRequestId(long allocationRequestId);
+
+  public abstract Priority getPriority();
+
+  public abstract void setPriority(Priority priority);
+
+  public abstract ExecutionTypeRequest getExecutionType();
+
+  public abstract void setExecutionType(ExecutionTypeRequest executionType);
+
+  public abstract Set<String> getAllocationTags();
+
+  public abstract void setAllocationTags(Set<String> allocationTags);
+
+  public abstract ResourceSizing getResourceSizing();
+
+  public abstract void setResourceSizing(ResourceSizing resourceSizing);
+
+  public abstract PlacementConstraint getPlacementConstraint();
+
+  public abstract void setPlacementConstraint(
+      PlacementConstraint placementConstraint);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef1e90d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 968b75e..2dbdefb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -405,6 +405,20 @@ message ExecutionTypeRequestProto {
   optional bool enforce_execution_type = 2 [default = false];
 }
 
+message SchedulingRequestProto {
+  optional int64 allocationRequestId = 1 [default = 0];
+  optional PriorityProto priority = 2;
+  optional ExecutionTypeRequestProto executionType = 3;
+  repeated string allocationTags = 4;
+  optional ResourceSizingProto resourceSizing = 5;
+  optional PlacementConstraintProto placementConstraint = 6;
+}
+
+message ResourceSizingProto {
+  optional int32 numAllocations = 1;
+  optional ResourceProto resources = 2;
+}
+
 enum AMCommandProto {
   AM_RESYNC = 1;
   AM_SHUTDOWN = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef1e90d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
new file mode 100644
index 0000000..05bb3bd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceSizingProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceSizingProtoOrBuilder;
+
+@Private
+@Unstable
+public class ResourceSizingPBImpl extends ResourceSizing {
+  ResourceSizingProto proto = ResourceSizingProto.getDefaultInstance();
+  ResourceSizingProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private Resource resources = null;
+
+  public ResourceSizingPBImpl() {
+    builder = ResourceSizingProto.newBuilder();
+  }
+
+  public ResourceSizingPBImpl(ResourceSizingProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ResourceSizingProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.resources != null) {
+      builder.setResources(convertToProtoFormat(this.resources));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ResourceSizingProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public int getNumAllocations() {
+    ResourceSizingProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getNumAllocations());
+  }
+
+  @Override
+  public void setNumAllocations(int numAllocations) {
+    maybeInitBuilder();
+    builder.setNumAllocations(numAllocations);
+  }
+
+  @Override
+  public Resource getResources() {
+    ResourceSizingProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.resources != null) {
+      return this.resources;
+    }
+    if (!p.hasResources()) {
+      return null;
+    }
+    this.resources = convertFromProtoFormat(p.getResources());
+    return this.resources;
+  }
+
+  @Override
+  public void setResources(Resource resources) {
+    maybeInitBuilder();
+    if (resources == null) {
+      builder.clearResources();
+    }
+    this.resources = resources;
+  }
+
+  private ResourcePBImpl convertFromProtoFormat(ResourceProto r) {
+    return new ResourcePBImpl(r);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource r) {
+    return ((ResourcePBImpl) r).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef1e90d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
new file mode 100644
index 0000000..7826b36
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
@@ -0,0 +1,266 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records.impl.pb;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintFromProtoConverter;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintToProtoConverter;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceSizingProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SchedulingRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SchedulingRequestProtoOrBuilder;
+
+@Private
+@Unstable
+public class SchedulingRequestPBImpl extends SchedulingRequest {
+  SchedulingRequestProto proto = SchedulingRequestProto.getDefaultInstance();
+  SchedulingRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private Priority priority = null;
+  private ExecutionTypeRequest executionType = null;
+  private Set<String> allocationTags = null;
+  private ResourceSizing resourceSizing = null;
+  private PlacementConstraint placementConstraint = null;
+
+  public SchedulingRequestPBImpl() {
+    builder = SchedulingRequestProto.newBuilder();
+  }
+
+  public SchedulingRequestPBImpl(SchedulingRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SchedulingRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.priority != null) {
+      builder.setPriority(convertToProtoFormat(this.priority));
+    }
+    if (this.executionType != null) {
+      builder.setExecutionType(convertToProtoFormat(this.executionType));
+    }
+    if (this.allocationTags != null) {
+      builder.clearAllocationTags();
+      builder.addAllAllocationTags(this.allocationTags);
+    }
+    if (this.resourceSizing != null) {
+      builder.setResourceSizing(convertToProtoFormat(this.resourceSizing));
+    }
+    if (this.placementConstraint != null) {
+      builder.setPlacementConstraint(
+          convertToProtoFormat(this.placementConstraint));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SchedulingRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public long getAllocationRequestId() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getAllocationRequestId());
+  }
+
+  @Override
+  public void setAllocationRequestId(long allocationRequestId) {
+    maybeInitBuilder();
+    builder.setAllocationRequestId(allocationRequestId);
+  }
+
+  @Override
+  public Priority getPriority() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.priority != null) {
+      return this.priority;
+    }
+    if (!p.hasPriority()) {
+      return null;
+    }
+    this.priority = convertFromProtoFormat(p.getPriority());
+    return this.priority;
+  }
+
+  @Override
+  public void setPriority(Priority priority) {
+    maybeInitBuilder();
+    if (priority == null) {
+      builder.clearPriority();
+    }
+    this.priority = priority;
+  }
+
+  @Override
+  public ExecutionTypeRequest getExecutionType() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.executionType != null) {
+      return this.executionType;
+    }
+    if (!p.hasExecutionType()) {
+      return null;
+    }
+    this.executionType = convertFromProtoFormat(p.getExecutionType());
+    return this.executionType;
+  }
+
+  @Override
+  public void setExecutionType(ExecutionTypeRequest executionType) {
+    maybeInitBuilder();
+    if (executionType == null) {
+      builder.clearExecutionType();
+    }
+    this.executionType = executionType;
+  }
+
+  @Override
+  public Set<String> getAllocationTags() {
+    initAllocationTags();
+    return this.allocationTags;
+  }
+
+  @Override
+  public void setAllocationTags(Set<String> allocationTags) {
+    maybeInitBuilder();
+    builder.clearAllocationTags();
+    this.allocationTags = allocationTags;
+  }
+
+  @Override
+  public ResourceSizing getResourceSizing() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.resourceSizing != null) {
+      return this.resourceSizing;
+    }
+    if (!p.hasResourceSizing()) {
+      return null;
+    }
+    this.resourceSizing = convertFromProtoFormat(p.getResourceSizing());
+    return this.resourceSizing;
+  }
+
+  @Override
+  public void setResourceSizing(ResourceSizing resourceSizing) {
+    maybeInitBuilder();
+    if (resourceSizing == null) {
+      builder.clearResourceSizing();
+    }
+    this.resourceSizing = resourceSizing;
+  }
+
+  @Override
+  public PlacementConstraint getPlacementConstraint() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.placementConstraint != null) {
+      return this.placementConstraint;
+    }
+    if (!p.hasPlacementConstraint()) {
+      return null;
+    }
+    this.placementConstraint =
+        convertFromProtoFormat(p.getPlacementConstraint());
+    return this.placementConstraint;
+  }
+
+  @Override
+  public void setPlacementConstraint(PlacementConstraint placementConstraint) {
+    maybeInitBuilder();
+    if (placementConstraint == null) {
+      builder.clearPlacementConstraint();
+    }
+    this.placementConstraint = placementConstraint;
+  }
+
+  private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
+    return new PriorityPBImpl(p);
+  }
+
+  private PriorityProto convertToProtoFormat(Priority p) {
+    return ((PriorityPBImpl) p).getProto();
+  }
+
+  private ExecutionTypeRequestPBImpl convertFromProtoFormat(
+      ExecutionTypeRequestProto p) {
+    return new ExecutionTypeRequestPBImpl(p);
+  }
+
+  private ExecutionTypeRequestProto convertToProtoFormat(
+      ExecutionTypeRequest p) {
+    return ((ExecutionTypeRequestPBImpl) p).getProto();
+  }
+
+  private ResourceSizingPBImpl convertFromProtoFormat(ResourceSizingProto p) {
+    return new ResourceSizingPBImpl(p);
+  }
+
+  private ResourceSizingProto convertToProtoFormat(ResourceSizing p) {
+    return ((ResourceSizingPBImpl) p).getProto();
+  }
+
+  private PlacementConstraint convertFromProtoFormat(
+      PlacementConstraintProto c) {
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(c);
+    return fromProtoConverter.convert();
+  }
+
+  private PlacementConstraintProto convertToProtoFormat(PlacementConstraint c) {
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(c);
+    return toProtoConverter.convert();
+  }
+
+  private void initAllocationTags() {
+    if (this.allocationTags != null) {
+      return;
+    }
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    this.allocationTags = new HashSet<>();
+    this.allocationTags.addAll(p.getAllocationTagsList());
+  }
+}


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


[09/50] [abbrv] hadoop git commit: HDFS-12889. Addendum patch to add missing file.

Posted by wa...@apache.org.
HDFS-12889. Addendum patch to add missing file.


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

Branch: refs/heads/YARN-6592
Commit: 6cca5b3bcb440095f12d3eda88101fa250ac000a
Parents: 40b0045e
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 7 13:22:48 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 7 13:23:37 2017 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/robots.txt | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cca5b3b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/robots.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/robots.txt b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/robots.txt
new file mode 100644
index 0000000..77470cb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/robots.txt
@@ -0,0 +1,2 @@
+User-agent: *
+Disallow: /
\ No newline at end of file


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


[27/50] [abbrv] hadoop git commit: YARN-7483. CapacityScheduler test cases cleanup post YARN-5881. (Sunil G via wangda)

Posted by wa...@apache.org.
YARN-7483. CapacityScheduler test cases cleanup post YARN-5881. (Sunil G via wangda)

Change-Id: I9741a6baf5cb7352d05636efb6c0b24790e7589a


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

Branch: refs/heads/YARN-6592
Commit: 7462c38277fa415703fc9074c1288d3bec73609c
Parents: e65ca92
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Nov 16 11:35:48 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../api/records/impl/pb/ResourcePBImpl.java     |  11 ++
 .../scheduler/capacity/TestLeafQueue.java       | 130 +++++++++++++++++--
 2 files changed, 131 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7462c382/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
index 4f90133..6ebed6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
@@ -151,6 +151,17 @@ public class ResourcePBImpl extends Resource {
         .newInstance(ResourceInformation.VCORES);
     this.setMemorySize(p.getMemory());
     this.setVirtualCores(p.getVirtualCores());
+
+    // Update missing resource information on respective index.
+    updateResourceInformationMap(types);
+  }
+
+  private void updateResourceInformationMap(ResourceInformation[] types) {
+    for (int i = 0; i < types.length; i++) {
+      if (resources[i] == null) {
+        resources[i] = ResourceInformation.newInstance(types[i]);
+      }
+    }
   }
 
   private static ResourceInformation newDefaultInformation(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7462c382/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 30bff78..1426e88 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
@@ -1027,6 +1027,8 @@ public class TestLeafQueue {
     Resource clusterResource =
         Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     // app_0 asks for 3 3-GB containers
@@ -1083,9 +1085,15 @@ public class TestLeafQueue {
         a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource),
         SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps);
-    assertEquals(12*GB, a.getUsedResources().getMemorySize());
-    assertEquals(12*GB, app_0.getCurrentConsumption().getMemorySize());
-    assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
+    assertEquals(9*GB, a.getUsedResources().getMemorySize());
+    assertEquals(8*GB, app_0.getCurrentConsumption().getMemorySize());
+    assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
+
+    assertEquals(4 * GB,
+        app_0.getTotalPendingRequestsPerPartition().get("").getMemorySize());
+
+    assertEquals(1 * GB,
+        app_1.getTotalPendingRequestsPerPartition().get("").getMemorySize());
   }
 
   @SuppressWarnings({ "unchecked", "rawtypes" })
@@ -1317,11 +1325,6 @@ public class TestLeafQueue {
     Resource clusterResource = Resources.createResource(numNodes * (8*GB), 1);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
-    ParentQueue root = (ParentQueue) queues
-        .get(CapacitySchedulerConfiguration.ROOT);
-    root.updateClusterResource(clusterResource,
-        new ResourceLimits(clusterResource));
-
     // Setup resource-requests
     Priority priority = TestUtils.createMockPriority(1);
     app_0.updateResourceRequests(Collections.singletonList(
@@ -1340,6 +1343,11 @@ public class TestLeafQueue {
     a.setUserLimit(50);
     a.setUserLimitFactor(2);
 
+    ParentQueue root = (ParentQueue) queues
+        .get(CapacitySchedulerConfiguration.ROOT);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
+
     // Now, only user_0 should be active since he is the only one with
     // outstanding requests
     assertEquals("There should only be 1 active user!", 
@@ -1368,8 +1376,8 @@ public class TestLeafQueue {
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, app_0.getHeadroom().getMemorySize());
-    assertEquals(0*GB, app_1.getHeadroom().getMemorySize()); // 4G - 3G
-    
+    assertEquals(0*GB, app_1.getHeadroom().getMemorySize());
+
     // Submit requests for app_1 and set max-cap
     a.setMaxCapacity(.1f);
     root.updateClusterResource(clusterResource,
@@ -1405,6 +1413,108 @@ public class TestLeafQueue {
   }
 
   @Test
+  public void testUserHeadroomMultiApp() throws Exception {
+    // Mock the queue
+    LeafQueue a = stubLeafQueue((LeafQueue) queues.get(A));
+    // unset maxCapacity
+    a.setMaxCapacity(1.0f);
+
+    // Users
+    final String user_0 = "user_0";
+    final String user_1 = "user_1";
+
+    // Submit applications
+    final ApplicationAttemptId appAttemptId_0 = TestUtils
+        .getMockApplicationAttemptId(0, 0);
+    FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a,
+        a.getAbstractUsersManager(), spyRMContext);
+    a.submitApplicationAttempt(app_0, user_0);
+
+    final ApplicationAttemptId appAttemptId_1 = TestUtils
+        .getMockApplicationAttemptId(1, 0);
+    FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a,
+        a.getAbstractUsersManager(), spyRMContext);
+    a.submitApplicationAttempt(app_1, user_0); // same user
+
+    final ApplicationAttemptId appAttemptId_2 = TestUtils
+        .getMockApplicationAttemptId(2, 0);
+    FiCaSchedulerApp app_2 = new FiCaSchedulerApp(appAttemptId_2, user_1, a,
+        a.getAbstractUsersManager(), spyRMContext);
+    a.submitApplicationAttempt(app_2, user_1);
+
+    // Setup some nodes
+    String host_0 = "127.0.0.1";
+    FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0,
+        16 * GB);
+    String host_1 = "127.0.0.2";
+    FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0,
+        16 * GB);
+
+    Map<ApplicationAttemptId, FiCaSchedulerApp> apps = ImmutableMap.of(
+        app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(),
+        app_1, app_2.getApplicationAttemptId(), app_2);
+    Map<NodeId, FiCaSchedulerNode> nodes = ImmutableMap.of(node_0.getNodeID(),
+        node_0, node_1.getNodeID(), node_1);
+
+    final int numNodes = 2;
+    Resource clusterResource = Resources.createResource(numNodes * (16 * GB),
+        1);
+    when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
+
+    Priority priority = TestUtils.createMockPriority(1);
+
+    app_0.updateResourceRequests(
+        Collections.singletonList(TestUtils.createResourceRequest(
+            ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory)));
+
+    applyCSAssignment(clusterResource,
+        a.assignContainers(clusterResource, node_0,
+            new ResourceLimits(clusterResource),
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY),
+        a, nodes, apps);
+    assertEquals(1 * GB, a.getUsedResources().getMemorySize());
+    assertEquals(1 * GB, app_0.getCurrentConsumption().getMemorySize());
+    assertEquals(0 * GB, app_1.getCurrentConsumption().getMemorySize());
+    // Now, headroom is the same for all apps for a given user + queue combo
+    // and a change to any app's headroom is reflected for all the user's apps
+    // once those apps are active/have themselves calculated headroom for
+    // allocation at least one time
+    assertEquals(2 * GB, app_0.getHeadroom().getMemorySize());
+    assertEquals(2 * GB, app_1.getHeadroom().getMemorySize());// not yet active
+    assertEquals(3 * GB, app_2.getHeadroom().getMemorySize());// not yet active
+
+    app_1.updateResourceRequests(
+        Collections.singletonList(TestUtils.createResourceRequest(
+            ResourceRequest.ANY, 1 * GB, 2, true, priority, recordFactory)));
+
+    applyCSAssignment(clusterResource,
+        a.assignContainers(clusterResource, node_0,
+            new ResourceLimits(clusterResource),
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY),
+        a, nodes, apps);
+    assertEquals(2 * GB, a.getUsedResources().getMemorySize());
+    assertEquals(1 * GB, app_0.getCurrentConsumption().getMemorySize());
+    assertEquals(1 * GB, app_1.getCurrentConsumption().getMemorySize());
+    assertEquals(1 * GB, app_0.getHeadroom().getMemorySize());
+    assertEquals(1 * GB, app_1.getHeadroom().getMemorySize());// now active
+    assertEquals(3 * GB, app_2.getHeadroom().getMemorySize());// not yet active
+
+    // Complete container and verify that headroom is updated, for both apps
+    // for the user
+    RMContainer rmContainer = app_0.getLiveContainers().iterator().next();
+    a.completedContainer(clusterResource, app_0, node_0, rmContainer,
+        ContainerStatus.newInstance(rmContainer.getContainerId(),
+            ContainerState.COMPLETE, "",
+            ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
+        RMContainerEventType.KILL, null, true);
+
+    assertEquals(2 * GB, app_0.getHeadroom().getMemorySize());
+    assertEquals(2 * GB, app_1.getHeadroom().getMemorySize());
+  }
+
+  @Test
   public void testSingleQueueWithMultipleUsers() throws Exception {
     
     // Mock the queue


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


[04/50] [abbrv] hadoop git commit: HDFS-12889. Router UI is missing robots.txt file. Contributed by Bharat Viswanadham.

Posted by wa...@apache.org.
HDFS-12889. Router UI is missing robots.txt file. Contributed by Bharat Viswanadham.


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

Branch: refs/heads/YARN-6592
Commit: 44b06d34a537f8b558007cc92a5d1a8e59b5d86b
Parents: 0311cf0
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Dec 6 11:40:33 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Dec 6 11:40:33 2017 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44b06d34/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 65eea31..cfdadf2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -389,6 +389,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/webapps/hdfs/robots.txt</exclude>
             <exclude>src/main/webapps/journal/robots.txt</exclude>
             <exclude>src/main/webapps/secondary/robots.txt</exclude>
+            <exclude>src/main/webapps/router/robots.txt</exclude>
             <exclude>src/contrib/**</exclude>
             <exclude>src/site/resources/images/*</exclude>
             <exclude>src/main/webapps/static/bootstrap-3.0.2/**</exclude>


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


[11/50] [abbrv] hadoop git commit: MAPREDUCE-6998. Moving logging APIs over to slf4j in hadoop-mapreduce-client-jobclient. Contributed by Gergely Novák.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java
index 3dbc577..9a5ca07 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.mapred;
 import java.io.DataOutputStream;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -34,13 +32,15 @@ import org.junit.Test;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A JUnit test to test Job System Directory with Mini-DFS.
  */
 public class TestJobSysDirWithDFS {
-  private static final Log LOG =
-    LogFactory.getLog(TestJobSysDirWithDFS.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestJobSysDirWithDFS.class);
   
   static final int NUM_MAPS = 10;
   static final int NUM_SAMPLES = 100000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java
index bacc196..0991ae0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java
@@ -21,19 +21,21 @@ package org.apache.hadoop.mapred;
 import java.io.*;
 import java.util.*;
 
-import org.apache.commons.logging.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.io.compress.*;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
 public class TestKeyValueTextInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestKeyValueTextInputFormat.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestKeyValueTextInputFormat.class);
 
   private static int MAX_LENGTH = 10000;
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/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 19313d3..a23caa5 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
@@ -31,8 +31,6 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -59,14 +57,16 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.Assert;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Sets;
 
 public class TestMRTimelineEventHandling {
 
   private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector";
-  private static final Log LOG =
-      LogFactory.getLog(TestMRTimelineEventHandling.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMRTimelineEventHandling.class);
 
   @Test
   public void testTimelineServiceStartInMiniCluster() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java
index b8ff016..f364c18 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java
@@ -22,8 +22,6 @@ import java.io.File;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ipc.ProtocolSignature;
@@ -39,6 +37,9 @@ import org.apache.hadoop.mapreduce.split.JobSplitWriter;
 import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -56,7 +57,8 @@ import static org.junit.Assert.assertTrue;
  *  validated here.
  */
 public class TestMapProgress {
-  public static final Log LOG = LogFactory.getLog(TestMapProgress.class);
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestMapProgress.class);
   private static String TEST_ROOT_DIR;
   static {
     String root = new File(System.getProperty("test.build.data", "/tmp"))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
index 51f0120..340cba8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
@@ -27,8 +27,6 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -46,6 +44,8 @@ import org.apache.hadoop.util.Shell;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class to test mapred task's 
@@ -53,8 +53,8 @@ import org.junit.Test;
  *   - child env
  */
 public class TestMiniMRChildTask {
-  private static final Log LOG =
-    LogFactory.getLog(TestMiniMRChildTask.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMiniMRChildTask.class);
 
   private final static String OLD_CONFIGS = "test.old.configs";
   private final static String TASK_OPTS_VAL = "-Xmx200m";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java
index 1bd2954..3a02f4c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java
@@ -21,13 +21,14 @@ import java.io.IOException;
 import java.util.BitSet;
 import java.util.HashMap;
 import java.util.Random;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
@@ -35,7 +36,8 @@ public class TestMultiFileInputFormat {
 
   private static JobConf job = new JobConf();
 
-  private static final Log LOG = LogFactory.getLog(TestMultiFileInputFormat.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMultiFileInputFormat.class);
   
   private static final int MAX_SPLIT_COUNT  = 10000;
   private static final int SPLIT_COUNT_INCR = 6000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java
index 03dc6a6..ec53892 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java
@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.mapred;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.io.IOException;
+import java.util.Random;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BooleanWritable;
@@ -31,17 +31,16 @@ import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Random;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class TestSequenceFileAsBinaryOutputFormat {
-  private static final Log LOG =
-      LogFactory.getLog(TestSequenceFileAsBinaryOutputFormat.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSequenceFileAsBinaryOutputFormat.class);
   private static final int RECORDS = 10000;
   // A random task attempt id for testing.
   private static final String attempt = "attempt_200707121733_0001_m_000000_0";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java
index 82d1d2d..ffa42e4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java
@@ -17,18 +17,18 @@
  */
 package org.apache.hadoop.mapred;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapred.SortedRanges.Range;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.Iterator;
 
 import static org.junit.Assert.assertEquals;
 
 public class TestSortedRanges {
-  private static final Log LOG =
-    LogFactory.getLog(TestSortedRanges.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSortedRanges.class);
 
   @Test
   public void testAdd() {
@@ -56,7 +56,7 @@ public class TestSortedRanges {
     sr.add(new Range(21,50));
     assertEquals(70, sr.getIndicesCount());
     
-    LOG.debug(sr);
+    LOG.debug(sr.toString());
     
     Iterator<Long> it = sr.skipRangeIterator();
     int i = 0;
@@ -96,7 +96,7 @@ public class TestSortedRanges {
     sr.remove(new SortedRanges.Range(5,1));
     assertEquals(3, sr.getIndicesCount());
     
-    LOG.debug(sr);
+    LOG.debug(sr.toString());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
index b9e3275..5b4a141 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
@@ -18,8 +18,9 @@
 
 package org.apache.hadoop.mapred;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.net.URI;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -30,10 +31,8 @@ import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.util.Progressable;
 import org.junit.Test;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.net.URI;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -42,8 +41,8 @@ import static org.junit.Assert.fail;
  * A JUnit test to test that jobs' output filenames are not HTML-encoded (cf HADOOP-1795).
  */
 public class TestSpecialCharactersInOutputPath {
-  private static final Log LOG =
-    LogFactory.getLog(TestSpecialCharactersInOutputPath.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSpecialCharactersInOutputPath.class);
   
   private static final String OUTPUT_FILENAME = "result[0]";
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskStatus.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskStatus.java
index e71103d..94299b6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskStatus.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskStatus.java
@@ -17,13 +17,11 @@
  */
 package org.apache.hadoop.mapred;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.junit.Test;
+
 import static org.junit.Assert.*;
 
 public class TestTaskStatus {
-  private static final Log LOG = LogFactory.getLog(TestTaskStatus.class);
 
   @Test
   public void testMapTaskStatusStartAndFinishTimes() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
index 11f0bb5..0ea1d6d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
@@ -38,16 +38,15 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.*;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.ReflectionUtils;
-
 import org.junit.Test;
-import static org.junit.Assert.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import static org.junit.Assert.*;
 
 public class TestTextInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestTextInputFormat.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestTextInputFormat.class);
 
   private static int MAX_LENGTH = 10000;
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
index 378363b..10d4c21 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
@@ -48,8 +48,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CopyOnWriteArrayList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -117,7 +115,6 @@ import org.apache.log4j.Appender;
 import org.apache.log4j.AppenderSkeleton;
 import org.apache.log4j.Layout;
 import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 import org.apache.log4j.SimpleLayout;
 import org.apache.log4j.WriterAppender;
 import org.apache.log4j.spi.LoggingEvent;
@@ -128,6 +125,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.ImmutableList;
 
@@ -136,7 +135,8 @@ import com.google.common.collect.ImmutableList;
  * fine
  */
 public class TestYARNRunner {
-  private static final Log LOG = LogFactory.getLog(TestYARNRunner.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestYARNRunner.class);
   private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
 
   // prefix before <LOG_DIR>/profile.out
@@ -574,7 +574,8 @@ public class TestYARNRunner {
   }
   @Test(timeout=20000)
   public void testWarnCommandOpts() throws Exception {
-    Logger logger = Logger.getLogger(YARNRunner.class);
+    org.apache.log4j.Logger logger =
+        org.apache.log4j.Logger.getLogger(YARNRunner.class);
     
     ByteArrayOutputStream bout = new ByteArrayOutputStream();
     Layout layout = new SimpleLayout();
@@ -1006,7 +1007,8 @@ public class TestYARNRunner {
         MRJobConfig.RESOURCE_TYPE_NAME_MEMORY,
         MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) {
       TestAppender testAppender = new TestAppender();
-      Logger logger = Logger.getLogger(YARNRunner.class);
+      org.apache.log4j.Logger  logger =
+          org.apache.log4j.Logger.getLogger(YARNRunner.class);
       logger.addAppender(testAppender);
       try {
         JobConf jobConf = new JobConf();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ThreadedMapBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ThreadedMapBenchmark.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ThreadedMapBenchmark.java
index c607bfb..68fe1a0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ThreadedMapBenchmark.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ThreadedMapBenchmark.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.io.File;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -36,6 +34,8 @@ import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Distributed threaded map benchmark.
@@ -52,7 +52,8 @@ import org.apache.hadoop.util.ToolRunner;
 
 public class ThreadedMapBenchmark extends Configured implements Tool {
 
-  private static final Log LOG = LogFactory.getLog(ThreadedMapBenchmark.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ThreadedMapBenchmark.class);
   private static Path BASE_DIR =
     new Path(System.getProperty("test.build.data", 
                                 File.separator + "benchmarks" + File.separator 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
index 935c175..89ac3ea 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
@@ -34,8 +34,6 @@ import java.util.Properties;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -60,6 +58,8 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Supplier;
 
@@ -69,7 +69,7 @@ import com.google.common.base.Supplier;
  */
 public class UtilsForTests {
 
-  static final Log LOG = LogFactory.getLog(UtilsForTests.class);
+  static final Logger LOG = LoggerFactory.getLogger(UtilsForTests.class);
 
   final static long KB = 1024L * 1;
   final static long MB = 1024L * KB;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
index 07b1306..3cd9c24 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
@@ -21,13 +21,14 @@ package org.apache.hadoop.mapred.jobcontrol;
 import java.io.IOException;
 import java.util.ArrayList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -35,8 +36,8 @@ import static org.junit.Assert.assertEquals;
  */
 public class TestLocalJobControl extends HadoopTestCase {
 
-  public static final Log LOG = LogFactory.getLog(TestLocalJobControl.class
-      .getName());
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestLocalJobControl.class);
 
   /**
    * Initialises a new instance of this test case to use a Local MR cluster and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java
index 84b491a..f7ef958 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java
@@ -23,8 +23,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.ArrayList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileUtil;
@@ -45,14 +43,16 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 
 @Ignore
 public class TestPipes {
-  private static final Log LOG =
-    LogFactory.getLog(TestPipes.class.getName());
+  private static final Logger LOG = LoggerFactory.getLogger(TestPipes.class);
 
   private static Path cppExamples =
     new Path(System.getProperty("install.c++.examples"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/GrowingSleepJob.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/GrowingSleepJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/GrowingSleepJob.java
index 55740f7..92b72cc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/GrowingSleepJob.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/GrowingSleepJob.java
@@ -17,20 +17,20 @@
  */
 package org.apache.hadoop.mapreduce;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.io.IOException;
+import java.util.ArrayList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.util.ToolRunner;
-
-import java.io.IOException;
-import java.util.ArrayList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A sleep job whose mappers create 1MB buffer for every record.
  */
 public class GrowingSleepJob extends SleepJob {
-  private static final Log LOG = LogFactory.getLog(GrowingSleepJob.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(GrowingSleepJob.class);
 
   public static class GrowingSleepMapper extends SleepMapper {
     private final int MB = 1024 * 1024;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java
index 5d9dc0b..3783466 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java
@@ -20,19 +20,20 @@ package org.apache.hadoop.mapreduce;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Used to parse job history and configuration files.
  */
 class JobHistoryFileParser {
-  private static final Log LOG = LogFactory.getLog(JobHistoryFileParser.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JobHistoryFileParser.class);
 
   private final FileSystem fs;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayHelper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayHelper.java
index 8acd26e..4b47b43 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayHelper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayHelper.java
@@ -24,19 +24,18 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class JobHistoryFileReplayHelper {
-  private static final Log LOG =
-      LogFactory.getLog(JobHistoryFileReplayHelper.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JobHistoryFileReplayHelper.class);
   static final String PROCESSING_PATH = "processing path";
   static final String REPLAY_MODE = "replay mode";
   static final int WRITE_ALL_AT_ONCE = 1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java
index d553596..1d0896f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java
@@ -23,8 +23,6 @@ import java.util.Collection;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
@@ -38,7 +36,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Mapper for TimelineServicePerformanceV1 that replays job history files to the
@@ -48,8 +47,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 class JobHistoryFileReplayMapperV1 extends
     org.apache.hadoop.mapreduce.
         Mapper<IntWritable,IntWritable,Writable,Writable> {
-  private static final Log LOG =
-      LogFactory.getLog(JobHistoryFileReplayMapperV1.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JobHistoryFileReplayMapperV1.class);
 
   public void map(IntWritable key, IntWritable val, Context context) throws IOException {
     // collect the apps it needs to process

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java
index 2ec4833..f8e6ca0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java
@@ -23,8 +23,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapreduce.JobHistoryFileReplayHelper.JobFiles;
@@ -38,6 +36,8 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector;
 import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Mapper for TimelineServicePerformance that replays job history files to the
@@ -45,8 +45,8 @@ import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector
  *
  */
 class JobHistoryFileReplayMapperV2 extends EntityWriterV2 {
-  private static final Log LOG =
-      LogFactory.getLog(JobHistoryFileReplayMapperV2.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JobHistoryFileReplayMapperV2.class);
 
   @Override
   protected void writeEntities(Configuration tlConf,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MapReduceTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MapReduceTestUtil.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MapReduceTestUtil.java
index 8351b53..2f30bb5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MapReduceTestUtil.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MapReduceTestUtil.java
@@ -31,8 +31,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -46,22 +44,19 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.TaskLog;
 import org.apache.hadoop.mapred.Utils;
-import org.apache.hadoop.mapred.TaskLog.LogName;
-import org.apache.hadoop.mapred.TaskLog.Reader;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Utility methods used in various Job Control unit tests.
  */
 public class MapReduceTestUtil {
-  public static final Log LOG = 
-    LogFactory.getLog(MapReduceTestUtil.class.getName());
+  public static final Logger LOG =
+      LoggerFactory.getLogger(MapReduceTestUtil.class);
 
   static private Random rand = new Random();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java
index 324f0ca..b89cdc0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java
@@ -34,8 +34,6 @@ import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -46,6 +44,8 @@ import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.eclipse.jetty.util.ajax.JSON;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class drives the creation of a mini-cluster on the local machine. By
@@ -60,8 +60,8 @@ import org.eclipse.jetty.util.ajax.JSON;
  * To shutdown the cluster, kill the process.
  */
 public class MiniHadoopClusterManager {
-  private static final Log LOG = LogFactory
-      .getLog(MiniHadoopClusterManager.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MiniHadoopClusterManager.class);
 
   private MiniMRClientCluster mr;
   private MiniDFSCluster dfs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java
index 6d6151f..219f552 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
@@ -32,6 +30,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
    * Adds simple entities with random string payload, events, metrics, and
@@ -41,7 +41,8 @@ class SimpleEntityWriterV1
     extends org.apache.hadoop.mapreduce.Mapper
         <IntWritable, IntWritable, Writable, Writable>
     implements SimpleEntityWriterConstants {
-  private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV1.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SimpleEntityWriterV1.class);
 
   public void map(IntWritable key, IntWritable val, Context context)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java
index d66deb0..93633a06 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -35,6 +33,8 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector;
 import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Adds simple entities with random string payload, events, metrics, and
@@ -42,7 +42,8 @@ import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector
  */
 class SimpleEntityWriterV2 extends EntityWriterV2
     implements SimpleEntityWriterConstants {
-  private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV2.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SimpleEntityWriterV2.class);
 
   protected void writeEntities(Configuration tlConf,
       TimelineCollectorManager manager, Context context) throws IOException {
@@ -128,4 +129,4 @@ class SimpleEntityWriterV2 extends EntityWriterV2
       manager.remove(appId);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestCounters.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestCounters.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestCounters.java
index 83d689c..8c79435 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestCounters.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestCounters.java
@@ -19,20 +19,19 @@ package org.apache.hadoop.mapreduce;
 
 import java.util.Random;
 
-import org.junit.Test;
-import static org.junit.Assert.*;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapreduce.counters.LimitExceededException;
 import org.apache.hadoop.mapreduce.counters.Limits;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.junit.Test;
 
+import static org.junit.Assert.*;
 /**
  * TestCounters checks the sanity and recoverability of {@code Counters}
  */
 public class TestCounters {
 
-  static final Log LOG = LogFactory.getLog(TestCounters.class);
+  static final Logger LOG = LoggerFactory.getLogger(TestCounters.class);
 
   /**
    * Verify counter value works

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java
index 8177ecd..7630e85 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.mapreduce;
 
-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.FileStatus;
@@ -33,6 +31,8 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
@@ -54,7 +54,8 @@ import static org.junit.Assert.assertTrue;
  */
 public class TestLocalRunner {
 
-  private static final Log LOG = LogFactory.getLog(TestLocalRunner.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestLocalRunner.class);
 
   private static int INPUT_SIZES[] =
     new int[] { 50000, 500, 500, 20,  5000, 500};
@@ -330,9 +331,9 @@ public class TestLocalRunner {
     try {
       job.waitForCompletion(true);
     } catch (InterruptedException ie) {
-      LOG.fatal("Interrupted while waiting for job completion", ie);
+      LOG.error("Interrupted while waiting for job completion", ie);
       for (int i = 0; i < 10; i++) {
-        LOG.fatal("Dumping stacks");
+        LOG.error("Dumping stacks");
         ReflectionUtils.logThreadInfo(LOG, "multimap threads", 0);
         Thread.sleep(1000);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java
index 3a33bb1..f4ccc56 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.mapreduce;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -33,6 +31,8 @@ import org.apache.hadoop.util.ToolRunner;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
@@ -60,7 +60,8 @@ import static org.junit.Assert.fail;
  */
 public class TestMRJobClient extends ClusterMapReduceTestCase {
 
-  private static final Log LOG = LogFactory.getLog(TestMRJobClient.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMRJobClient.class);
 
   private Job runJob(Configuration conf) throws Exception {
     String input = "hello1\nhello2\nhello3\n";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapCollection.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapCollection.java
index afe4a10..b6ec120 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapCollection.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapCollection.java
@@ -24,22 +24,21 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
-import org.junit.Test;
-import static org.junit.Assert.*;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
 
 public class TestMapCollection {
 
-  private static final Log LOG = LogFactory.getLog(
+  private static final Logger LOG = LoggerFactory.getLogger(
       TestMapCollection.class.getName());
 
   public static abstract class FillWritable implements Writable, Configurable {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java
index b757fb2..4bcacd8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java
@@ -27,8 +27,6 @@ import java.io.Writer;
 import java.util.ArrayList;
 import java.util.StringTokenizer;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -42,6 +40,8 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertTrue;
 
@@ -66,8 +66,8 @@ public class TestValueIterReset {
     }
   }
 
-  private static final Log LOG =
-    LogFactory.getLog(TestValueIterReset.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestValueIterReset.class);
 
   public static class TestMapper 
   extends Mapper<LongWritable, Text, IntWritable, IntWritable> {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java
index dcc3ce0..a6fc58e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java
@@ -22,17 +22,17 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class TimelineEntityConverterV1 {
-  private static final Log LOG =
-      LogFactory.getLog(TimelineEntityConverterV1.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineEntityConverterV1.class);
 
   static final String JOB = "MAPREDUCE_JOB";
   static final String TASK = "MAPREDUCE_TASK";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java
index 45812fe..48c470b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java
@@ -24,8 +24,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
@@ -34,8 +34,8 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 class TimelineEntityConverterV2 {
-  private static final Log LOG =
-      LogFactory.getLog(TimelineEntityConverterV2.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineEntityConverterV2.class);
 
   static final String JOB = "MAPREDUCE_JOB";
   static final String TASK = "MAPREDUCE_TASK";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
index 81a3249..39b63d5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.mapreduce.lib.db;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -36,6 +34,8 @@ import org.hsqldb.server.Server;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -58,7 +58,7 @@ import static org.junit.Assert.assertTrue;
  */
 public class TestDataDrivenDBInputFormat extends HadoopTestCase {
 
-  private static final Log LOG = LogFactory.getLog(
+  private static final Logger LOG = LoggerFactory.getLogger(
       TestDataDrivenDBInputFormat.class);
 
   private static final String DB_NAME = "dddbif";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java
index cb54158..dbbdc3d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java
@@ -27,8 +27,6 @@ import java.util.BitSet;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -44,10 +42,12 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestCombineSequenceFileInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestCombineSequenceFileInputFormat.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestCombineSequenceFileInputFormat.class);
   private static Configuration conf = new Configuration();
   private static FileSystem localFs = null;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java
index cb1dffc..3dd2333 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java
@@ -32,8 +32,6 @@ import java.util.BitSet;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -51,10 +49,12 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestCombineTextInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestCombineTextInputFormat.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestCombineTextInputFormat.class);
 
   private static Configuration defaultConf = new Configuration();
   private static FileSystem localFs = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java
index b82b495..684d3e1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java
@@ -26,8 +26,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -41,16 +39,19 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
-
 import org.apache.hadoop.util.ReflectionUtils;
-
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.junit.Assert.*;
 
 public class TestFixedLengthInputFormat {
 
-  private static Log LOG;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFixedLengthInputFormat.class);
+
   private static Configuration defaultConf;
   private static FileSystem localFs;
   private static Path workDir;
@@ -62,7 +63,6 @@ public class TestFixedLengthInputFormat {
   @BeforeClass
   public static void onlyOnce() {
     try {
-      LOG = LogFactory.getLog(TestFixedLengthInputFormat.class.getName());
       defaultConf = new Configuration();
       defaultConf.set("fs.defaultFS", "file:///");
       localFs = FileSystem.getLocal(defaultConf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRKeyValueTextInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRKeyValueTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRKeyValueTextInputFormat.java
index f181c9c..537d23c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRKeyValueTextInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRKeyValueTextInputFormat.java
@@ -28,7 +28,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
@@ -42,16 +41,15 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.ReflectionUtils;
-
 import org.junit.Test;
-import static org.junit.Assert.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import static org.junit.Assert.*;
 
 public class TestMRKeyValueTextInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestMRKeyValueTextInputFormat.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMRKeyValueTextInputFormat.class);
 
   private static Configuration defaultConf = new Configuration();
   private static FileSystem localFs = null; 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java
index 89aa7b2..178eb50 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.mapreduce.lib.input;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -35,6 +33,8 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Random;
@@ -42,8 +42,8 @@ import java.util.Random;
 import static org.junit.Assert.assertEquals;
 
 public class TestMRSequenceFileInputFilter {
-  private static final Log LOG =
-    LogFactory.getLog(TestMRSequenceFileInputFilter.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMRSequenceFileInputFilter.class);
 
   private static final int MAX_LENGTH = 15000;
   private static final Configuration conf = new Configuration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
index da011a2..534593e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
@@ -22,17 +22,17 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.junit.Assert;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.junit.Assert;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -43,8 +43,8 @@ import static org.junit.Assert.assertFalse;
  */
 public class TestMapReduceJobControl extends HadoopTestCase {
 
-  public static final Log LOG = 
-      LogFactory.getLog(TestMapReduceJobControl.class.getName());
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestMapReduceJobControl.class);
 
   static Path rootDataDir = new Path(
     System.getProperty("test.build.data", "."), "TestData");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java
index 5a8aeda..ad213d3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.mapreduce.lib.output;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -44,6 +42,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Random;
@@ -53,8 +53,8 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class TestMRSequenceFileAsBinaryOutputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestMRSequenceFileAsBinaryOutputFormat.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMRSequenceFileAsBinaryOutputFormat.class);
 
   private static final int RECORDS = 10000;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java
index 4d05d13..54c4cc3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java
@@ -17,15 +17,17 @@
  */
 package org.apache.hadoop.mapreduce.lib.partition;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 public class TestKeyFieldHelper {
-  private static final Log LOG = LogFactory.getLog(TestKeyFieldHelper.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestKeyFieldHelper.class);
+
   /**
    * Test is key-field-helper's parse option.
    */
@@ -427,4 +429,4 @@ public class TestKeyFieldHelper {
     }
     return true;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java
index 387e202..d8c4b2a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java
@@ -28,8 +28,6 @@ import java.security.PrivilegedExceptionAction;
 
 import org.junit.Assert;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.mapred.JobConf;
@@ -54,17 +52,19 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestJHSSecurity {
 
-  private static final Log LOG = LogFactory.getLog(TestJHSSecurity.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestJHSSecurity.class);
   
   @Test
   public void testDelegationToken() throws IOException, InterruptedException {
 
-    Logger rootLogger = LogManager.getRootLogger();
+    org.apache.log4j.Logger rootLogger = LogManager.getRootLogger();
     rootLogger.setLevel(Level.DEBUG);
 
     final YarnConfiguration conf = new YarnConfiguration(new JobConf());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java
index be46385..0c7a173 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java
@@ -24,8 +24,6 @@ import java.util.Date;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -34,6 +32,8 @@ import org.apache.hadoop.util.AsyncDiskService;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class is a container of multiple thread pools, each for a volume,
@@ -53,7 +53,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Unstable
 public class MRAsyncDiskService {
   
-  public static final Log LOG = LogFactory.getLog(MRAsyncDiskService.class);
+  public static final Logger LOG =
+      LoggerFactory.getLogger(MRAsyncDiskService.class);
   
   AsyncDiskService asyncDiskService;
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java
index 07b5d8b..fbccd9f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java
@@ -20,16 +20,14 @@ package org.apache.hadoop.mapreduce.util;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.mapreduce.util.MRAsyncDiskService;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -42,7 +40,8 @@ import static org.junit.Assert.fail;
  */
 public class TestMRAsyncDiskService {
 
-  public static final Log LOG = LogFactory.getLog(TestMRAsyncDiskService.class);
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestMRAsyncDiskService.class);
   
   private static String TEST_ROOT_DIR = new Path(System.getProperty(
       "test.build.data", "/tmp")).toString();


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


[31/50] [abbrv] hadoop git commit: YARN-7564. Cleanup to fix checkstyle issues of YARN-5881 branch. Contributed by Sunil G.

Posted by wa...@apache.org.
YARN-7564. Cleanup to fix checkstyle issues of YARN-5881 branch. Contributed by Sunil G.


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

Branch: refs/heads/YARN-6592
Commit: daa1cdd062657a47acbf4b23f895860296241199
Parents: 1012b90
Author: Sunil G <su...@apache.org>
Authored: Tue Nov 28 14:07:09 2017 +0530
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/util/StringUtils.java     | 18 ++++++-------
 .../impl/pb/QueueConfigurationsPBImpl.java      | 10 +++----
 .../resource/DominantResourceCalculator.java    |  2 +-
 .../yarn/util/resource/ResourceCalculator.java  | 12 ++++-----
 .../scheduler/AbstractResourceUsage.java        |  8 +++++-
 .../scheduler/ResourceUsage.java                |  6 -----
 .../scheduler/capacity/CSQueue.java             | 28 +++++++++++++++++---
 .../CapacitySchedulerConfiguration.java         | 28 ++++++++++++--------
 .../scheduler/capacity/TestReservations.java    |  3 +--
 9 files changed, 70 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/daa1cdd0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
index 1be8a08..ebe7013 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
@@ -1171,16 +1171,16 @@ public class StringUtils {
    * @return <code>true</code> if only contains letters, and is non-null
    */
   public static boolean isAlpha(String str) {
-      if (str == null) {
-          return false;
-      }
-      int sz = str.length();
-      for (int i = 0; i < sz; i++) {
-          if (Character.isLetter(str.charAt(i)) == false) {
-              return false;
-          }
+    if (str == null) {
+      return false;
+    }
+    int sz = str.length();
+    for (int i = 0; i < sz; i++) {
+      if (!Character.isLetter(str.charAt(i))) {
+        return false;
       }
-      return true;
+    }
+    return true;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/daa1cdd0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java
index 80ef4b2..8266013 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java
@@ -236,12 +236,12 @@ public class QueueConfigurationsPBImpl extends QueueConfigurations {
   }
 
   @Override
-  public void setConfiguredMinCapacity(Resource configuredMinResource) {
+  public void setConfiguredMinCapacity(Resource minResource) {
     maybeInitBuilder();
-    if (configuredMinResource == null) {
+    if (minResource == null) {
       builder.clearConfiguredMinCapacity();
     }
-    this.configuredMinResource = configuredMinResource;
+    this.configuredMinResource = minResource;
   }
 
   @Override
@@ -259,11 +259,11 @@ public class QueueConfigurationsPBImpl extends QueueConfigurations {
   }
 
   @Override
-  public void setConfiguredMaxCapacity(Resource configuredMaxResource) {
+  public void setConfiguredMaxCapacity(Resource maxResource) {
     maybeInitBuilder();
     if (configuredMaxResource == null) {
       builder.clearConfiguredMaxCapacity();
     }
-    this.configuredMaxResource = configuredMaxResource;
+    this.configuredMaxResource = maxResource;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/daa1cdd0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index 26ddc18..6fed23b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -515,7 +515,7 @@ public class DominantResourceCalculator extends ResourceCalculator {
     }
     return ret;
   }
-  
+
   @Override
   public Resource multiplyAndNormalizeUp(Resource r, double by,
       Resource stepFactor) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/daa1cdd0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
index dfd6684..1c42126 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
@@ -127,12 +127,12 @@ public abstract class ResourceCalculator {
       Resource r, double by, Resource stepFactor);
 
   /**
-   * Multiply resource <code>r</code> by factor <code>by</code> 
+   * Multiply resource <code>r</code> by factor <code>by</code>
    * and normalize up using step-factor <code>stepFactor</code>.
-   * 
+   *
    * @param r resource to be multiplied
    * @param by multiplier array for all resource types
-   * @param stepFactor factor by which to normalize up 
+   * @param stepFactor factor by which to normalize up
    * @return resulting normalized resource
    */
   public abstract Resource multiplyAndNormalizeUp(
@@ -149,7 +149,7 @@ public abstract class ResourceCalculator {
    */
   public abstract Resource multiplyAndNormalizeDown(
       Resource r, double by, Resource stepFactor);
-  
+
   /**
    * Normalize resource <code>r</code> given the base 
    * <code>minimumResource</code> and verify against max allowed
@@ -162,9 +162,7 @@ public abstract class ResourceCalculator {
    * @return normalized resource
    */
   public abstract Resource normalize(Resource r, Resource minimumResource,
-                                     Resource maximumResource, 
-                                     Resource stepFactor);
-
+      Resource maximumResource, Resource stepFactor);
 
   /**
    * Round-up resource <code>r</code> given factor <code>stepFactor</code>.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/daa1cdd0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.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/AbstractResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java
index 3203543..1406fec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java
@@ -55,7 +55,10 @@ public class AbstractResourceUsage {
     usages.put(CommonNodeLabelsManager.NO_LABEL, noLabelUsages);
   }
 
-  // Usage enum here to make implement cleaner
+  /**
+   * Use enum here to make implementation more cleaner and readable.
+   * Indicates array index for each resource usage type.
+   */
   public enum ResourceType {
     // CACHED_USED and CACHED_PENDING may be read by anyone, but must only
     // be written by ordering policies
@@ -70,6 +73,9 @@ public class AbstractResourceUsage {
     }
   }
 
+  /**
+   * UsageByLabel stores resource array for all resource usage types.
+   */
   public static class UsageByLabel {
     // usage by label, contains all UsageType
     private final AtomicReferenceArray<Resource> resArr;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/daa1cdd0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.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/ResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
index ede4aec..711a468 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
@@ -18,13 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/daa1cdd0/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 624fcc7..6d79b6a 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
@@ -366,7 +366,7 @@ public interface CSQueue extends SchedulerQueue<CSQueue> {
   public QueueResourceQuotas getQueueResourceQuotas();
 
   /**
-   * Get CapacityConfigType as PERCENTAGE or ABSOLUTE_RESOURCE
+   * Get CapacityConfigType as PERCENTAGE or ABSOLUTE_RESOURCE.
    * @return CapacityConfigType
    */
   public CapacityConfigType getCapacityConfigType();
@@ -374,24 +374,46 @@ public interface CSQueue extends SchedulerQueue<CSQueue> {
   /**
    * Get effective capacity of queue. If min/max resource is configured,
    * preference will be given to absolute configuration over normal capacity.
-   * Also round down the result to normalizeDown.
    *
    * @param label
    *          partition
    * @return effective queue capacity
    */
   Resource getEffectiveCapacity(String label);
+
+  /**
+   * Get effective capacity of queue. If min/max resource is configured,
+   * preference will be given to absolute configuration over normal capacity.
+   * Also round down the result to normalizeDown.
+   *
+   * @param label
+   *          partition
+   * @param factor
+   *          factor to normalize down 
+   * @return effective queue capacity
+   */
   Resource getEffectiveCapacityDown(String label, Resource factor);
 
   /**
    * Get effective max capacity of queue. If min/max resource is configured,
    * preference will be given to absolute configuration over normal capacity.
-   * Also round down the result to normalizeDown.
    *
    * @param label
    *          partition
    * @return effective max queue capacity
    */
   Resource getEffectiveMaxCapacity(String label);
+
+  /**
+   * Get effective max capacity of queue. If min/max resource is configured,
+   * preference will be given to absolute configuration over normal capacity.
+   * Also round down the result to normalizeDown.
+   *
+   * @param label
+   *          partition
+   * @param factor
+   *          factor to normalize down 
+   * @return effective max queue capacity
+   */
   Resource getEffectiveMaxCapacityDown(String label, Resource factor);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/daa1cdd0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index e47c8d0..a33d81a 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.ReservationACL;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
@@ -320,7 +321,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @Private
   public static final int DEFAULT_MAX_ASSIGN_PER_HEARTBEAT = -1;
 
-  /** Configuring absolute min/max resources in a queue **/
+  /** Configuring absolute min/max resources in a queue. **/
   @Private
   public static final String MINIMUM_RESOURCE = "min-resource";
 
@@ -333,6 +334,9 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
 
   private static final Pattern RESOURCE_PATTERN = Pattern.compile(PATTERN_FOR_ABSOLUTE_RESOURCE);
 
+  /**
+   * Different resource types supported.
+   */
   public enum AbsoluteResourceType {
     MEMORY, VCORES;
   }
@@ -1825,7 +1829,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     }
 
     // Define resource here.
-    Resource resource = Resource.newInstance(0l, 0);
+    Resource resource = Resource.newInstance(0L, 0);
     Matcher matcher = RESOURCE_PATTERN.matcher(resourceString);
 
     /*
@@ -1852,7 +1856,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     }
 
     // Memory has to be configured always.
-    if (resource.getMemorySize() == 0l) {
+    if (resource.getMemorySize() == 0L) {
       return Resources.none();
     }
 
@@ -1884,14 +1888,16 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     AbsoluteResourceType resType = AbsoluteResourceType
         .valueOf(StringUtils.toUpperCase(splits[0].trim()));
     switch (resType) {
-      case MEMORY :
-        resource.setMemorySize(resourceValue);
-        break;
-      case VCORES :
-        resource.setVirtualCores(resourceValue.intValue());
-        break;
-      default :
-        break;
+    case MEMORY :
+      resource.setMemorySize(resourceValue);
+      break;
+    case VCORES :
+      resource.setVirtualCores(resourceValue.intValue());
+      break;
+    default :
+      resource.setResourceInformation(splits[0].trim(), ResourceInformation
+          .newInstance(splits[0].trim(), units, resourceValue));
+      break;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/daa1cdd0/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 0855720..8705887 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
@@ -904,12 +904,11 @@ 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);
     root.updateClusterResource(clusterResource,
         new ResourceLimits(clusterResource));
 
-
     // Setup resource-requests
     Priority p = TestUtils.createMockPriority(5);
     SchedulerRequestKey priorityMap = toSchedulerKey(p);


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


[20/50] [abbrv] hadoop git commit: YARN-7411. Inter-Queue preemption's computeFixpointAllocation need to handle absolute resources while computing normalizedGuarantee. (Sunil G via wangda)

Posted by wa...@apache.org.
YARN-7411. Inter-Queue preemption's computeFixpointAllocation need to handle absolute resources while computing normalizedGuarantee. (Sunil G via wangda)

Change-Id: I41b1d7558c20fc4eb2050d40134175a2ef6330cb


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

Branch: refs/heads/YARN-6592
Commit: 034b312d9f19024d2eabd377210d17d4080ef70e
Parents: aa3f627
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Nov 13 16:26:27 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../api/records/impl/pb/ResourcePBImpl.java     | 12 ----
 .../resource/DefaultResourceCalculator.java     |  8 +++
 .../resource/DominantResourceCalculator.java    | 21 ++++++
 .../yarn/util/resource/ResourceCalculator.java  | 14 +++-
 .../hadoop/yarn/util/resource/Resources.java    |  5 ++
 .../AbstractPreemptableResourceCalculator.java  | 24 ++++++-
 .../monitor/capacity/TempQueuePerPartition.java | 12 ++--
 ...alCapacityPreemptionPolicyMockFramework.java | 14 ++++
 ...pacityPreemptionPolicyForNodePartitions.java | 76 ++++++++++++++++++++
 9 files changed, 166 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/034b312d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
index 401e0c0..4f90133 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceInformationProto;
@@ -152,17 +151,6 @@ public class ResourcePBImpl extends Resource {
         .newInstance(ResourceInformation.VCORES);
     this.setMemorySize(p.getMemory());
     this.setVirtualCores(p.getVirtualCores());
-
-    // Update missing resource information on respective index.
-    updateResourceInformationMap(types);
-  }
-
-  private void updateResourceInformationMap(ResourceInformation[] types) {
-    for (int i = 0; i < types.length; i++) {
-      if (resources[i] == null) {
-        resources[i] = ResourceInformation.newInstance(types[i]);
-      }
-    }
   }
 
   private static ResourceInformation newDefaultInformation(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/034b312d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
index aefa85c..6375c4a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
@@ -112,6 +112,14 @@ public class DefaultResourceCalculator extends ResourceCalculator {
   }
 
   @Override
+  public Resource multiplyAndNormalizeUp(Resource r, double[] by,
+      Resource stepFactor) {
+    return Resources.createResource(
+        roundUp((long) (r.getMemorySize() * by[0] + 0.5),
+            stepFactor.getMemorySize()));
+  }
+
+  @Override
   public Resource multiplyAndNormalizeDown(Resource r, double by,
       Resource stepFactor) {
     return Resources.createResource(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/034b312d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index cc4acf6..26ddc18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -496,6 +496,27 @@ public class DominantResourceCalculator extends ResourceCalculator {
   }
 
   @Override
+  public Resource multiplyAndNormalizeUp(Resource r, double[] by,
+      Resource stepFactor) {
+    Resource ret = Resource.newInstance(r);
+    int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
+    for (int i = 0; i < maxLength; i++) {
+      ResourceInformation rResourceInformation = r.getResourceInformation(i);
+      ResourceInformation stepFactorResourceInformation = stepFactor
+          .getResourceInformation(i);
+
+      long rValue = rResourceInformation.getValue();
+      long stepFactorValue = UnitsConversionUtil.convert(
+          stepFactorResourceInformation.getUnits(),
+          rResourceInformation.getUnits(),
+          stepFactorResourceInformation.getValue());
+      ret.setResourceValue(i, ResourceCalculator
+          .roundUp((long) Math.ceil(rValue * by[i]), stepFactorValue));
+    }
+    return ret;
+  }
+  
+  @Override
   public Resource multiplyAndNormalizeUp(Resource r, double by,
       Resource stepFactor) {
     return this.multiplyAndNormalize(r, by, stepFactor, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/034b312d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
index a816290..dfd6684 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
@@ -125,7 +125,19 @@ public abstract class ResourceCalculator {
    */
   public abstract Resource multiplyAndNormalizeUp(
       Resource r, double by, Resource stepFactor);
-  
+
+  /**
+   * Multiply resource <code>r</code> by factor <code>by</code> 
+   * and normalize up using step-factor <code>stepFactor</code>.
+   * 
+   * @param r resource to be multiplied
+   * @param by multiplier array for all resource types
+   * @param stepFactor factor by which to normalize up 
+   * @return resulting normalized resource
+   */
+  public abstract Resource multiplyAndNormalizeUp(
+      Resource r, double[] by, Resource stepFactor);
+
   /**
    * Multiply resource <code>r</code> by factor <code>by</code> 
    * and normalize down using step-factor <code>stepFactor</code>.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/034b312d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index 0426ab7..1c08844 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -347,6 +347,11 @@ public class Resources {
     return lhs;
   }
 
+  public static Resource multiplyAndNormalizeUp(ResourceCalculator calculator,
+      Resource lhs, double[] by, Resource factor) {
+    return calculator.multiplyAndNormalizeUp(lhs, by, factor);
+  }
+
   public static Resource multiplyAndNormalizeUp(
       ResourceCalculator calculator,Resource lhs, double by, Resource factor) {
     return calculator.multiplyAndNormalizeUp(lhs, by, factor);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/034b312d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.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/monitor/capacity/AbstractPreemptableResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java
index a80f317..5196831 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java
@@ -19,8 +19,11 @@
 package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.PriorityUtilizationQueueOrderingPolicy;
+import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import java.util.ArrayList;
@@ -198,18 +201,33 @@ public class AbstractPreemptableResourceCalculator {
   private void resetCapacity(Resource clusterResource,
       Collection<TempQueuePerPartition> queues, boolean ignoreGuar) {
     Resource activeCap = Resource.newInstance(0, 0);
+    int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
 
     if (ignoreGuar) {
       for (TempQueuePerPartition q : queues) {
-        q.normalizedGuarantee = 1.0f / queues.size();
+        for (int i = 0; i < maxLength; i++) {
+          q.normalizedGuarantee[i] = 1.0f / queues.size();
+        }
       }
     } else {
       for (TempQueuePerPartition q : queues) {
         Resources.addTo(activeCap, q.getGuaranteed());
       }
       for (TempQueuePerPartition q : queues) {
-        q.normalizedGuarantee = Resources.divide(rc, clusterResource,
-            q.getGuaranteed(), activeCap);
+        for (int i = 0; i < maxLength; i++) {
+          ResourceInformation nResourceInformation = q.getGuaranteed()
+              .getResourceInformation(i);
+          ResourceInformation dResourceInformation = activeCap
+              .getResourceInformation(i);
+
+          long nValue = nResourceInformation.getValue();
+          long dValue = UnitsConversionUtil.convert(
+              dResourceInformation.getUnits(), nResourceInformation.getUnits(),
+              dResourceInformation.getValue());
+          if (dValue != 0) {
+            q.normalizedGuarantee[i] = (float) nValue / dValue;
+          }
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/034b312d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.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/monitor/capacity/TempQueuePerPartition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
index bd236fe..4d71223 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
@@ -22,9 +22,11 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -46,7 +48,7 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
   Resource untouchableExtra;
   Resource preemptableExtra;
 
-  double normalizedGuarantee;
+  double[] normalizedGuarantee;
 
   private Resource effMinRes;
   private Resource effMaxRes;
@@ -88,7 +90,8 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
       pendingDeductReserved = Resources.createResource(0);
     }
 
-    this.normalizedGuarantee = Float.NaN;
+    this.normalizedGuarantee = new double[ResourceUtils
+        .getNumberOfKnownResourceTypes()];
     this.children = new ArrayList<>();
     this.apps = new ArrayList<>();
     this.untouchableExtra = Resource.newInstance(0, 0);
@@ -240,8 +243,9 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
     sb.append(" NAME: " + queueName).append(" CUR: ").append(current)
         .append(" PEN: ").append(pending).append(" RESERVED: ").append(reserved)
         .append(" GAR: ").append(getGuaranteed()).append(" NORM: ")
-        .append(normalizedGuarantee).append(" IDEAL_ASSIGNED: ")
-        .append(idealAssigned).append(" IDEAL_PREEMPT: ").append(toBePreempted)
+        .append(Arrays.toString(normalizedGuarantee))
+        .append(" IDEAL_ASSIGNED: ").append(idealAssigned)
+        .append(" IDEAL_PREEMPT: ").append(toBePreempted)
         .append(" ACTUAL_PREEMPT: ").append(getActuallyToBePreempted())
         .append(" UNTOUCHABLE: ").append(untouchableExtra)
         .append(" PREEMPTABLE: ").append(preemptableExtra).append("\n");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/034b312d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.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/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
index 1dda8a6..ca43a95 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 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.ResourceInformation;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -56,6 +57,7 @@ import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Before;
@@ -533,6 +535,18 @@ public class ProportionalCapacityPreemptionPolicyMockFramework {
     } else {
       res = Resources.createResource(Integer.valueOf(resource[0]),
           Integer.valueOf(resource[1]));
+      if (resource.length > 2) {
+        // Using the same order of resources from ResourceUtils, set resource
+        // informations.
+        ResourceInformation[] storedResourceInfo = ResourceUtils
+            .getResourceTypesArray();
+        for (int i = 2; i < resource.length; i++) {
+          res.setResourceInformation(storedResourceInfo[i].getName(),
+              ResourceInformation.newInstance(storedResourceInfo[i].getName(),
+                  storedResourceInfo[i].getUnits(),
+                  Integer.valueOf(resource[i])));
+        }
+      }
     }
     return res;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/034b312d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.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/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
index 1fd455a..67c09cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
@@ -18,11 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 
 import static org.mockito.Matchers.argThat;
 import static org.mockito.Mockito.never;
@@ -613,4 +619,74 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions
     verify(mDisp, never()).handle(
         argThat(new IsPreemptionRequestFor(getAppAttemptId(3))));
   }
+
+  @Test
+  public void testNormalizeGuaranteeWithMultipleResource() throws IOException {
+    // Initialize resource map
+    Map<String, ResourceInformation> riMap = new HashMap<>();
+    String RESOURCE_1 = "res1";
+
+    // Initialize mandatory resources
+    ResourceInformation memory = ResourceInformation.newInstance(
+        ResourceInformation.MEMORY_MB.getName(),
+        ResourceInformation.MEMORY_MB.getUnits(),
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
+    ResourceInformation vcores = ResourceInformation.newInstance(
+        ResourceInformation.VCORES.getName(),
+        ResourceInformation.VCORES.getUnits(),
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+    riMap.put(ResourceInformation.MEMORY_URI, memory);
+    riMap.put(ResourceInformation.VCORES_URI, vcores);
+    riMap.put(RESOURCE_1, ResourceInformation.newInstance(RESOURCE_1, "", 0,
+        ResourceTypes.COUNTABLE, 0, Integer.MAX_VALUE));
+
+    ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
+
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *           root
+     *           /  \
+     *          a    b
+     *        /  \  /  \
+     *       a1  a2 b1  b2
+     * </pre>
+     *
+     * a1 and b2 are using most of resources.
+     * a2 and b1 needs more resources. Both are under served.
+     * hence demand will consider both queue's need while trying to
+     * do preemption.
+     */
+    String labelsConfig =
+        "=100,true;";
+    String nodesConfig =
+        "n1=;"; // n1 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100:100:10 100:100:10 100:100:10 100:100:10]);" + //root
+        "-a(=[50:80:4 100:100:10 80:90:10 30:20:4]);" + // a
+        "--a1(=[25:30:2 100:50:10 80:90:10 0]);" + // a1
+        "--a2(=[25:50:2 100:50:10 0 30:20:4]);" + // a2
+        "-b(=[50:20:6 100:100:10 20:10 40:50:8]);" + // b
+        "--b1(=[25:5:4 100:20:10 0 20:10:4]);" + // b1
+        "--b2(=[25:15:2 100:20:10 20:10 20:10:4])"; // b2
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a1\t" // app1 in a1
+        + "(1,8:9:1,n1,,10,false);" +
+        "b2\t" // app2 in b2
+        + "(1,2:1,n1,,10,false)"; // 80 of y
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(7)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+
+    riMap.remove(RESOURCE_1);
+    ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
+  }
 }


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


[28/50] [abbrv] hadoop git commit: YARN-7544. Use queue-path.capacity/maximum-capacity to specify absolute min/max resources. (Sunil G via wangda)

Posted by wa...@apache.org.
YARN-7544. Use queue-path.capacity/maximum-capacity to specify absolute min/max resources. (Sunil G via wangda)

Change-Id: I685341be213eee500f51e02f01c91def89391c17


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

Branch: refs/heads/YARN-6592
Commit: 1012b901c8eeeb46c7c792c94ed0befca1c860b4
Parents: b7b8cd5
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Nov 23 19:53:26 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../CapacitySchedulerConfiguration.java         | 79 +++++++++++++++-----
 .../TestAbsoluteResourceConfiguration.java      | 30 ++++----
 2 files changed, 74 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1012b901/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index f81e097..e47c8d0 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -329,7 +329,9 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
 
   public static final String DEFAULT_RESOURCE_TYPES = "memory,vcores";
 
-  public static final String PATTERN_FOR_ABSOLUTE_RESOURCE = "\\[([^\\]]+)";
+  public static final String PATTERN_FOR_ABSOLUTE_RESOURCE = "^\\[[\\w\\.,\\-_=\\ /]+\\]$";
+
+  private static final Pattern RESOURCE_PATTERN = Pattern.compile(PATTERN_FOR_ABSOLUTE_RESOURCE);
 
   public enum AbsoluteResourceType {
     MEMORY, VCORES;
@@ -411,14 +413,29 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   }
   
   public float getNonLabeledQueueCapacity(String queue) {
-    float capacity = queue.equals("root") ? 100.0f : getFloat(
-        getQueuePrefix(queue) + CAPACITY, 0f);
-    if (capacity < MINIMUM_CAPACITY_VALUE || capacity > MAXIMUM_CAPACITY_VALUE) {
-      throw new IllegalArgumentException("Illegal " +
-      		"capacity of " + capacity + " for queue " + queue);
+    String configuredCapacity = get(getQueuePrefix(queue) + CAPACITY);
+    boolean matcher = (configuredCapacity != null)
+        && RESOURCE_PATTERN.matcher(configuredCapacity).find();
+    if (matcher) {
+      // Return capacity in percentage as 0 for non-root queues and 100 for
+      // root.From AbstractCSQueue, absolute resource will be parsed and
+      // updated. Once nodes are added/removed in cluster, capacity in
+      // percentage will also be re-calculated.
+      return queue.equals("root") ? 100.0f : 0f;
     }
-    LOG.debug("CSConf - getCapacity: queuePrefix=" + getQueuePrefix(queue) + 
-        ", capacity=" + capacity);
+
+    float capacity = queue.equals("root")
+        ? 100.0f
+        : (configuredCapacity == null)
+            ? 0f
+            : Float.parseFloat(configuredCapacity);
+    if (capacity < MINIMUM_CAPACITY_VALUE
+        || capacity > MAXIMUM_CAPACITY_VALUE) {
+      throw new IllegalArgumentException(
+          "Illegal " + "capacity of " + capacity + " for queue " + queue);
+    }
+    LOG.debug("CSConf - getCapacity: queuePrefix=" + getQueuePrefix(queue)
+        + ", capacity=" + capacity);
     return capacity;
   }
   
@@ -433,10 +450,23 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   }
 
   public float getNonLabeledQueueMaximumCapacity(String queue) {
-    float maxCapacity = getFloat(getQueuePrefix(queue) + MAXIMUM_CAPACITY,
-        MAXIMUM_CAPACITY_VALUE);
-    maxCapacity = (maxCapacity == DEFAULT_MAXIMUM_CAPACITY_VALUE) ? 
-        MAXIMUM_CAPACITY_VALUE : maxCapacity;
+    String configuredCapacity = get(getQueuePrefix(queue) + MAXIMUM_CAPACITY);
+    boolean matcher = (configuredCapacity != null)
+        && RESOURCE_PATTERN.matcher(configuredCapacity).find();
+    if (matcher) {
+      // Return capacity in percentage as 0 for non-root queues and 100 for
+      // root.From AbstractCSQueue, absolute resource will be parsed and
+      // updated. Once nodes are added/removed in cluster, capacity in
+      // percentage will also be re-calculated.
+      return 100.0f;
+    }
+
+    float maxCapacity = (configuredCapacity == null)
+        ? MAXIMUM_CAPACITY_VALUE
+        : Float.parseFloat(configuredCapacity);
+    maxCapacity = (maxCapacity == DEFAULT_MAXIMUM_CAPACITY_VALUE)
+        ? MAXIMUM_CAPACITY_VALUE
+        : maxCapacity;
     return maxCapacity;
   }
   
@@ -590,6 +620,16 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   private float internalGetLabeledQueueCapacity(String queue, String label, String suffix,
       float defaultValue) {
     String capacityPropertyName = getNodeLabelPrefix(queue, label) + suffix;
+    boolean matcher = (capacityPropertyName != null)
+        && RESOURCE_PATTERN.matcher(capacityPropertyName).find();
+    if (matcher) {
+      // Return capacity in percentage as 0 for non-root queues and 100 for
+      // root.From AbstractCSQueue, absolute resource will be parsed and
+      // updated. Once nodes are added/removed in cluster, capacity in
+      // percentage will also be re-calculated.
+      return defaultValue;
+    }
+
     float capacity = getFloat(capacityPropertyName, defaultValue);
     if (capacity < MINIMUM_CAPACITY_VALUE
         || capacity > MAXIMUM_CAPACITY_VALUE) {
@@ -1722,7 +1762,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public Resource getMinimumResourceRequirement(String label, String queue,
       Set<String> resourceTypes) {
     return internalGetLabeledResourceRequirementForQueue(queue, label,
-        resourceTypes, MINIMUM_RESOURCE);
+        resourceTypes, CAPACITY);
   }
 
   /**
@@ -1739,19 +1779,19 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public Resource getMaximumResourceRequirement(String label, String queue,
       Set<String> resourceTypes) {
     return internalGetLabeledResourceRequirementForQueue(queue, label,
-        resourceTypes, MAXIMUM_RESOURCE);
+        resourceTypes, MAXIMUM_CAPACITY);
   }
 
   @VisibleForTesting
   public void setMinimumResourceRequirement(String label, String queue,
       Resource resource) {
-    updateMinMaxResourceToConf(label, queue, resource, MINIMUM_RESOURCE);
+    updateMinMaxResourceToConf(label, queue, resource, CAPACITY);
   }
 
   @VisibleForTesting
   public void setMaximumResourceRequirement(String label, String queue,
       Resource resource) {
-    updateMinMaxResourceToConf(label, queue, resource, MAXIMUM_RESOURCE);
+    updateMinMaxResourceToConf(label, queue, resource, MAXIMUM_CAPACITY);
   }
 
   private void updateMinMaxResourceToConf(String label, String queue,
@@ -1786,8 +1826,8 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
 
     // Define resource here.
     Resource resource = Resource.newInstance(0l, 0);
-    Matcher matcher = Pattern.compile(PATTERN_FOR_ABSOLUTE_RESOURCE)
-        .matcher(resourceString);
+    Matcher matcher = RESOURCE_PATTERN.matcher(resourceString);
+
     /*
      * Absolute resource configuration for a queue will be grouped by "[]".
      * Syntax of absolute resource config could be like below
@@ -1795,11 +1835,12 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
      */
     if (matcher.find()) {
       // Get the sub-group.
-      String subGroup = matcher.group(1);
+      String subGroup = matcher.group(0);
       if (subGroup.trim().isEmpty()) {
         return Resources.none();
       }
 
+      subGroup = subGroup.substring(1, subGroup.length() - 1);
       for (String kvPair : subGroup.trim().split(",")) {
         String[] splits = kvPair.split("=");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1012b901/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.java
index 5a66281..80d16bb 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/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -233,7 +232,12 @@ public class TestAbsoluteResourceConfiguration {
   @Test
   public void testSimpleValidateAbsoluteResourceConfig() throws Exception {
     /**
-     * Queue structure is as follows. root / | \ a b c / \ | a1 a2 b1
+     * Queue structure is as follows.
+     *    root
+     *   / | \
+     *   a b c
+     *   / \ |
+     *  a1 a2 b1
      *
      * Test below cases 1) Configure percentage based capacity and absolute
      * resource together. 2) As per above tree structure, ensure all values
@@ -258,21 +262,15 @@ public class TestAbsoluteResourceConfiguration {
     // Get queue object to verify min/max resource configuration.
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
 
-    // 1. Create a new config with capcity and min/max together. Ensure an
-    // exception is thrown.
+    // 1. Create a new config with min/max.
     CapacitySchedulerConfiguration csConf1 = setupSimpleQueueConfiguration(
         true);
     setupMinMaxResourceConfiguration(csConf1);
 
     try {
       cs.reinitialize(csConf1, rm.getRMContext());
-      Assert.fail();
     } catch (IOException e) {
-      Assert.assertTrue(e instanceof IOException);
-      Assert.assertEquals(
-          "Failed to re-init queues : Queue 'queueA' should use either"
-              + " percentage based capacity configuration or absolute resource.",
-          e.getMessage());
+      Assert.fail();
     }
     rm.stop();
 
@@ -368,7 +366,12 @@ public class TestAbsoluteResourceConfiguration {
   @Test
   public void testComplexValidateAbsoluteResourceConfig() throws Exception {
     /**
-     * Queue structure is as follows. root / | \ a b c / \ | a1 a2 b1
+     * Queue structure is as follows.
+     *   root
+     *  / | \
+     *  a b c
+     * / \ |
+     * a1 a2 b1
      *
      * Test below cases: 1) Parent and its child queues must use either
      * percentage based or absolute resource configuration. 2) Parent's min
@@ -396,11 +399,6 @@ public class TestAbsoluteResourceConfiguration {
     csConf.setCapacity(QUEUEB_FULL, 25f);
     csConf.setCapacity(QUEUEC_FULL, 25f);
 
-    // Also unset resource based config.
-    csConf.setMinimumResourceRequirement("", QUEUEA_FULL, Resources.none());
-    csConf.setMinimumResourceRequirement("", QUEUEB_FULL, Resources.none());
-    csConf.setMinimumResourceRequirement("", QUEUEC_FULL, Resources.none());
-
     // Get queue object to verify min/max resource configuration.
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
     try {


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


[50/50] [abbrv] hadoop git commit: YARN-7522. Introduce AllocationTagsManager to associate allocation tags to nodes. (Wangda Tan via asuresh)

Posted by wa...@apache.org.
YARN-7522. Introduce AllocationTagsManager to associate allocation tags to nodes. (Wangda Tan via asuresh)


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

Branch: refs/heads/YARN-6592
Commit: 0e66d31e21d7fe069df442450331efc44a850c10
Parents: e2b1dc0
Author: Arun Suresh <as...@apache.org>
Authored: Fri Dec 8 00:24:00 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Sun Dec 10 10:27:02 2017 -0800

----------------------------------------------------------------------
 .../resourcemanager/RMActiveServiceContext.java |  15 +
 .../yarn/server/resourcemanager/RMContext.java  |   5 +
 .../server/resourcemanager/RMContextImpl.java   |  12 +
 .../server/resourcemanager/ResourceManager.java |   9 +
 .../constraint/AllocationTagsManager.java       | 431 +++++++++++++++++++
 .../constraint/AllocationTagsNamespaces.java    |  31 ++
 .../InvalidAllocationTagsQueryException.java    |  35 ++
 .../rmcontainer/RMContainer.java                |   8 +
 .../rmcontainer/RMContainerImpl.java            |  21 +
 .../constraint/TestAllocationTagsManager.java   | 328 ++++++++++++++
 .../rmcontainer/TestRMContainerImpl.java        | 124 ++++++
 .../scheduler/capacity/TestUtils.java           |   9 +
 .../scheduler/fifo/TestFifoScheduler.java       |   5 +
 13 files changed, 1033 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.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/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index 9dc5945..6ee3a4c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
@@ -107,6 +108,7 @@ public class RMActiveServiceContext {
 
   private RMAppLifetimeMonitor rmAppLifetimeMonitor;
   private QueueLimitCalculator queueLimitCalculator;
+  private AllocationTagsManager allocationTagsManager;
 
   public RMActiveServiceContext() {
     queuePlacementManager = new PlacementManager();
@@ -398,6 +400,19 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
+  public AllocationTagsManager getAllocationTagsManager() {
+    return allocationTagsManager;
+  }
+
+  @Private
+  @Unstable
+  public void setAllocationTagsManager(
+      AllocationTagsManager allocationTagsManager) {
+    this.allocationTagsManager = allocationTagsManager;
+  }
+
+  @Private
+  @Unstable
   public RMDelegatedNodeLabelsUpdater getRMDelegatedNodeLabelsUpdater() {
     return rmDelegatedNodeLabelsUpdater;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.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/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index ec94030..62899d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
@@ -166,4 +167,8 @@ public interface RMContext extends ApplicationMasterServiceContext {
   void setResourceProfilesManager(ResourceProfilesManager mgr);
 
   String getAppProxyUrl(Configuration conf, ApplicationId applicationId);
+
+  AllocationTagsManager getAllocationTagsManager();
+
+  void setAllocationTagsManager(AllocationTagsManager allocationTagsManager);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.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/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index 80a9109..315fdc1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
@@ -504,6 +505,17 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
+  public AllocationTagsManager getAllocationTagsManager() {
+    return activeServiceContext.getAllocationTagsManager();
+  }
+
+  @Override
+  public void setAllocationTagsManager(
+      AllocationTagsManager allocationTagsManager) {
+    activeServiceContext.setAllocationTagsManager(allocationTagsManager);
+  }
+
+  @Override
   public RMDelegatedNodeLabelsUpdater getRMDelegatedNodeLabelsUpdater() {
     return activeServiceContext.getRMDelegatedNodeLabelsUpdater();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.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/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index a0317f6..8d1000e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV1Pu
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Publisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
@@ -490,6 +491,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
       throws InstantiationException, IllegalAccessException {
     return new RMNodeLabelsManager();
   }
+
+  protected AllocationTagsManager createAllocationTagsManager() {
+    return new AllocationTagsManager();
+  }
   
   protected DelegationTokenRenewer createDelegationTokenRenewer() {
     return new DelegationTokenRenewer();
@@ -609,6 +614,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
       addService(nlm);
       rmContext.setNodeLabelManager(nlm);
 
+      AllocationTagsManager allocationTagsManager =
+          createAllocationTagsManager();
+      rmContext.setAllocationTagsManager(allocationTagsManager);
+
       RMDelegatedNodeLabelsUpdater delegatedNodeLabelsUpdater =
           createRMDelegatedNodeLabelsUpdater();
       if (delegatedNodeLabelsUpdater != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsManager.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/constraint/AllocationTagsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsManager.java
new file mode 100644
index 0000000..b67fab9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsManager.java
@@ -0,0 +1,431 @@
+/*
+ * *
+ *  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.constraint;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.log4j.Logger;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.LongBinaryOperator;
+
+/**
+ * Support storing maps between container-tags/applications and
+ * nodes. This will be required by affinity/anti-affinity implementation and
+ * cardinality.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class AllocationTagsManager {
+
+  private static final Logger LOG = Logger.getLogger(
+      AllocationTagsManager.class);
+
+  private ReentrantReadWriteLock.ReadLock readLock;
+  private ReentrantReadWriteLock.WriteLock writeLock;
+
+  // Application's tags to node
+  private Map<ApplicationId, NodeToCountedTags> perAppMappings =
+      new HashMap<>();
+
+  // Global tags to node mapping (used to fast return aggregated tags
+  // cardinality across apps)
+  private NodeToCountedTags globalMapping = new NodeToCountedTags();
+
+  /**
+   * Store node to counted tags.
+   */
+  @VisibleForTesting
+  static class NodeToCountedTags {
+    // Map<NodeId, Map<Tag, Count>>
+    private Map<NodeId, Map<String, Long>> nodeToTagsWithCount =
+        new HashMap<>();
+
+    // protected by external locks
+    private void addTagsToNode(NodeId nodeId, Set<String> tags) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.computeIfAbsent(nodeId,
+          k -> new HashMap<>());
+
+      for (String tag : tags) {
+        Long count = innerMap.get(tag);
+        if (count == null) {
+          innerMap.put(tag, 1L);
+        } else{
+          innerMap.put(tag, count + 1);
+        }
+      }
+    }
+
+    // protected by external locks
+    private void addTagToNode(NodeId nodeId, String tag) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.computeIfAbsent(nodeId,
+          k -> new HashMap<>());
+
+      Long count = innerMap.get(tag);
+      if (count == null) {
+        innerMap.put(tag, 1L);
+      } else{
+        innerMap.put(tag, count + 1);
+      }
+    }
+
+    private void removeTagFromInnerMap(Map<String, Long> innerMap, String tag) {
+      Long count = innerMap.get(tag);
+      if (count > 1) {
+        innerMap.put(tag, count - 1);
+      } else {
+        if (count <= 0) {
+          LOG.warn(
+              "Trying to remove tags from node, however the count already"
+                  + " becomes 0 or less, it could be a potential bug.");
+        }
+        innerMap.remove(tag);
+      }
+    }
+
+    private void removeTagsFromNode(NodeId nodeId, Set<String> tags) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.get(nodeId);
+      if (innerMap == null) {
+        LOG.warn("Failed to find node=" + nodeId
+            + " while trying to remove tags, please double check.");
+        return;
+      }
+
+      for (String tag : tags) {
+        removeTagFromInnerMap(innerMap, tag);
+      }
+
+      if (innerMap.isEmpty()) {
+        nodeToTagsWithCount.remove(nodeId);
+      }
+    }
+
+    private void removeTagFromNode(NodeId nodeId, String tag) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.get(nodeId);
+      if (innerMap == null) {
+        LOG.warn("Failed to find node=" + nodeId
+            + " while trying to remove tags, please double check.");
+        return;
+      }
+
+      removeTagFromInnerMap(innerMap, tag);
+
+      if (innerMap.isEmpty()) {
+        nodeToTagsWithCount.remove(nodeId);
+      }
+    }
+
+    private long getCardinality(NodeId nodeId, String tag) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.get(nodeId);
+      if (innerMap == null) {
+        return 0;
+      }
+      Long value = innerMap.get(tag);
+      return value == null ? 0 : value;
+    }
+
+    private long getCardinality(NodeId nodeId, Set<String> tags,
+        LongBinaryOperator op) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.get(nodeId);
+      if (innerMap == null) {
+        return 0;
+      }
+
+      long returnValue = 0;
+      boolean firstTag = true;
+
+      if (tags != null && !tags.isEmpty()) {
+        for (String tag : tags) {
+          Long value = innerMap.get(tag);
+          if (value == null) {
+            value = 0L;
+          }
+
+          if (firstTag) {
+            returnValue = value;
+            firstTag = false;
+            continue;
+          }
+
+          returnValue = op.applyAsLong(returnValue, value);
+        }
+      } else {
+        // Similar to above if, but only iterate values for better performance
+        for (long value : innerMap.values()) {
+          // For the first value, we will not apply op
+          if (firstTag) {
+            returnValue = value;
+            firstTag = false;
+            continue;
+          }
+          returnValue = op.applyAsLong(returnValue, value);
+        }
+      }
+      return returnValue;
+    }
+
+    private boolean isEmpty() {
+      return nodeToTagsWithCount.isEmpty();
+    }
+
+    @VisibleForTesting
+    public Map<NodeId, Map<String, Long>> getNodeToTagsWithCount() {
+      return nodeToTagsWithCount;
+    }
+  }
+
+  @VisibleForTesting
+  Map<ApplicationId, NodeToCountedTags> getPerAppMappings() {
+    return perAppMappings;
+  }
+
+  @VisibleForTesting
+  NodeToCountedTags getGlobalMapping() {
+    return globalMapping;
+  }
+
+  public AllocationTagsManager() {
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+  }
+
+  /**
+   * Notify container allocated on a node.
+   *
+   * @param nodeId         allocated node.
+   * @param applicationId  applicationId
+   * @param containerId    container id.
+   * @param allocationTags allocation tags, see
+   *                       {@link SchedulingRequest#getAllocationTags()}
+   *                       application_id will be added to allocationTags.
+   */
+  public void addContainer(NodeId nodeId, ApplicationId applicationId,
+      ContainerId containerId, Set<String> allocationTags) {
+    String applicationIdTag =
+        AllocationTagsNamespaces.APP_ID + applicationId.toString();
+
+    boolean useSet = false;
+    if (allocationTags != null && !allocationTags.isEmpty()) {
+      // Copy before edit it.
+      allocationTags = new HashSet<>(allocationTags);
+      allocationTags.add(applicationIdTag);
+      useSet = true;
+    }
+
+    writeLock.lock();
+    try {
+      NodeToCountedTags perAppTagsMapping = perAppMappings.computeIfAbsent(
+          applicationId, k -> new NodeToCountedTags());
+
+      if (useSet) {
+        perAppTagsMapping.addTagsToNode(nodeId, allocationTags);
+        globalMapping.addTagsToNode(nodeId, allocationTags);
+      } else {
+        perAppTagsMapping.addTagToNode(nodeId, applicationIdTag);
+        globalMapping.addTagToNode(nodeId, applicationIdTag);
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Added container=" + containerId + " with tags=[" + StringUtils
+                .join(allocationTags, ",") + "]");
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Notify container removed.
+   *
+   * @param nodeId         nodeId
+   * @param applicationId  applicationId
+   * @param containerId    containerId.
+   * @param allocationTags allocation tags for given container
+   */
+  public void removeContainer(NodeId nodeId, ApplicationId applicationId,
+      ContainerId containerId, Set<String> allocationTags) {
+    String applicationIdTag =
+        AllocationTagsNamespaces.APP_ID + applicationId.toString();
+    boolean useSet = false;
+
+    if (allocationTags != null && !allocationTags.isEmpty()) {
+      // Copy before edit it.
+      allocationTags = new HashSet<>(allocationTags);
+      allocationTags.add(applicationIdTag);
+      useSet = true;
+    }
+
+    writeLock.lock();
+    try {
+      NodeToCountedTags perAppTagsMapping = perAppMappings.get(applicationId);
+      if (perAppTagsMapping == null) {
+        return;
+      }
+
+      if (useSet) {
+        perAppTagsMapping.removeTagsFromNode(nodeId, allocationTags);
+        globalMapping.removeTagsFromNode(nodeId, allocationTags);
+      } else {
+        perAppTagsMapping.removeTagFromNode(nodeId, applicationIdTag);
+        globalMapping.removeTagFromNode(nodeId, applicationIdTag);
+      }
+
+      if (perAppTagsMapping.isEmpty()) {
+        perAppMappings.remove(applicationId);
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Removed container=" + containerId + " with tags=[" + StringUtils
+                .join(allocationTags, ",") + "]");
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Get cardinality for following conditions. External can pass-in a binary op
+   * to implement customized logic.   *
+   * @param nodeId        nodeId, required.
+   * @param applicationId applicationId. When null is specified, return
+   *                      aggregated cardinality among all nodes.
+   * @param tag           allocation tag, see
+   *                      {@link SchedulingRequest#getAllocationTags()},
+   *                      When multiple tags specified. Returns cardinality
+   *                      depends on op. If a specified tag doesn't exist,
+   *                      0 will be its cardinality.
+   *                      When null/empty tags specified, all tags
+   *                      (of the node/app) will be considered.
+   * @return cardinality of specified query on the node.
+   * @throws InvalidAllocationTagsQueryException when illegal query
+   *                                            parameter specified
+   */
+  public long getNodeCardinality(NodeId nodeId, ApplicationId applicationId,
+      String tag) throws InvalidAllocationTagsQueryException {
+    readLock.lock();
+
+    try {
+      if (nodeId == null) {
+        throw new InvalidAllocationTagsQueryException(
+            "Must specify nodeId/tags/op to query cardinality");
+      }
+
+      NodeToCountedTags mapping;
+      if (applicationId != null) {
+        mapping = perAppMappings.get(applicationId);
+      } else{
+        mapping = globalMapping;
+      }
+
+      if (mapping == null) {
+        return 0;
+      }
+
+      return mapping.getCardinality(nodeId, tag);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Check if given tag exists on node.
+   *
+   * @param nodeId        nodeId, required.
+   * @param applicationId applicationId. When null is specified, return
+   *                      aggregated cardinality among all nodes.
+   * @param tag           allocation tag, see
+   *                      {@link SchedulingRequest#getAllocationTags()},
+   *                      When multiple tags specified. Returns cardinality
+   *                      depends on op. If a specified tag doesn't exist,
+   *                      0 will be its cardinality.
+   *                      When null/empty tags specified, all tags
+   *                      (of the node/app) will be considered.
+   * @return cardinality of specified query on the node.
+   * @throws InvalidAllocationTagsQueryException when illegal query
+   *                                            parameter specified
+   */
+  public boolean allocationTagExistsOnNode(NodeId nodeId,
+      ApplicationId applicationId, String tag)
+      throws InvalidAllocationTagsQueryException {
+    return getNodeCardinality(nodeId, applicationId, tag) > 0;
+  }
+
+  /**
+   * Get cardinality for following conditions. External can pass-in a binary op
+   * to implement customized logic.
+   *
+   * @param nodeId        nodeId, required.
+   * @param applicationId applicationId. When null is specified, return
+   *                      aggregated cardinality among all nodes.
+   * @param tags          allocation tags, see
+   *                      {@link SchedulingRequest#getAllocationTags()},
+   *                      When multiple tags specified. Returns cardinality
+   *                      depends on op. If a specified tag doesn't exist, 0
+   *                      will be its cardinality. When null/empty tags
+   *                      specified, all tags (of the node/app) will be
+   *                      considered.
+   * @param op            operator. Such as Long::max, Long::sum, etc. Required.
+   *                      This sparameter only take effect when #values >= 2.
+   * @return cardinality of specified query on the node.
+   * @throws InvalidAllocationTagsQueryException when illegal query
+   *                                            parameter specified
+   */
+  public long getNodeCardinalityByOp(NodeId nodeId, ApplicationId applicationId,
+      Set<String> tags, LongBinaryOperator op)
+      throws InvalidAllocationTagsQueryException {
+    readLock.lock();
+
+    try {
+      if (nodeId == null || op == null) {
+        throw new InvalidAllocationTagsQueryException(
+            "Must specify nodeId/tags/op to query cardinality");
+      }
+
+      NodeToCountedTags mapping;
+      if (applicationId != null) {
+        mapping = perAppMappings.get(applicationId);
+      } else{
+        mapping = globalMapping;
+      }
+
+      if (mapping == null) {
+        return 0;
+      }
+
+      return mapping.getCardinality(nodeId, tags, op);
+    } finally {
+      readLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsNamespaces.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/constraint/AllocationTagsNamespaces.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsNamespaces.java
new file mode 100644
index 0000000..893ff1c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsNamespaces.java
@@ -0,0 +1,31 @@
+/*
+ * *
+ *  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.constraint;
+
+/**
+ * Predefined namespaces for tags
+ *
+ * Same as namespace  of resource types. Namespaces of placement tags are start
+ * with alphabets and ended with "/"
+ */
+public class AllocationTagsNamespaces {
+  public static final String APP_ID = "yarn_app_id/";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/InvalidAllocationTagsQueryException.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/constraint/InvalidAllocationTagsQueryException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/InvalidAllocationTagsQueryException.java
new file mode 100644
index 0000000..5519e39
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/InvalidAllocationTagsQueryException.java
@@ -0,0 +1,35 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.constraint;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * Exception when invalid parameter specified to do placement tags related
+ * queries.
+ */
+public class InvalidAllocationTagsQueryException extends YarnException {
+  private static final long serialVersionUID = 12312831974894L;
+
+  public InvalidAllocationTagsQueryException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.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/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
index f3cbf63..8f751b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
 
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -30,6 +31,7 @@ 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.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
@@ -115,4 +117,10 @@ public interface RMContainer extends EventHandler<RMContainerEvent>,
   boolean completed();
 
   NodeId getNodeId();
+
+  /**
+   * Return {@link SchedulingRequest#getAllocationTags()} specified by AM.
+   * @return allocation tags, could be null/empty
+   */
+  Set<String> getAllocationTags();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.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/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index e26689e..184cdfc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -189,6 +190,9 @@ public class RMContainerImpl implements RMContainer {
   private boolean isExternallyAllocated;
   private SchedulerRequestKey allocatedSchedulerKey;
 
+  // TODO, set it when container allocated by scheduler (From SchedulingRequest)
+  private Set<String> allocationTags = null;
+
   public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       RMContext rmContext) {
@@ -501,6 +505,11 @@ public class RMContainerImpl implements RMContainer {
     return nodeId;
   }
 
+  @Override
+  public Set<String> getAllocationTags() {
+    return allocationTags;
+  }
+
   private static class BaseTransition implements
       SingleArcTransition<RMContainerImpl, RMContainerEvent> {
 
@@ -565,6 +574,12 @@ public class RMContainerImpl implements RMContainer {
 
     @Override
     public void transition(RMContainerImpl container, RMContainerEvent event) {
+      // Notify placementManager
+      container.rmContext.getAllocationTagsManager().addContainer(
+          container.getNodeId(),
+          container.getApplicationAttemptId().getApplicationId(),
+          container.getContainerId(), container.getAllocationTags());
+
       container.eventHandler.handle(new RMAppAttemptEvent(
           container.appAttemptId, RMAppAttemptEventType.CONTAINER_ALLOCATED));
     }
@@ -676,6 +691,12 @@ public class RMContainerImpl implements RMContainer {
 
     @Override
     public void transition(RMContainerImpl container, RMContainerEvent event) {
+      // Notify placementManager
+      container.rmContext.getAllocationTagsManager().removeContainer(
+          container.getNodeId(),
+          container.getApplicationAttemptId().getApplicationId(),
+          container.getContainerId(), container.getAllocationTags());
+
       RMContainerFinishedEvent finishedEvent = (RMContainerFinishedEvent) event;
 
       container.finishTime = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/TestAllocationTagsManager.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/constraint/TestAllocationTagsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/TestAllocationTagsManager.java
new file mode 100644
index 0000000..0358792
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/TestAllocationTagsManager.java
@@ -0,0 +1,328 @@
+/*
+ * *
+ *  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.constraint;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test functionality of AllocationTagsManager.
+ */
+public class TestAllocationTagsManager {
+  @Test
+  public void testAllocationTagsManagerSimpleCases()
+      throws InvalidAllocationTagsQueryException {
+    AllocationTagsManager atm = new AllocationTagsManager();
+
+    /**
+     * Construct test case:
+     * Node1:
+     *    container_1_1 (mapper/reducer/app_1)
+     *    container_1_3 (service/app_1)
+     *
+     * Node2:
+     *    container_1_2 (mapper/reducer/app_1)
+     *    container_1_4 (reducer/app_1)
+     *    container_2_1 (service/app_2)
+     */
+
+    // 3 Containers from app1
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    // 1 Container from app2
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // Get Cardinality of app1 on node1, with tag "mapper"
+    Assert.assertEquals(1,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node1:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of("mapper"),
+            Long::max));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=min
+    Assert.assertEquals(1,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::min));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=max
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::max));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=sum
+    Assert.assertEquals(3,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::sum));
+
+    // Get Cardinality by passing single tag.
+    Assert.assertEquals(1,
+        atm.getNodeCardinality(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), "mapper"));
+
+    Assert.assertEquals(2,
+        atm.getNodeCardinality(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), "reducer"));
+
+    // Get Cardinality of app1 on node2, with tag "no_existed/reducer", op=min
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("no_existed", "reducer"), Long::min));
+
+    // Get Cardinality of app1 on node2, with tag "<applicationId>", op=max
+    // (Expect this returns #containers from app1 on node2)
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet
+                .of(AllocationTagsNamespaces.APP_ID + TestUtils
+                    .getMockApplicationId(1).toString()), Long::max));
+
+    // Get Cardinality of app1 on node2, with empty tag set, op=max
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::max));
+
+    // Get Cardinality of all apps on node2, with empty tag set, op=sum
+    Assert.assertEquals(7,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"), null,
+            ImmutableSet.of(), Long::sum));
+
+    // Get Cardinality of app_1 on node2, with empty tag set, op=sum
+    Assert.assertEquals(5,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::sum));
+
+    // Get Cardinality of app_1 on node2, with empty tag set, op=sum
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(2), ImmutableSet.of(), Long::sum));
+
+    // Finish all containers:
+    atm.removeContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.removeContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // Expect all cardinality to be 0
+    // Get Cardinality of app1 on node1, with tag "mapper"
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node1:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of("mapper"),
+            Long::max));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=min
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::min));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=max
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::max));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=sum
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::sum));
+
+    // Get Cardinality of app1 on node2, with tag "<applicationId>", op=max
+    // (Expect this returns #containers from app1 on node2)
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of(TestUtils.getMockApplicationId(1).toString()),
+            Long::max));
+
+    Assert.assertEquals(0,
+        atm.getNodeCardinality(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            TestUtils.getMockApplicationId(1).toString()));
+
+    // Get Cardinality of app1 on node2, with empty tag set, op=max
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::max));
+
+    // Get Cardinality of all apps on node2, with empty tag set, op=sum
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"), null,
+            ImmutableSet.of(), Long::sum));
+
+    // Get Cardinality of app_1 on node2, with empty tag set, op=sum
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::sum));
+
+    // Get Cardinality of app_1 on node2, with empty tag set, op=sum
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(2), ImmutableSet.of(), Long::sum));
+  }
+
+  @Test
+  public void testAllocationTagsManagerMemoryAfterCleanup()
+      throws InvalidAllocationTagsQueryException {
+    /**
+     * Make sure YARN cleans up all memory once container/app finishes.
+     */
+
+    AllocationTagsManager atm = new AllocationTagsManager();
+
+    // Add a bunch of containers
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // Remove all these containers
+    atm.removeContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.removeContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // Check internal data structure
+    Assert.assertEquals(0,
+        atm.getGlobalMapping().getNodeToTagsWithCount().size());
+    Assert.assertEquals(0, atm.getPerAppMappings().size());
+  }
+
+  @Test
+  public void testQueryCardinalityWithIllegalParameters()
+      throws InvalidAllocationTagsQueryException {
+    /**
+     * Make sure YARN cleans up all memory once container/app finishes.
+     */
+
+    AllocationTagsManager atm = new AllocationTagsManager();
+
+    // Add a bunch of containers
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // No node-id
+    boolean caughtException = false;
+    try {
+      atm.getNodeCardinalityByOp(null, TestUtils.getMockApplicationId(2),
+          ImmutableSet.of("mapper"), Long::min);
+    } catch (InvalidAllocationTagsQueryException e) {
+      caughtException = true;
+    }
+    Assert.assertTrue("should fail because of nodeId specified",
+        caughtException);
+
+    // No op
+    caughtException = false;
+    try {
+      atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+          TestUtils.getMockApplicationId(2), ImmutableSet.of("mapper"), null);
+    } catch (InvalidAllocationTagsQueryException e) {
+      caughtException = true;
+    }
+    Assert.assertTrue("should fail because of nodeId specified",
+        caughtException);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.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/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
index 6c189b3..27ff311 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
@@ -109,6 +110,8 @@ public class TestRMContainerImpl {
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getRMApps()).thenReturn(rmApps);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
+    when(rmContext.getAllocationTagsManager()).thenReturn(ptm);
     YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(
         YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
@@ -209,6 +212,8 @@ public class TestRMContainerImpl {
     when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer);
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
+    when(rmContext.getAllocationTagsManager()).thenReturn(ptm);
 
     YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(
@@ -367,4 +372,123 @@ public class TestRMContainerImpl {
     verify(publisher, times(1)).containerCreated(any(RMContainer.class), anyLong());
     verify(publisher, times(1)).containerFinished(any(RMContainer.class), anyLong());
   }
+
+  @Test
+  public void testContainerTransitionNotifyPlacementTagsManager()
+      throws Exception {
+    DrainDispatcher drainDispatcher = new DrainDispatcher();
+    EventHandler<RMAppAttemptEvent> appAttemptEventHandler = mock(
+        EventHandler.class);
+    EventHandler generic = mock(EventHandler.class);
+    drainDispatcher.register(RMAppAttemptEventType.class,
+        appAttemptEventHandler);
+    drainDispatcher.register(RMNodeEventType.class, generic);
+    drainDispatcher.init(new YarnConfiguration());
+    drainDispatcher.start();
+    NodeId nodeId = BuilderUtils.newNodeId("host", 3425);
+    ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
+    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
+        appId, 1);
+    ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
+    ContainerAllocationExpirer expirer = mock(ContainerAllocationExpirer.class);
+
+    Resource resource = BuilderUtils.newResource(512, 1);
+    Priority priority = BuilderUtils.newPriority(5);
+
+    Container container = BuilderUtils.newContainer(containerId, nodeId,
+        "host:3465", resource, priority, null);
+    ConcurrentMap<ApplicationId, RMApp> rmApps =
+        spy(new ConcurrentHashMap<ApplicationId, RMApp>());
+    RMApp rmApp = mock(RMApp.class);
+    when(rmApp.getRMAppAttempt(Matchers.any())).thenReturn(null);
+    Mockito.doReturn(rmApp).when(rmApps).get(Matchers.any());
+
+    RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
+    SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
+    AllocationTagsManager tagsManager = new AllocationTagsManager();
+    RMContext rmContext = mock(RMContext.class);
+    when(rmContext.getDispatcher()).thenReturn(drainDispatcher);
+    when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer);
+    when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
+    when(rmContext.getRMApps()).thenReturn(rmApps);
+    when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    when(rmContext.getAllocationTagsManager()).thenReturn(tagsManager);
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(
+        YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
+        true);
+    when(rmContext.getYarnConfiguration()).thenReturn(conf);
+
+    /* First container: ALLOCATED -> KILLED */
+    RMContainer rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
+        nodeId, "user", rmContext);
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.START));
+
+    Assert.assertEquals(1,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(new RMContainerFinishedEvent(containerId, ContainerStatus
+        .newInstance(containerId, ContainerState.COMPLETE, "", 0),
+        RMContainerEventType.KILL));
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    /* Second container: ACQUIRED -> FINISHED */
+    rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
+        nodeId, "user", rmContext);
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.START));
+
+    Assert.assertEquals(1,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(
+        new RMContainerEvent(containerId, RMContainerEventType.ACQUIRED));
+
+    rmContainer.handle(new RMContainerFinishedEvent(containerId, ContainerStatus
+        .newInstance(containerId, ContainerState.COMPLETE, "", 0),
+        RMContainerEventType.FINISHED));
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    /* Third container: RUNNING -> FINISHED */
+    rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
+        nodeId, "user", rmContext);
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.START));
+
+    Assert.assertEquals(1,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(
+        new RMContainerEvent(containerId, RMContainerEventType.ACQUIRED));
+
+    rmContainer.handle(
+        new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED));
+
+    rmContainer.handle(new RMContainerFinishedEvent(containerId, ContainerStatus
+        .newInstance(containerId, ContainerState.COMPLETE, "", 0),
+        RMContainerEventType.FINISHED));
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.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/TestUtils.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/TestUtils.java
index 8036a40..2b04d40 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/TestUtils.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/TestUtils.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -135,6 +136,9 @@ public class TestUtils {
         new DefaultResourceCalculator());
     rmContext.setScheduler(mockScheduler);
 
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
+    rmContext.setAllocationTagsManager(ptm);
+
     return rmContext;
   }
   
@@ -234,6 +238,11 @@ public class TestUtils {
     doReturn(id).when(containerId).getContainerId();
     return containerId;
   }
+
+  public static ContainerId getMockContainerId(int appId, int containerId) {
+    ApplicationAttemptId attemptId = getMockApplicationAttemptId(appId, 1);
+    return ContainerId.newContainerId(attemptId, containerId);
+  }
   
   public static Container getMockContainer(
       ContainerId containerId, NodeId nodeId, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e66d31e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.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/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index 3f97b59..4b902a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -234,6 +235,8 @@ public class TestFifoScheduler {
     FifoScheduler scheduler = new FifoScheduler();
     RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
         null, containerTokenSecretManager, nmTokenSecretManager, null, scheduler);
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
+    rmContext.setAllocationTagsManager(ptm);
     rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class));
     rmContext.setRMApplicationHistoryWriter(
         mock(RMApplicationHistoryWriter.class));
@@ -312,12 +315,14 @@ public class TestFifoScheduler {
     FifoScheduler scheduler = new FifoScheduler();
     RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
         null, containerTokenSecretManager, nmTokenSecretManager, null, scheduler);
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
     rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class));
     rmContext.setRMApplicationHistoryWriter(mock(RMApplicationHistoryWriter.class));
     ((RMContextImpl) rmContext).setYarnConfiguration(new YarnConfiguration());
     NullRMNodeLabelsManager nlm = new NullRMNodeLabelsManager();
     nlm.init(new Configuration());
     rmContext.setNodeLabelManager(nlm);
+    rmContext.setAllocationTagsManager(ptm);
 
     scheduler.setRMContext(rmContext);
     ((RMContextImpl) rmContext).setScheduler(scheduler);


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


[18/50] [abbrv] hadoop git commit: HDFS-11915. Sync rbw dir on the first hsync() to avoid file lost on power failure. Contributed by Vinayakumar B.

Posted by wa...@apache.org.
HDFS-11915. Sync rbw dir on the first hsync() to avoid file lost on power failure. Contributed by Vinayakumar B.


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

Branch: refs/heads/YARN-6592
Commit: d6c31a3e6b60c4b8af9ae4661f16614805654e59
Parents: acb9290
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Thu Dec 7 14:50:30 2017 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Thu Dec 7 14:50:30 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java   | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6c31a3e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index c052d52..afb9e2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -126,6 +126,7 @@ class BlockReceiver implements Closeable {
 
   private boolean syncOnClose;
   private volatile boolean dirSyncOnFinalize;
+  private boolean dirSyncOnHSyncDone = false;
   private long restartBudget;
   /** the reference of the volume where the block receiver writes to */
   private ReplicaHandler replicaHandler;
@@ -424,6 +425,10 @@ class BlockReceiver implements Closeable {
       }
       flushTotalNanos += flushEndNanos - flushStartNanos;
     }
+    if (isSync && !dirSyncOnHSyncDone && replicaInfo instanceof LocalReplica) {
+      ((LocalReplica) replicaInfo).fsyncDirectory();
+      dirSyncOnHSyncDone = true;
+    }
     if (checksumOut != null || streams.getDataOut() != null) {
       datanode.metrics.addFlushNanos(flushTotalNanos);
       if (isSync) {


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


[39/50] [abbrv] hadoop git commit: YARN-7473. Implement Framework and policy for capacity management of auto created queues. (Suma Shivaprasad via wangda)

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ManagedParentQueue.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/ManagedParentQueue.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/ManagedParentQueue.java
index ff795e4..cbdb21d 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/ManagedParentQueue.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/ManagedParentQueue.java
@@ -17,13 +17,23 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler
     .SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
+    .FiCaSchedulerApp;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Auto Creation enabled Parent queue. This queue initially does not have any
@@ -44,54 +54,125 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
       final String queueName, final CSQueue parent, final CSQueue old)
       throws IOException {
     super(cs, queueName, parent, old);
-    String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(
-        csContext.getConfiguration());
-    this.leafQueueTemplate = initializeLeafQueueConfigs(
-        leafQueueTemplateConfPrefix).build();
+
+    shouldFailAutoCreationWhenGuaranteedCapacityExceeded =
+        csContext.getConfiguration()
+            .getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
+                getQueuePath());
+
+    leafQueueTemplate = initializeLeafQueueConfigs().build();
 
     StringBuffer queueInfo = new StringBuffer();
     queueInfo.append("Created Managed Parent Queue: ").append(queueName).append(
         "]\nwith capacity: [").append(super.getCapacity()).append(
         "]\nwith max capacity: [").append(super.getMaximumCapacity()).append(
-        "\nwith max apps: [").append(leafQueueTemplate.getMaxApps()).append(
-        "]\nwith max apps per user: [").append(
-        leafQueueTemplate.getMaxAppsPerUser()).append("]\nwith user limit: [")
-        .append(leafQueueTemplate.getUserLimit()).append(
-        "]\nwith user limit factor: [").append(
-        leafQueueTemplate.getUserLimitFactor()).append("].");
+        "].");
     LOG.info(queueInfo.toString());
+
+    initializeQueueManagementPolicy();
   }
 
   @Override
   public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
       throws IOException {
-    validate(newlyParsedQueue);
-    super.reinitialize(newlyParsedQueue, clusterResource);
-    String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(
-        csContext.getConfiguration());
-    this.leafQueueTemplate = initializeLeafQueueConfigs(
-        leafQueueTemplateConfPrefix).build();
+
+    try {
+      writeLock.lock();
+      validate(newlyParsedQueue);
+
+      shouldFailAutoCreationWhenGuaranteedCapacityExceeded =
+          csContext.getConfiguration()
+              .getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
+                  getQueuePath());
+
+      //validate if capacity is exceeded for child queues
+      if (shouldFailAutoCreationWhenGuaranteedCapacityExceeded) {
+        float childCap = sumOfChildCapacities();
+        if (getCapacity() < childCap) {
+          throw new IOException(
+              "Total of Auto Created leaf queues guaranteed capacity : "
+                  + childCap + " exceeds Parent queue's " + getQueuePath()
+                  + " guaranteed capacity " + getCapacity() + ""
+                  + ".Cannot enforce policy to auto"
+                  + " create queues beyond parent queue's capacity");
+        }
+      }
+
+      leafQueueTemplate = initializeLeafQueueConfigs().build();
+
+      super.reinitialize(newlyParsedQueue, clusterResource);
+
+      // run reinitialize on each existing queue, to trigger absolute cap
+      // recomputations
+      for (CSQueue res : this.getChildQueues()) {
+        res.reinitialize(res, clusterResource);
+      }
+
+      //clear state in policy
+      reinitializeQueueManagementPolicy();
+
+      //reassign capacities according to policy
+      final List<QueueManagementChange> queueManagementChanges =
+          queueManagementPolicy.computeQueueManagementChanges();
+
+      validateAndApplyQueueManagementChanges(queueManagementChanges);
+
+      StringBuffer queueInfo = new StringBuffer();
+      queueInfo.append("Reinitialized Managed Parent Queue: ").append(queueName)
+          .append("]\nwith capacity: [").append(super.getCapacity()).append(
+          "]\nwith max capacity: [").append(super.getMaximumCapacity()).append(
+          "].");
+      LOG.info(queueInfo.toString());
+    } catch (YarnException ye) {
+      LOG.error("Exception while computing policy changes for leaf queue : "
+          + getQueueName(), ye);
+      throw new IOException(ye);
+    } finally {
+      writeLock.unlock();
+    }
   }
 
-  @Override
-  protected AutoCreatedLeafQueueTemplate.Builder initializeLeafQueueConfigs(
-      String queuePath) {
+  private void initializeQueueManagementPolicy() {
+    queueManagementPolicy =
+        csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass(
+            getQueuePath());
+
+    queueManagementPolicy.init(csContext, this);
+  }
+
+  private void reinitializeQueueManagementPolicy() {
+    AutoCreatedQueueManagementPolicy managementPolicy =
+        csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass(
+            getQueuePath());
+
+    if (!(managementPolicy.getClass().equals(
+        this.queueManagementPolicy.getClass()))) {
+      queueManagementPolicy = managementPolicy;
+      queueManagementPolicy.init(csContext, this);
+    } else{
+      queueManagementPolicy.reinitialize(csContext, this);
+    }
+  }
+
+  protected AutoCreatedLeafQueueConfig.Builder initializeLeafQueueConfigs() {
 
-    AutoCreatedLeafQueueTemplate.Builder leafQueueTemplate =
-        super.initializeLeafQueueConfigs(queuePath);
+    AutoCreatedLeafQueueConfig.Builder builder =
+        new AutoCreatedLeafQueueConfig.Builder();
 
-    CapacitySchedulerConfiguration conf = csContext.getConfiguration();
-    String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(conf);
+    String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(
+        csContext.getConfiguration());
+    //Load template configuration
+    builder.configuration(
+        super.initializeLeafQueueConfigs(leafQueueTemplateConfPrefix));
+
+    //Load template capacities
     QueueCapacities queueCapacities = new QueueCapacities(false);
-    CSQueueUtils.loadUpdateAndCheckCapacities(leafQueueTemplateConfPrefix,
+    CSQueueUtils.loadUpdateAndCheckCapacities(csContext.getConfiguration()
+            .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()),
         csContext.getConfiguration(), queueCapacities, getQueueCapacities());
-    leafQueueTemplate.capacities(queueCapacities);
-
-    shouldFailAutoCreationWhenGuaranteedCapacityExceeded =
-        conf.getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
-            getQueuePath());
+    builder.capacities(queueCapacities);
 
-    return leafQueueTemplate;
+    return builder;
   }
 
   protected void validate(final CSQueue newlyParsedQueue) throws IOException {
@@ -106,7 +187,7 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
 
   @Override
   public void addChildQueue(CSQueue childQueue)
-      throws SchedulerDynamicEditException {
+      throws SchedulerDynamicEditException, IOException {
     try {
       writeLock.lock();
 
@@ -138,21 +219,164 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
           throw new SchedulerDynamicEditException(
               "Cannot auto create leaf queue " + leafQueueName + ". Child "
                   + "queues capacities have reached parent queue : "
-                  + parentQueue.getQueuePath() + " guaranteed capacity");
+                  + parentQueue.getQueuePath() + "'s guaranteed capacity");
         }
       }
 
       AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue;
       super.addChildQueue(leafQueue);
-      //TODO - refresh policy queue after capacity management is added
+      final AutoCreatedLeafQueueConfig initialLeafQueueTemplate =
+          queueManagementPolicy.getInitialLeafQueueConfiguration(leafQueue);
 
+      leafQueue.reinitializeFromTemplate(initialLeafQueueTemplate);
     } finally {
       writeLock.unlock();
     }
   }
 
-  private String getLeafQueueConfigPrefix(CapacitySchedulerConfiguration conf) {
-    return conf.getAutoCreatedQueueTemplateConfPrefix(getQueuePath());
+  public List<FiCaSchedulerApp> getScheduleableApplications() {
+    try {
+      readLock.lock();
+      List<FiCaSchedulerApp> apps = new ArrayList<>();
+      for (CSQueue childQueue : getChildQueues()) {
+        apps.addAll(((LeafQueue) childQueue).getApplications());
+      }
+      return Collections.unmodifiableList(apps);
+    } finally {
+      readLock.unlock();
+    }
   }
 
+  public List<FiCaSchedulerApp> getPendingApplications() {
+    try {
+      readLock.lock();
+      List<FiCaSchedulerApp> apps = new ArrayList<>();
+      for (CSQueue childQueue : getChildQueues()) {
+        apps.addAll(((LeafQueue) childQueue).getPendingApplications());
+      }
+      return Collections.unmodifiableList(apps);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public List<FiCaSchedulerApp> getAllApplications() {
+    try {
+      readLock.lock();
+      List<FiCaSchedulerApp> apps = new ArrayList<>();
+      for (CSQueue childQueue : getChildQueues()) {
+        apps.addAll(((LeafQueue) childQueue).getAllApplications());
+      }
+      return Collections.unmodifiableList(apps);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public String getLeafQueueConfigPrefix(CapacitySchedulerConfiguration conf) {
+    return CapacitySchedulerConfiguration.PREFIX + conf
+        .getAutoCreatedQueueTemplateConfPrefix(getQueuePath());
+  }
+
+  public boolean shouldFailAutoCreationWhenGuaranteedCapacityExceeded() {
+    return shouldFailAutoCreationWhenGuaranteedCapacityExceeded;
+  }
+
+  /**
+   * Asynchronously called from scheduler to apply queue management changes
+   *
+   * @param queueManagementChanges
+   */
+  public void validateAndApplyQueueManagementChanges(
+      List<QueueManagementChange> queueManagementChanges)
+      throws IOException, SchedulerDynamicEditException {
+    try {
+      writeLock.lock();
+
+      validateQueueManagementChanges(queueManagementChanges);
+
+      applyQueueManagementChanges(queueManagementChanges);
+
+      AutoCreatedQueueManagementPolicy policy =
+          getAutoCreatedQueueManagementPolicy();
+
+      //acquires write lock on policy
+      policy.commitQueueManagementChanges(queueManagementChanges);
+
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void validateQueueManagementChanges(
+      List<QueueManagementChange> queueManagementChanges)
+      throws SchedulerDynamicEditException {
+
+    for (QueueManagementChange queueManagementChange : queueManagementChanges) {
+
+      CSQueue childQueue = queueManagementChange.getQueue();
+
+      if (!(childQueue instanceof AutoCreatedLeafQueue)) {
+        throw new SchedulerDynamicEditException(
+            "queue should be " + "AutoCreatedLeafQueue. Found " + childQueue
+                .getClass());
+      }
+
+      if (!(AbstractManagedParentQueue.class.
+          isAssignableFrom(childQueue.getParent().getClass()))) {
+        LOG.error("Queue " + getQueueName()
+            + " is not an instance of PlanQueue or ManagedParentQueue." + " "
+            + "Ignoring update " + queueManagementChanges);
+        throw new SchedulerDynamicEditException(
+            "Queue " + getQueueName() + " is not a AutoEnabledParentQueue."
+                + " Ignoring update " + queueManagementChanges);
+      }
+
+      switch (queueManagementChange.getQueueAction()){
+      case UPDATE_QUEUE:
+        AutoCreatedLeafQueueConfig template =
+            queueManagementChange.getUpdatedQueueTemplate();
+        ((AutoCreatedLeafQueue) childQueue).validateConfigurations(template);
+        break;
+      }
+    }
+  }
+
+  private void applyQueueManagementChanges(
+      List<QueueManagementChange> queueManagementChanges)
+      throws SchedulerDynamicEditException, IOException {
+    for (QueueManagementChange queueManagementChange : queueManagementChanges) {
+      switch (queueManagementChange.getQueueAction()){
+      case UPDATE_QUEUE:
+        AutoCreatedLeafQueue childQueueToBeUpdated =
+            (AutoCreatedLeafQueue) queueManagementChange.getQueue();
+        //acquires write lock on leaf queue
+        childQueueToBeUpdated.reinitializeFromTemplate(
+            queueManagementChange.getUpdatedQueueTemplate());
+        break;
+      }
+    }
+  }
+
+  public CapacitySchedulerConfiguration getLeafQueueConfigs(
+      String leafQueueName) {
+    return getLeafQueueConfigs(getLeafQueueTemplate().getLeafQueueConfigs(),
+        leafQueueName);
+  }
+
+  public CapacitySchedulerConfiguration getLeafQueueConfigs(
+      CapacitySchedulerConfiguration templateConfig, String leafQueueName) {
+    CapacitySchedulerConfiguration leafQueueConfigTemplate = new
+        CapacitySchedulerConfiguration(new Configuration(false), false);
+    for (final Iterator<Map.Entry<String, String>> iterator =
+         templateConfig.iterator(); iterator.hasNext(); ) {
+      Map.Entry<String, String> confKeyValuePair = iterator.next();
+      final String name = confKeyValuePair.getKey().replaceFirst(
+          CapacitySchedulerConfiguration
+              .AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX,
+          leafQueueName);
+      leafQueueConfigTemplate.set(name, confKeyValuePair.getValue());
+    }
+    return leafQueueConfigTemplate;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.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/PlanQueue.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/PlanQueue.java
index b7f8aa6..757002f 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/PlanQueue.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/PlanQueue.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,61 +36,132 @@ public class PlanQueue extends AbstractManagedParentQueue {
 
   private static final Logger LOG = LoggerFactory.getLogger(PlanQueue.class);
 
+  private int maxAppsForReservation;
+  private int maxAppsPerUserForReservation;
+  private int userLimit;
+  private float userLimitFactor;
+  protected CapacitySchedulerContext schedulerContext;
   private boolean showReservationsAsQueues;
 
   public PlanQueue(CapacitySchedulerContext cs, String queueName,
       CSQueue parent, CSQueue old) throws IOException {
     super(cs, queueName, parent, old);
-    this.leafQueueTemplate = initializeLeafQueueConfigs(getQueuePath()).build();
+
+    this.schedulerContext = cs;
+    // Set the reservation queue attributes for the Plan
+    CapacitySchedulerConfiguration conf = cs.getConfiguration();
+    String queuePath = super.getQueuePath();
+    int maxAppsForReservation = conf.getMaximumApplicationsPerQueue(queuePath);
+    showReservationsAsQueues = conf.getShowReservationAsQueues(queuePath);
+    if (maxAppsForReservation < 0) {
+      maxAppsForReservation =
+          (int) (CapacitySchedulerConfiguration.
+              DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS * super
+              .getAbsoluteCapacity());
+    }
+    int userLimit = conf.getUserLimit(queuePath);
+    float userLimitFactor = conf.getUserLimitFactor(queuePath);
+    int maxAppsPerUserForReservation =
+        (int) (maxAppsForReservation * (userLimit / 100.0f) * userLimitFactor);
+    updateQuotas(userLimit, userLimitFactor, maxAppsForReservation,
+        maxAppsPerUserForReservation);
 
     StringBuffer queueInfo = new StringBuffer();
-    queueInfo.append("Created Plan Queue: ").append(queueName).append(
-        "]\nwith capacity: [").append(super.getCapacity()).append(
-        "]\nwith max capacity: [").append(super.getMaximumCapacity()).append(
-        "\nwith max apps: [").append(leafQueueTemplate.getMaxApps()).append(
-        "]\nwith max apps per user: [").append(
-        leafQueueTemplate.getMaxAppsPerUser()).append("]\nwith user limit: [")
-        .append(leafQueueTemplate.getUserLimit()).append(
-        "]\nwith user limit factor: [").append(
-        leafQueueTemplate.getUserLimitFactor()).append("].");
+    queueInfo.append("Created Plan Queue: ").append(queueName)
+        .append("\nwith capacity: [").append(super.getCapacity())
+        .append("]\nwith max capacity: [").append(super.getMaximumCapacity())
+        .append("\nwith max reservation apps: [").append(maxAppsForReservation)
+        .append("]\nwith max reservation apps per user: [")
+        .append(maxAppsPerUserForReservation).append("]\nwith user limit: [")
+        .append(userLimit).append("]\nwith user limit factor: [")
+        .append(userLimitFactor).append("].");
     LOG.info(queueInfo.toString());
   }
 
   @Override
-  public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
-      throws IOException {
-    validate(newlyParsedQueue);
-    super.reinitialize(newlyParsedQueue, clusterResource);
-    this.leafQueueTemplate = initializeLeafQueueConfigs(getQueuePath()).build();
+  public void reinitialize(CSQueue newlyParsedQueue,
+      Resource clusterResource) throws IOException {
+    try {
+      writeLock.lock();
+      // Sanity check
+      if (!(newlyParsedQueue instanceof PlanQueue) || !newlyParsedQueue
+          .getQueuePath().equals(getQueuePath())) {
+        throw new IOException(
+            "Trying to reinitialize " + getQueuePath() + " from "
+                + newlyParsedQueue.getQueuePath());
+      }
+
+      PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
+
+      if (newlyParsedParentQueue.getChildQueues().size() != 1) {
+        throw new IOException(
+            "Reservable Queue should not have sub-queues in the"
+                + "configuration expect the default reservation queue");
+      }
+
+      // Set new configs
+      setupQueueConfigs(clusterResource);
+
+      updateQuotas(newlyParsedParentQueue.userLimit,
+          newlyParsedParentQueue.userLimitFactor,
+          newlyParsedParentQueue.maxAppsForReservation,
+          newlyParsedParentQueue.maxAppsPerUserForReservation);
+
+      // run reinitialize on each existing queue, to trigger absolute cap
+      // recomputations
+      for (CSQueue res : this.getChildQueues()) {
+        res.reinitialize(res, clusterResource);
+      }
+      showReservationsAsQueues =
+          newlyParsedParentQueue.showReservationsAsQueues;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
-  @Override
-  protected AutoCreatedLeafQueueTemplate.Builder initializeLeafQueueConfigs
-      (String queuePath) {
-    AutoCreatedLeafQueueTemplate.Builder leafQueueTemplate = super
-        .initializeLeafQueueConfigs
-        (queuePath);
-    showReservationsAsQueues = csContext.getConfiguration()
-        .getShowReservationAsQueues(queuePath);
-    return leafQueueTemplate;
+  private void updateQuotas(int userLimit, float userLimitFactor,
+      int maxAppsForReservation, int maxAppsPerUserForReservation) {
+    this.userLimit = userLimit;
+    this.userLimitFactor = userLimitFactor;
+    this.maxAppsForReservation = maxAppsForReservation;
+    this.maxAppsPerUserForReservation = maxAppsPerUserForReservation;
   }
 
-  protected void validate(final CSQueue newlyParsedQueue) throws IOException {
-    // Sanity check
-    if (!(newlyParsedQueue instanceof PlanQueue) || !newlyParsedQueue
-        .getQueuePath().equals(getQueuePath())) {
-      throw new IOException(
-          "Trying to reinitialize " + getQueuePath() + " from "
-              + newlyParsedQueue.getQueuePath());
-    }
+  /**
+   * Number of maximum applications for each of the reservations in this Plan.
+   *
+   * @return maxAppsForreservation
+   */
+  public int getMaxApplicationsForReservations() {
+    return maxAppsForReservation;
+  }
 
-    PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
+  /**
+   * Number of maximum applications per user for each of the reservations in
+   * this Plan.
+   *
+   * @return maxAppsPerUserForreservation
+   */
+  public int getMaxApplicationsPerUserForReservation() {
+    return maxAppsPerUserForReservation;
+  }
 
-    if (newlyParsedParentQueue.getChildQueues().size() != 1) {
-      throw new IOException(
-          "Reservable Queue should not have sub-queues in the"
-              + "configuration expect the default reservation queue");
-    }
+  /**
+   * User limit value for each of the reservations in this Plan.
+   *
+   * @return userLimit
+   */
+  public int getUserLimitForReservation() {
+    return userLimit;
+  }
+
+  /**
+   * User limit factor value for each of the reservations in this Plan.
+   *
+   * @return userLimitFactor
+   */
+  public float getUserLimitFactor() {
+    return userLimitFactor;
   }
 
   /**
@@ -98,4 +170,4 @@ public class PlanQueue extends AbstractManagedParentQueue {
   public boolean showReservationsAsQueues() {
     return showReservationsAsQueues;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementChange.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/QueueManagementChange.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/QueueManagementChange.java
new file mode 100644
index 0000000..74d9b23
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementChange.java
@@ -0,0 +1,148 @@
+/**
+ * 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.scheduler.capacity;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.QueueState;
+
+/**
+ * Encapsulates Queue entitlement and state updates needed
+ * for adjusting capacity dynamically
+ *
+ */
+@Private
+@Unstable
+public abstract class QueueManagementChange {
+
+  private final CSQueue queue;
+
+  /**
+   * Updating the queue may involve entitlement updates
+   * and/or QueueState changes
+   *
+   * QueueAction can potentially be enhanced
+   * for adding, removing queues for queue management
+   */
+  public enum QueueAction {
+    UPDATE_QUEUE
+  }
+
+  private AutoCreatedLeafQueueConfig
+      queueTemplateUpdate;
+
+  private final QueueAction queueAction;
+  /**
+   * Updated Queue state with the new entitlement
+   */
+  private QueueState transitionToQueueState;
+
+  public QueueManagementChange(final CSQueue queue,
+      final QueueAction queueAction) {
+    this.queue = queue;
+    this.queueAction = queueAction;
+  }
+
+  public QueueManagementChange(final CSQueue queue,
+      final QueueAction queueAction, QueueState targetQueueState,
+      final AutoCreatedLeafQueueConfig
+          queueTemplateUpdates) {
+    this(queue, queueAction, queueTemplateUpdates);
+    this.transitionToQueueState = targetQueueState;
+  }
+
+  public QueueManagementChange(final CSQueue queue,
+      final QueueAction queueAction,
+      final AutoCreatedLeafQueueConfig
+      queueTemplateUpdates) {
+    this(queue, queueAction);
+    this.queueTemplateUpdate = queueTemplateUpdates;
+  }
+
+  public QueueState getTransitionToQueueState() {
+    return transitionToQueueState;
+  }
+
+  public CSQueue getQueue() {
+    return queue;
+  }
+
+  public AutoCreatedLeafQueueConfig getUpdatedQueueTemplate() {
+    return queueTemplateUpdate;
+  }
+
+  public QueueAction getQueueAction() {
+    return queueAction;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)
+      return true;
+    if (!(o instanceof QueueManagementChange))
+      return false;
+
+    QueueManagementChange that = (QueueManagementChange) o;
+
+    if (queue != null ? !queue.equals(that.queue) : that.queue != null)
+      return false;
+    if (queueTemplateUpdate != null ? !queueTemplateUpdate.equals(
+        that.queueTemplateUpdate) : that.queueTemplateUpdate != null)
+      return false;
+    if (queueAction != that.queueAction)
+      return false;
+    return transitionToQueueState == that.transitionToQueueState;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = queue != null ? queue.hashCode() : 0;
+    result = 31 * result + (queueTemplateUpdate != null ?
+        queueTemplateUpdate.hashCode() :
+        0);
+    result = 31 * result + (queueAction != null ? queueAction.hashCode() : 0);
+    result = 31 * result + (transitionToQueueState != null ?
+        transitionToQueueState.hashCode() :
+        0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "QueueManagementChange{" + "queue=" + queue
+        + ", updatedEntitlementsByPartition=" + queueTemplateUpdate
+        + ", queueAction=" + queueAction + ", transitionToQueueState="
+        + transitionToQueueState + '}';
+  }
+
+  public static class UpdateQueue extends QueueManagementChange {
+
+    public UpdateQueue(final CSQueue queue, QueueState targetQueueState,
+        final AutoCreatedLeafQueueConfig
+            queueTemplateUpdate) {
+      super(queue, QueueAction.UPDATE_QUEUE, targetQueueState,
+          queueTemplateUpdate);
+    }
+
+    public UpdateQueue(final CSQueue queue,
+        final AutoCreatedLeafQueueConfig
+            queueTemplateUpdate) {
+      super(queue, QueueAction.UPDATE_QUEUE, queueTemplateUpdate);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementDynamicEditPolicy.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/QueueManagementDynamicEditPolicy.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/QueueManagementDynamicEditPolicy.java
new file mode 100644
index 0000000..9b0cf7b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementDynamicEditPolicy.java
@@ -0,0 +1,272 @@
+/**
+ * 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.scheduler.capacity;
+
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
+    .QueueManagementChangeEvent;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Queue Management scheduling policy for managed parent queues which enable
+ * auto child queue creation
+ */
+public class QueueManagementDynamicEditPolicy implements SchedulingEditPolicy {
+
+  private static final Log LOG =
+      LogFactory.getLog(QueueManagementDynamicEditPolicy.class);
+
+  private Clock clock;
+
+  // Pointer to other RM components
+  private RMContext rmContext;
+  private ResourceCalculator rc;
+  private CapacityScheduler scheduler;
+  private RMNodeLabelsManager nlm;
+
+  private long monitoringInterval;
+
+  private Set<String> managedParentQueues = new HashSet<>();
+
+  /**
+   * Instantiated by CapacitySchedulerConfiguration
+   */
+  public QueueManagementDynamicEditPolicy() {
+    clock = SystemClock.getInstance();
+  }
+
+  @SuppressWarnings("unchecked")
+  @VisibleForTesting
+  public QueueManagementDynamicEditPolicy(RMContext context,
+      CapacityScheduler scheduler) {
+    init(context.getYarnConfiguration(), context, scheduler);
+  }
+
+  @SuppressWarnings("unchecked")
+  @VisibleForTesting
+  public QueueManagementDynamicEditPolicy(RMContext context,
+      CapacityScheduler scheduler, Clock clock) {
+    init(context.getYarnConfiguration(), context, scheduler);
+    this.clock = clock;
+  }
+
+  @Override
+  public void init(final Configuration config, final RMContext context,
+      final ResourceScheduler sched) {
+    LOG.info("Queue Management Policy monitor:" + this.
+        getClass().getCanonicalName());
+    assert null == scheduler : "Unexpected duplicate call to init";
+    if (!(sched instanceof CapacityScheduler)) {
+      throw new YarnRuntimeException("Class " +
+          sched.getClass().getCanonicalName() + " not instance of " +
+          CapacityScheduler.class.getCanonicalName());
+    }
+    rmContext = context;
+    scheduler = (CapacityScheduler) sched;
+    clock = scheduler.getClock();
+
+    rc = scheduler.getResourceCalculator();
+    nlm = scheduler.getRMContext().getNodeLabelManager();
+
+    CapacitySchedulerConfiguration csConfig = scheduler.getConfiguration();
+
+    monitoringInterval = csConfig.getLong(
+        CapacitySchedulerConfiguration.QUEUE_MANAGEMENT_MONITORING_INTERVAL,
+        CapacitySchedulerConfiguration.
+            DEFAULT_QUEUE_MANAGEMENT_MONITORING_INTERVAL);
+
+    initQueues();
+  }
+
+  /**
+   * Reinitializes queues(Called on scheduler.reinitialize)
+   * @param config Configuration
+   * @param context The resourceManager's context
+   * @param sched The scheduler
+   */
+  public void reinitialize(final Configuration config, final RMContext context,
+      final ResourceScheduler sched) {
+    //TODO - Wire with scheduler reinitialize and remove initQueues below?
+    initQueues();
+  }
+
+  private void initQueues() {
+    managedParentQueues.clear();
+    for (Map.Entry<String, CSQueue> queues : scheduler
+        .getCapacitySchedulerQueueManager()
+        .getQueues().entrySet()) {
+
+      String queueName = queues.getKey();
+      CSQueue queue = queues.getValue();
+
+      if ( queue instanceof ManagedParentQueue) {
+        managedParentQueues.add(queueName);
+      }
+    }
+  }
+
+  @Override
+  public void editSchedule() {
+    long startTs = clock.getTime();
+
+    initQueues();
+    manageAutoCreatedLeafQueues();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Total time used=" + (clock.getTime() - startTs) + " ms.");
+    }
+  }
+
+  @VisibleForTesting
+  List<QueueManagementChange> manageAutoCreatedLeafQueues()
+  {
+
+    List<QueueManagementChange> queueManagementChanges = new ArrayList<>();
+    // All partitions to look at
+
+    //Proceed only if there are queues to process
+    if (managedParentQueues.size() > 0) {
+      for (String parentQueueName : managedParentQueues) {
+        ManagedParentQueue parentQueue =
+            (ManagedParentQueue) scheduler.getCapacitySchedulerQueueManager().
+                getQueue(parentQueueName);
+
+        queueManagementChanges.addAll(
+            computeQueueManagementChanges
+            (parentQueue));
+      }
+    }
+    return queueManagementChanges;
+  }
+
+
+  @VisibleForTesting
+  List<QueueManagementChange> computeQueueManagementChanges
+      (ManagedParentQueue parentQueue) {
+
+    List<QueueManagementChange> queueManagementChanges =
+        Collections.emptyList();
+    if (!parentQueue.shouldFailAutoCreationWhenGuaranteedCapacityExceeded()) {
+
+      AutoCreatedQueueManagementPolicy policyClazz =
+          parentQueue.getAutoCreatedQueueManagementPolicy();
+      long startTime = 0;
+      try {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(MessageFormat
+              .format("Trying to use {0} to compute preemption "
+                      + "candidates",
+                  policyClazz.getClass().getName()));
+          startTime = clock.getTime();
+        }
+
+        queueManagementChanges = policyClazz.computeQueueManagementChanges();
+
+        //Scheduler update is asynchronous
+        if (queueManagementChanges.size() > 0) {
+          QueueManagementChangeEvent queueManagementChangeEvent =
+              new QueueManagementChangeEvent(parentQueue,
+                  queueManagementChanges);
+          scheduler.getRMContext().getDispatcher().getEventHandler().handle(
+              queueManagementChangeEvent);
+        }
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(MessageFormat.format("{0} uses {1} millisecond"
+                  + " to run",
+              policyClazz.getClass().getName(), clock.getTime()
+                  - startTime));
+          if (queueManagementChanges.size() > 0) {
+            LOG.debug(" Updated queue management updates for parent queue"
+                + " ["
+                + parentQueue.getQueueName() + ": [\n" + queueManagementChanges
+                .toString() + "\n]");
+          }
+        }
+      } catch (YarnException e) {
+        LOG.error(
+            "Could not compute child queue management updates for parent "
+                + "queue "
+                + parentQueue.getQueueName(), e);
+      }
+    } else{
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Skipping queue management updates for parent queue "
+                + parentQueue
+                .getQueuePath() + " "
+                + "since configuration for  auto creating queue's beyond "
+                + "parent's "
+                + "guaranteed capacity is disabled");
+      }
+    }
+    return queueManagementChanges;
+  }
+
+  @Override
+  public long getMonitoringInterval() {
+    return monitoringInterval;
+  }
+
+  @Override
+  public String getPolicyName() {
+    return "QueueManagementDynamicEditPolicy";
+  }
+
+  public ResourceCalculator getResourceCalculator() {
+    return rc;
+  }
+
+  public RMContext getRmContext() {
+    return rmContext;
+  }
+
+  public ResourceCalculator getRC() {
+    return rc;
+  }
+
+  public CapacityScheduler getScheduler() {
+    return scheduler;
+  }
+
+  public Set<String> getManagedParentQueues() {
+    return managedParentQueues;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.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/ReservationQueue.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/ReservationQueue.java
new file mode 100644
index 0000000..34f4aa1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
@@ -0,0 +1,91 @@
+/**
+ * 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.scheduler.capacity;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This represents a dynamic {@link LeafQueue} managed by the
+ * {@link ReservationSystem}
+ *
+ */
+public class ReservationQueue extends AbstractAutoCreatedLeafQueue {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ReservationQueue.class);
+
+  private PlanQueue parent;
+
+  public ReservationQueue(CapacitySchedulerContext cs, String queueName,
+      PlanQueue parent) throws IOException {
+    super(cs, queueName, parent, null);
+    // the following parameters are common to all reservation in the plan
+    updateQuotas(parent.getUserLimitForReservation(),
+        parent.getUserLimitFactor(),
+        parent.getMaxApplicationsForReservations(),
+        parent.getMaxApplicationsPerUserForReservation());
+    this.parent = parent;
+  }
+
+  @Override
+  public void reinitialize(CSQueue newlyParsedQueue,
+      Resource clusterResource) throws IOException {
+    try {
+      writeLock.lock();
+      // Sanity check
+      if (!(newlyParsedQueue instanceof ReservationQueue) || !newlyParsedQueue
+          .getQueuePath().equals(getQueuePath())) {
+        throw new IOException(
+            "Trying to reinitialize " + getQueuePath() + " from "
+                + newlyParsedQueue.getQueuePath());
+      }
+      super.reinitialize(newlyParsedQueue, clusterResource);
+      CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
+          this, labelManager, null);
+
+      updateQuotas(parent.getUserLimitForReservation(),
+          parent.getUserLimitFactor(),
+          parent.getMaxApplicationsForReservations(),
+          parent.getMaxApplicationsPerUserForReservation());
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private void updateQuotas(int userLimit, float userLimitFactor,
+      int maxAppsForReservation, int maxAppsPerUserForReservation) {
+    setUserLimit(userLimit);
+    setUserLimitFactor(userLimitFactor);
+    setMaxApplications(maxAppsForReservation);
+    maxApplicationsPerUser = maxAppsPerUserForReservation;
+  }
+
+  @Override
+  protected void setupConfigurableCapacities(CapacitySchedulerConfiguration
+      configuration) {
+    super.setupConfigurableCapacities(queueCapacities);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.java
new file mode 100644
index 0000000..aee6405
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.java
@@ -0,0 +1,745 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scheduler.capacity
+    .queuemanagement;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .AbstractAutoCreatedLeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .AutoCreatedLeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .AutoCreatedLeafQueueConfig;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .AutoCreatedQueueManagementPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .CapacitySchedulerContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .LeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .ManagedParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .ParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .QueueManagementChange;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
+    .FiCaSchedulerApp;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.MonotonicClock;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
+    .NO_LABEL;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.CSQueueUtils.EPSILON;
+
+/**
+ * Capacity Management policy for auto created leaf queues
+ * <p>
+ * Assigns capacity if available to leaf queues based on application
+ * submission order i.e leaf queues are assigned capacity in FCFS order based
+ * on application submission time.  Updates leaf queue capacities to 0 when
+ * there are no pending or running apps under that queue.
+ */
+public class GuaranteedOrZeroCapacityOverTimePolicy
+    implements AutoCreatedQueueManagementPolicy {
+
+  private CapacitySchedulerContext scheduler;
+  private ManagedParentQueue managedParentQueue;
+
+  private static final Log LOG = LogFactory.getLog(
+      GuaranteedOrZeroCapacityOverTimePolicy.class);
+
+  private AutoCreatedLeafQueueConfig ZERO_CAPACITY_ENTITLEMENT;
+
+  private ReentrantReadWriteLock.WriteLock writeLock;
+
+  private ReentrantReadWriteLock.ReadLock readLock;
+
+  private ParentQueueState parentQueueState = new ParentQueueState();
+
+  private AutoCreatedLeafQueueConfig leafQueueTemplate;
+
+  private QueueCapacities leafQueueTemplateCapacities;
+
+  private Map<String, LeafQueueState> leafQueueStateMap = new HashMap<>();
+
+  private Clock clock = new MonotonicClock();
+
+  private class LeafQueueState {
+
+    private AtomicBoolean isActive = new AtomicBoolean(false);
+
+    private long mostRecentActivationTime;
+
+    private long mostRecentDeactivationTime;
+
+    public long getMostRecentActivationTime() {
+      return mostRecentActivationTime;
+    }
+
+    public long getMostRecentDeactivationTime() {
+      return mostRecentDeactivationTime;
+    }
+
+    /**
+     * Is the queue currently active or deactivated?
+     *
+     * @return true if Active else false
+     */
+    public boolean isActive() {
+      return isActive.get();
+    }
+
+    private boolean activate() {
+      boolean ret = isActive.compareAndSet(false, true);
+      mostRecentActivationTime = clock.getTime();
+      return ret;
+    }
+
+    private boolean deactivate() {
+      boolean ret = isActive.compareAndSet(true, false);
+      mostRecentDeactivationTime = clock.getTime();
+      return ret;
+    }
+  }
+
+  private boolean containsLeafQueue(String leafQueueName) {
+    return leafQueueStateMap.containsKey(leafQueueName);
+  }
+
+  private boolean addLeafQueueStateIfNotExists(String leafQueueName,
+      LeafQueueState leafQueueState) {
+    if (!containsLeafQueue(leafQueueName)) {
+      leafQueueStateMap.put(leafQueueName, leafQueueState);
+      return true;
+    }
+    return false;
+  }
+
+  private boolean addLeafQueueStateIfNotExists(LeafQueue leafQueue) {
+    return addLeafQueueStateIfNotExists(leafQueue.getQueueName(),
+        new LeafQueueState());
+  }
+
+  private void clearLeafQueueState() {
+    leafQueueStateMap.clear();
+  }
+
+  private class ParentQueueState {
+
+    private Map<String, Float> totalAbsoluteActivatedChildQueueCapacityByLabel =
+        new HashMap<String, Float>();
+
+    private float getAbsoluteActivatedChildQueueCapacity() {
+      return getAbsoluteActivatedChildQueueCapacity(NO_LABEL);
+    }
+
+    private float getAbsoluteActivatedChildQueueCapacity(String nodeLabel) {
+      try {
+        readLock.lock();
+        Float totalActivatedCapacity = getByLabel(nodeLabel);
+        if (totalActivatedCapacity != null) {
+          return totalActivatedCapacity;
+        } else{
+          return 0;
+        }
+      } finally {
+        readLock.unlock();
+      }
+    }
+
+    private void incAbsoluteActivatedChildCapacity(String nodeLabel,
+        float childQueueCapacity) {
+      try {
+        writeLock.lock();
+        Float activatedChildCapacity = getByLabel(nodeLabel);
+        if (activatedChildCapacity != null) {
+          setByLabel(nodeLabel, activatedChildCapacity + childQueueCapacity);
+        } else{
+          setByLabel(nodeLabel, childQueueCapacity);
+        }
+      } finally {
+        writeLock.unlock();
+      }
+    }
+
+    private void decAbsoluteActivatedChildCapacity(String nodeLabel,
+        float childQueueCapacity) {
+      try {
+        writeLock.lock();
+        Float activatedChildCapacity = getByLabel(nodeLabel);
+        if (activatedChildCapacity != null) {
+          setByLabel(nodeLabel, activatedChildCapacity - childQueueCapacity);
+        } else{
+          setByLabel(nodeLabel, childQueueCapacity);
+        }
+      } finally {
+        writeLock.unlock();
+      }
+    }
+
+    Float getByLabel(String label) {
+      return totalAbsoluteActivatedChildQueueCapacityByLabel.get(label);
+    }
+
+    Float setByLabel(String label, float val) {
+      return totalAbsoluteActivatedChildQueueCapacityByLabel.put(label, val);
+    }
+
+    void clear() {
+      totalAbsoluteActivatedChildQueueCapacityByLabel.clear();
+    }
+  }
+
+  /**
+   * Comparator that orders applications by their submit time
+   */
+  private class PendingApplicationComparator
+      implements Comparator<FiCaSchedulerApp> {
+
+    @Override
+    public int compare(FiCaSchedulerApp app1, FiCaSchedulerApp app2) {
+      RMApp rmApp1 = scheduler.getRMContext().getRMApps().get(
+          app1.getApplicationId());
+      RMApp rmApp2 = scheduler.getRMContext().getRMApps().get(
+          app2.getApplicationId());
+      if (rmApp1 != null && rmApp2 != null) {
+        return Long.compare(rmApp1.getSubmitTime(), rmApp2.getSubmitTime());
+      } else if (rmApp1 != null) {
+        return -1;
+      } else if (rmApp2 != null) {
+        return 1;
+      } else{
+        return 0;
+      }
+    }
+  }
+
+  private PendingApplicationComparator applicationComparator =
+      new PendingApplicationComparator();
+
+  @Override
+  public void init(final CapacitySchedulerContext schedulerContext,
+      final ParentQueue parentQueue) {
+    this.scheduler = schedulerContext;
+
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+
+    if (!(parentQueue instanceof ManagedParentQueue)) {
+      throw new IllegalArgumentException(
+          "Expected instance of type " + ManagedParentQueue.class);
+    }
+
+    this.managedParentQueue = (ManagedParentQueue) parentQueue;
+
+    initializeLeafQueueTemplate(this.managedParentQueue);
+
+    LOG.info(
+        "Initialized queue management policy for parent queue " + parentQueue
+            .getQueueName() + " with leaf queue template capacities : ["
+            + leafQueueTemplate.getQueueCapacities() + "]");
+  }
+
+  private void initializeLeafQueueTemplate(ManagedParentQueue parentQueue) {
+    leafQueueTemplate = parentQueue.getLeafQueueTemplate();
+
+    leafQueueTemplateCapacities = leafQueueTemplate.getQueueCapacities();
+
+    ZERO_CAPACITY_ENTITLEMENT = buildTemplate(0.0f,
+        leafQueueTemplateCapacities.getMaximumCapacity());
+  }
+
+  @Override
+  public List<QueueManagementChange> computeQueueManagementChanges()
+      throws SchedulerDynamicEditException {
+
+    //TODO : Add support for node labels on leaf queue template configurations
+    //synch/add missing leaf queue(s) if any to state
+    updateLeafQueueState();
+
+    try {
+      readLock.lock();
+      List<QueueManagementChange> queueManagementChanges = new ArrayList<>();
+
+      // check if any leaf queues need to be deactivated based on pending
+      // applications and
+      float parentAbsoluteCapacity =
+          managedParentQueue.getQueueCapacities().getAbsoluteCapacity();
+
+      float leafQueueTemplateAbsoluteCapacity =
+          leafQueueTemplateCapacities.getAbsoluteCapacity();
+      Map<String, QueueCapacities> deactivatedLeafQueues =
+          deactivateLeafQueuesIfInActive(managedParentQueue, queueManagementChanges);
+
+      float deactivatedCapacity = getTotalDeactivatedCapacity(
+          deactivatedLeafQueues);
+
+      float sumOfChildQueueActivatedCapacity = parentQueueState.
+          getAbsoluteActivatedChildQueueCapacity();
+
+      //Check if we need to activate anything at all?
+      float availableCapacity = getAvailableCapacity(parentAbsoluteCapacity,
+          deactivatedCapacity, sumOfChildQueueActivatedCapacity);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Parent queue : " + managedParentQueue.getQueueName() + " absCapacity = "
+                + parentAbsoluteCapacity + ", leafQueueAbsoluteCapacity = "
+                + leafQueueTemplateAbsoluteCapacity + ", deactivatedCapacity = "
+                + deactivatedCapacity + " , absChildActivatedCapacity = "
+                + sumOfChildQueueActivatedCapacity + ", availableCapacity = "
+                + availableCapacity);
+      }
+
+      if (availableCapacity >= leafQueueTemplateAbsoluteCapacity) {
+        //sort applications across leaf queues by submit time
+        List<FiCaSchedulerApp> pendingApps = getSortedPendingApplications();
+
+        if (pendingApps.size() > 0) {
+          int maxLeafQueuesTobeActivated = getMaxLeavesToBeActivated(
+              availableCapacity, leafQueueTemplateAbsoluteCapacity,
+              pendingApps.size());
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Found " + maxLeafQueuesTobeActivated
+                + " leaf queues to be activated with " + pendingApps.size()
+                + " apps ");
+          }
+
+          LinkedHashSet<String> leafQueuesToBeActivated = getSortedLeafQueues(
+              pendingApps, maxLeafQueuesTobeActivated,
+              deactivatedLeafQueues.keySet());
+
+          //Compute entitlement changes for the identified leaf queues
+          // which is appended to the List of queueManagementChanges
+          computeQueueManagementChanges(leafQueuesToBeActivated,
+              queueManagementChanges, availableCapacity,
+              leafQueueTemplateAbsoluteCapacity);
+
+          if (LOG.isDebugEnabled()) {
+            if (leafQueuesToBeActivated.size() > 0) {
+              LOG.debug(
+                  "Activated leaf queues : [" + leafQueuesToBeActivated + "]");
+            }
+          }
+        }
+      }
+      return queueManagementChanges;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  private float getTotalDeactivatedCapacity(
+      Map<String, QueueCapacities> deactivatedLeafQueues) {
+    float deactivatedCapacity = 0;
+    for (Iterator<Map.Entry<String, QueueCapacities>> iterator =
+         deactivatedLeafQueues.entrySet().iterator(); iterator.hasNext(); ) {
+      Map.Entry<String, QueueCapacities> deactivatedQueueCapacity =
+          iterator.next();
+      deactivatedCapacity +=
+          deactivatedQueueCapacity.getValue().getAbsoluteCapacity();
+    }
+    return deactivatedCapacity;
+  }
+
+  @VisibleForTesting
+  void updateLeafQueueState() {
+    try {
+      writeLock.lock();
+      Set<String> newQueues = new HashSet<>();
+      for (CSQueue newQueue : managedParentQueue.getChildQueues()) {
+        if (newQueue instanceof LeafQueue) {
+          addLeafQueueStateIfNotExists((LeafQueue) newQueue);
+          newQueues.add(newQueue.getQueueName());
+        }
+      }
+
+      for (Iterator<Map.Entry<String, LeafQueueState>> itr =
+           leafQueueStateMap.entrySet().iterator(); itr.hasNext(); ) {
+        Map.Entry<String, LeafQueueState> e = itr.next();
+        String queueName = e.getKey();
+        if (!newQueues.contains(queueName)) {
+          itr.remove();
+        }
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private LinkedHashSet<String> getSortedLeafQueues(
+      final List<FiCaSchedulerApp> pendingApps, int leafQueuesNeeded,
+      Set<String> deactivatedQueues) throws SchedulerDynamicEditException {
+
+    LinkedHashSet<String> leafQueues = new LinkedHashSet<>(leafQueuesNeeded);
+    int ctr = 0;
+    for (FiCaSchedulerApp app : pendingApps) {
+
+      AutoCreatedLeafQueue leafQueue =
+          (AutoCreatedLeafQueue) app.getCSLeafQueue();
+      String leafQueueName = leafQueue.getQueueName();
+
+      //Check if leafQueue is not active already and has any pending apps
+      if (ctr < leafQueuesNeeded) {
+
+        if (!isActive(leafQueue)) {
+          if (!deactivatedQueues.contains(leafQueueName)) {
+            if (addLeafQueueIfNotExists(leafQueues, leafQueueName)) {
+              ctr++;
+            }
+          }
+        }
+      } else{
+        break;
+      }
+    }
+    return leafQueues;
+  }
+
+  private boolean addLeafQueueIfNotExists(Set<String> leafQueues,
+      String leafQueueName) {
+    boolean ret = false;
+    if (!leafQueues.contains(leafQueueName)) {
+      ret = leafQueues.add(leafQueueName);
+    }
+    return ret;
+  }
+
+  @VisibleForTesting
+  public boolean isActive(final AutoCreatedLeafQueue leafQueue)
+      throws SchedulerDynamicEditException {
+    try {
+      readLock.lock();
+      LeafQueueState leafQueueStatus = getLeafQueueState(leafQueue);
+      return leafQueueStatus.isActive();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  private Map<String, QueueCapacities> deactivateLeafQueuesIfInActive(
+      ParentQueue parentQueue,
+      List<QueueManagementChange> queueManagementChanges)
+      throws SchedulerDynamicEditException {
+    Map<String, QueueCapacities> deactivatedQueues = new HashMap<>();
+
+    for (CSQueue childQueue : parentQueue.getChildQueues()) {
+      AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue;
+
+      if (isActive(leafQueue) && !hasPendingApps(leafQueue)) {
+        queueManagementChanges.add(
+            new QueueManagementChange.UpdateQueue(leafQueue,
+                ZERO_CAPACITY_ENTITLEMENT));
+        deactivatedQueues.put(leafQueue.getQueueName(),
+            leafQueueTemplateCapacities);
+      } else{
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(" Leaf queue has pending applications :  " + leafQueue
+              .getNumApplications() + ".Skipping deactivation for "
+              + leafQueue);
+        }
+      }
+    }
+
+    if (LOG.isDebugEnabled()) {
+      if (deactivatedQueues.size() > 0) {
+        LOG.debug("Deactivated leaf queues : " + deactivatedQueues);
+      }
+    }
+    return deactivatedQueues;
+  }
+
+  private void computeQueueManagementChanges(
+      Set<String> leafQueuesToBeActivated,
+      List<QueueManagementChange> queueManagementChanges,
+      final float availableCapacity,
+      final float leafQueueTemplateAbsoluteCapacity) {
+
+    float curAvailableCapacity = availableCapacity;
+
+    for (String curLeafQueue : leafQueuesToBeActivated) {
+      // Activate queues if capacity is available
+      if (curAvailableCapacity >= leafQueueTemplateAbsoluteCapacity) {
+        AutoCreatedLeafQueue leafQueue =
+            (AutoCreatedLeafQueue) scheduler.getCapacitySchedulerQueueManager()
+                .getQueue(curLeafQueue);
+        if (leafQueue != null) {
+          AutoCreatedLeafQueueConfig newTemplate = buildTemplate(
+              leafQueueTemplateCapacities.getCapacity(),
+              leafQueueTemplateCapacities.getMaximumCapacity());
+          queueManagementChanges.add(
+              new QueueManagementChange.UpdateQueue(leafQueue, newTemplate));
+          curAvailableCapacity -= leafQueueTemplateAbsoluteCapacity;
+        } else{
+          LOG.warn(
+              "Could not find queue in scheduler while trying to deactivate "
+                  + curLeafQueue);
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public int getMaxLeavesToBeActivated(float availableCapacity,
+      float childQueueAbsoluteCapacity, int numPendingApps)
+      throws SchedulerDynamicEditException {
+
+    if (childQueueAbsoluteCapacity > 0) {
+      int numLeafQueuesNeeded = (int) Math.floor(
+          availableCapacity / childQueueAbsoluteCapacity);
+
+      return Math.min(numLeafQueuesNeeded, numPendingApps);
+    } else{
+      throw new SchedulerDynamicEditException("Child queue absolute capacity "
+          + "is initialized to 0. Check parent queue's  " + managedParentQueue
+          .getQueueName() + " leaf queue template configuration");
+    }
+  }
+
+  private float getAvailableCapacity(float parentAbsCapacity,
+      float deactivatedAbsCapacity, float totalChildQueueActivatedCapacity) {
+    return parentAbsCapacity - totalChildQueueActivatedCapacity
+        + deactivatedAbsCapacity + EPSILON;
+  }
+
+  /**
+   * Commit queue management changes - which involves updating required state
+   * on parent/underlying leaf queues
+   *
+   * @param queueManagementChanges Queue Management changes to commit
+   * @throws SchedulerDynamicEditException when validation fails
+   */
+  @Override
+  public void commitQueueManagementChanges(
+      List<QueueManagementChange> queueManagementChanges)
+      throws SchedulerDynamicEditException {
+    try {
+      writeLock.lock();
+      for (QueueManagementChange queueManagementChange :
+          queueManagementChanges) {
+        AutoCreatedLeafQueueConfig updatedQueueTemplate =
+            queueManagementChange.getUpdatedQueueTemplate();
+        CSQueue queue = queueManagementChange.getQueue();
+        if (!(queue instanceof AutoCreatedLeafQueue)) {
+          throw new SchedulerDynamicEditException(
+              "Expected queue management change for AutoCreatedLeafQueue. "
+                  + "Found " + queue.getClass().getName());
+        }
+
+        AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) queue;
+
+        if (updatedQueueTemplate.getQueueCapacities().getCapacity() > 0) {
+          if (isActive(leafQueue)) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(
+                  "Queue is already active. Skipping activation : " + queue
+                      .getQueuePath());
+            }
+          } else{
+            activate(leafQueue);
+          }
+        } else{
+          if (!isActive(leafQueue)) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(
+                  "Queue is already de-activated. " + "Skipping de-activation "
+                      + ": " + leafQueue.getQueuePath());
+            }
+          } else{
+            deactivate(leafQueue);
+          }
+        }
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private void activate(final AutoCreatedLeafQueue leafQueue)
+      throws SchedulerDynamicEditException {
+    try {
+      writeLock.lock();
+      getLeafQueueState(leafQueue).activate();
+
+      parentQueueState.incAbsoluteActivatedChildCapacity(NO_LABEL,
+          leafQueueTemplateCapacities.getAbsoluteCapacity());
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private void deactivate(final AutoCreatedLeafQueue leafQueue)
+      throws SchedulerDynamicEditException {
+    try {
+      writeLock.lock();
+      getLeafQueueState(leafQueue).deactivate();
+
+      for (String nodeLabel : managedParentQueue.getQueueCapacities()
+          .getExistingNodeLabels()) {
+        parentQueueState.decAbsoluteActivatedChildCapacity(nodeLabel,
+            leafQueueTemplateCapacities.getAbsoluteCapacity());
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public boolean hasPendingApps(final AutoCreatedLeafQueue leafQueue) {
+    return leafQueue.getNumApplications() > 0;
+  }
+
+  @Override
+  public void reinitialize(CapacitySchedulerContext schedulerContext,
+      final ParentQueue parentQueue) {
+    if (!(parentQueue instanceof ManagedParentQueue)) {
+      throw new IllegalStateException(
+          "Expected instance of type " + ManagedParentQueue.class + " found  "
+              + " : " + parentQueue.getClass());
+    }
+
+    if (this.managedParentQueue != null && !parentQueue.getQueuePath().equals(
+        this.managedParentQueue.getQueuePath())) {
+      throw new IllegalStateException(
+          "Expected parent queue path to match " + this.managedParentQueue
+              .getQueuePath() + " found : " + parentQueue.getQueuePath());
+    }
+
+    this.managedParentQueue = (ManagedParentQueue) parentQueue;
+
+    initializeLeafQueueTemplate(this.managedParentQueue);
+
+    //clear state
+    parentQueueState.clear();
+    clearLeafQueueState();
+
+    LOG.info(
+        "Reinitialized queue management policy for parent queue "
+            + parentQueue.getQueueName() +" with leaf queue template "
+            + "capacities : ["
+            + leafQueueTemplate.getQueueCapacities() + "]");
+  }
+
+  @Override
+  public AutoCreatedLeafQueueConfig getInitialLeafQueueConfiguration(
+      AbstractAutoCreatedLeafQueue leafQueue)
+      throws SchedulerDynamicEditException {
+
+    if ( !(leafQueue instanceof  AutoCreatedLeafQueue)) {
+      throw new SchedulerDynamicEditException("Not an instance of "
+          + "AutoCreatedLeafQueue : " + leafQueue.getClass());
+    }
+
+    AutoCreatedLeafQueue autoCreatedLeafQueue =
+        (AutoCreatedLeafQueue) leafQueue;
+    AutoCreatedLeafQueueConfig template = ZERO_CAPACITY_ENTITLEMENT;
+    try {
+      writeLock.lock();
+      if (!addLeafQueueStateIfNotExists(leafQueue)) {
+        LOG.error("Leaf queue already exists in state : " + getLeafQueueState(
+            leafQueue));
+        throw new SchedulerDynamicEditException(
+            "Leaf queue already exists in state : " + getLeafQueueState(
+                leafQueue));
+      }
+
+      float availableCapacity = getAvailableCapacity(
+          managedParentQueue.getQueueCapacities().getAbsoluteCapacity(), 0,
+          parentQueueState.getAbsoluteActivatedChildQueueCapacity());
+
+      if (availableCapacity >= leafQueueTemplateCapacities
+          .getAbsoluteCapacity()) {
+        activate(autoCreatedLeafQueue);
+        template = buildTemplate(leafQueueTemplateCapacities.getCapacity(),
+            leafQueueTemplateCapacities.getMaximumCapacity());
+      }
+    } finally {
+      writeLock.unlock();
+    }
+    return template;
+  }
+
+  @VisibleForTesting
+  LeafQueueState getLeafQueueState(LeafQueue queue)
+      throws SchedulerDynamicEditException {
+    try {
+      readLock.lock();
+      String queueName = queue.getQueueName();
+      if (!containsLeafQueue(queueName)) {
+        throw new SchedulerDynamicEditException(
+            "Could not find leaf queue in " + "state " + queueName);
+      } else{
+        return leafQueueStateMap.get(queueName);
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @VisibleForTesting
+  public float getAbsoluteActivatedChildQueueCapacity() {
+    return parentQueueState.getAbsoluteActivatedChildQueueCapacity();
+  }
+
+  private List<FiCaSchedulerApp> getSortedPendingApplications() {
+    List<FiCaSchedulerApp> apps = new ArrayList<>(
+        managedParentQueue.getAllApplications());
+    Collections.sort(apps, applicationComparator);
+    return apps;
+  }
+
+  private AutoCreatedLeafQueueConfig buildTemplate(float capacity,
+      float maxCapacity) {
+    AutoCreatedLeafQueueConfig.Builder templateBuilder =
+        new AutoCreatedLeafQueueConfig.Builder();
+
+    QueueCapacities capacities = new QueueCapacities(false);
+    templateBuilder.capacities(capacities);
+
+    for (String nodeLabel : managedParentQueue.getQueueCapacities()
+        .getExistingNodeLabels()) {
+      capacities.setCapacity(nodeLabel, capacity);
+      capacities.setMaximumCapacity(nodeLabel, maxCapacity);
+    }
+
+    return new AutoCreatedLeafQueueConfig(templateBuilder);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/QueueEntitlement.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/QueueEntitlement.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/QueueEntitlement.java
index 2a751e3..f4182f3 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/QueueEntitlement.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/QueueEntitlement.java
@@ -43,4 +43,26 @@ public class QueueEntitlement {
   public void setCapacity(float capacity) {
     this.capacity = capacity;
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)
+      return true;
+    if (!(o instanceof QueueEntitlement))
+      return false;
+
+    QueueEntitlement that = (QueueEntitlement) o;
+
+    if (Float.compare(that.capacity, capacity) != 0)
+      return false;
+    return Float.compare(that.maxCapacity, maxCapacity) == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = (capacity != +0.0f ? Float.floatToIntBits(capacity) : 0);
+    result = 31 * result + (maxCapacity != +0.0f ? Float.floatToIntBits(
+        maxCapacity) : 0);
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/QueueManagementChangeEvent.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/event/QueueManagementChangeEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/QueueManagementChangeEvent.java
new file mode 100644
index 0000000..926e1be
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/QueueManagementChangeEvent.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .ParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .QueueManagementChange;
+
+import java.util.List;
+
+/**
+ * Event to update scheduler of any queue management changes
+ */
+public class QueueManagementChangeEvent extends SchedulerEvent {
+
+  private ParentQueue parentQueue;
+  private List<QueueManagementChange> queueManagementChanges;
+
+  public QueueManagementChangeEvent(ParentQueue parentQueue,
+      List<QueueManagementChange> queueManagementChanges) {
+    super(SchedulerEventType.MANAGE_QUEUE);
+    this.parentQueue = parentQueue;
+    this.queueManagementChanges = queueManagementChanges;
+  }
+
+  public ParentQueue getParentQueue() {
+    return parentQueue;
+  }
+
+  public List<QueueManagementChange> getQueueManagementChanges() {
+    return queueManagementChanges;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.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/event/SchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
index 229e0bb..b107cf4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
@@ -51,5 +51,8 @@ public enum SchedulerEventType {
   MARK_CONTAINER_FOR_KILLABLE,
 
   // Cancel a killable container
-  MARK_CONTAINER_FOR_NONKILLABLE
+  MARK_CONTAINER_FOR_NONKILLABLE,
+
+  //Queue Management Change
+  MANAGE_QUEUE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAutoCreatedLeafQueue.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/TestAutoCreatedLeafQueue.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/TestAutoCreatedLeafQueue.java
deleted file mode 100644
index b403e72..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAutoCreatedLeafQueue.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
-import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test class for dynamic auto created leaf queues.
- * @see AutoCreatedLeafQueue
- */
-public class TestAutoCreatedLeafQueue {
-
-  private CapacitySchedulerConfiguration csConf;
-  private CapacitySchedulerContext csContext;
-  final static int DEF_MAX_APPS = 10000;
-  final static int GB = 1024;
-  private final ResourceCalculator resourceCalculator =
-      new DefaultResourceCalculator();
-  private AutoCreatedLeafQueue autoCreatedLeafQueue;
-
-  @Before
-  public void setup() throws IOException {
-    // setup a context / conf
-    csConf = new CapacitySchedulerConfiguration();
-    YarnConfiguration conf = new YarnConfiguration();
-    csContext = mock(CapacitySchedulerContext.class);
-    when(csContext.getConfiguration()).thenReturn(csConf);
-    when(csContext.getConf()).thenReturn(conf);
-    when(csContext.getMinimumResourceCapability()).thenReturn(
-        Resources.createResource(GB, 1));
-    when(csContext.getMaximumResourceCapability()).thenReturn(
-        Resources.createResource(16 * GB, 32));
-    when(csContext.getClusterResource()).thenReturn(
-        Resources.createResource(100 * 16 * GB, 100 * 32));
-    when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
-    RMContext mockRMContext = TestUtils.getMockRMContext();
-    when(csContext.getRMContext()).thenReturn(mockRMContext);
-
-    // create a queue
-    PlanQueue pq = new PlanQueue(csContext, "root", null, null);
-    autoCreatedLeafQueue = new AutoCreatedLeafQueue(csContext, "a", pq);
-  }
-
-  private void validateAutoCreatedLeafQueue(double capacity) {
-    assertTrue(" actual capacity: " + autoCreatedLeafQueue.getCapacity(),
-        autoCreatedLeafQueue.getCapacity() - capacity < CSQueueUtils.EPSILON);
-    assertEquals(autoCreatedLeafQueue.maxApplications, DEF_MAX_APPS);
-    assertEquals(autoCreatedLeafQueue.maxApplicationsPerUser, DEF_MAX_APPS);
-  }
-
-  @Test
-  public void testAddSubtractCapacity() throws Exception {
-
-    // verify that setting, adding, subtracting capacity works
-    autoCreatedLeafQueue.setCapacity(1.0F);
-    validateAutoCreatedLeafQueue(1);
-    autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(0.9f, 1f));
-    validateAutoCreatedLeafQueue(0.9);
-    autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(1f, 1f));
-    validateAutoCreatedLeafQueue(1);
-    autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(0f, 1f));
-    validateAutoCreatedLeafQueue(0);
-
-    try {
-      autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(1.1f, 1f));
-      fail();
-    } catch (SchedulerDynamicEditException iae) {
-      // expected
-      validateAutoCreatedLeafQueue(1);
-    }
-
-    try {
-      autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(-0.1f, 1f));
-      fail();
-    } catch (SchedulerDynamicEditException iae) {
-      // expected
-      validateAutoCreatedLeafQueue(1);
-    }
-
-  }
-}


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


[26/50] [abbrv] hadoop git commit: YARN-7482. Max applications calculation per queue has to be retrospected with absolute resource support. Contributed by Sunil G.

Posted by wa...@apache.org.
YARN-7482. Max applications calculation per queue has to be retrospected with absolute resource support. Contributed by Sunil G.


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

Branch: refs/heads/YARN-6592
Commit: e65ca92fb6897a3004a235d7b69e308399189aae
Parents: 034b312
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Thu Nov 16 17:33:00 2017 +0530
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../scheduler/capacity/ParentQueue.java         | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e65ca92f/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 940637e..a427fb1 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
@@ -1088,6 +1088,26 @@ public class ParentQueue extends AbstractCSQueue {
     childQueue.getQueueCapacities().setAbsoluteMaximumCapacity(label,
         (float) childQueue.getQueueCapacities().getMaximumCapacity(label)
             / getQueueCapacities().getAbsoluteMaximumCapacity(label));
+
+    // Re-visit max applications for a queue based on absolute capacity if
+    // needed.
+    if (childQueue instanceof LeafQueue) {
+      LeafQueue leafQueue = (LeafQueue) childQueue;
+      CapacitySchedulerConfiguration conf = csContext.getConfiguration();
+      int maxApplications = (int) (conf.getMaximumSystemApplications()
+          * childQueue.getQueueCapacities().getAbsoluteCapacity(label));
+      leafQueue.setMaxApplications(maxApplications);
+
+      int maxApplicationsPerUser = Math.min(maxApplications,
+          (int) (maxApplications
+              * (leafQueue.getUsersManager().getUserLimit() / 100.0f)
+              * leafQueue.getUsersManager().getUserLimitFactor()));
+      leafQueue.setMaxApplicationsPerUser(maxApplicationsPerUser);
+      LOG.info("LeafQueue:" + leafQueue.getQueueName() + ", maxApplications="
+          + maxApplications + ", maxApplicationsPerUser="
+          + maxApplicationsPerUser + ", Abs Cap:"
+          + childQueue.getQueueCapacities().getAbsoluteCapacity(label));
+    }
   }
 
   @Override


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


[02/50] [abbrv] hadoop git commit: HADOOP-14475 Metrics of S3A don't print out when enabled. Contributed by Younger and Sean Mackrory.

Posted by wa...@apache.org.
HADOOP-14475 Metrics of S3A don't print out when enabled. Contributed by Younger and Sean Mackrory.


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

Branch: refs/heads/YARN-6592
Commit: 6555af81a26b0b72ec3bee7034e01f5bd84b1564
Parents: a957f1c
Author: Aaron Fabbri <fa...@apache.org>
Authored: Tue Dec 5 11:06:32 2017 -0800
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Tue Dec 5 11:06:32 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |   3 +
 .../hadoop/fs/s3a/S3AInstrumentation.java       | 107 ++++++++++++++++---
 .../src/site/markdown/tools/hadoop-aws/index.md |  38 ++++++-
 .../apache/hadoop/fs/s3a/ITestS3AMetrics.java   |  51 +++++++++
 4 files changed, 186 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index fba8ea1..63a4349 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -78,6 +78,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.ListeningExecutorService;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -2423,6 +2424,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
         metadataStore.close();
         metadataStore = null;
       }
+      IOUtils.closeQuietly(instrumentation);
+      instrumentation = null;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index ef5a434..0fbcc00 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -29,8 +30,10 @@ import org.apache.hadoop.metrics2.MetricStringBuilder;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.MetricsTag;
-import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
 import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
@@ -58,16 +61,49 @@ import static org.apache.hadoop.fs.s3a.Statistic.*;
  * the operations to increment/query metric values are designed to handle
  * lookup failures.
  */
-@Metrics(about = "Metrics for S3a", context = "S3AFileSystem")
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class S3AInstrumentation {
+public class S3AInstrumentation implements Closeable, MetricsSource {
   private static final Logger LOG = LoggerFactory.getLogger(
       S3AInstrumentation.class);
 
-  public static final String CONTEXT = "S3AFileSystem";
+  private static final String METRICS_SOURCE_BASENAME = "S3AMetrics";
+
+  /**
+   * {@value #METRICS_SYSTEM_NAME} The name of the s3a-specific metrics
+   * system instance used for s3a metrics.
+   */
+  public static final String METRICS_SYSTEM_NAME = "s3a-file-system";
+
+  /**
+   * {@value #CONTEXT} Currently all s3a metrics are placed in a single
+   * "context". Distinct contexts may be used in the future.
+   */
+  public static final String CONTEXT = "s3aFileSystem";
+
+  /**
+   * {@value #METRIC_TAG_FILESYSTEM_ID} The name of a field added to metrics
+   * records that uniquely identifies a specific FileSystem instance.
+   */
+  public static final String METRIC_TAG_FILESYSTEM_ID = "s3aFileSystemId";
+
+  /**
+   * {@value #METRIC_TAG_BUCKET} The name of a field added to metrics records
+   * that indicates the hostname portion of the FS URL.
+   */
+  public static final String METRIC_TAG_BUCKET = "bucket";
+
+  // metricsSystemLock must be used to synchronize modifications to
+  // metricsSystem and the following counters.
+  private static Object metricsSystemLock = new Object();
+  private static MetricsSystem metricsSystem = null;
+  private static int metricsSourceNameCounter = 0;
+  private static int metricsSourceActiveCounter = 0;
+
+  private String metricsSourceName;
+
   private final MetricsRegistry registry =
-      new MetricsRegistry("S3AFileSystem").setContext(CONTEXT);
+      new MetricsRegistry("s3aFileSystem").setContext(CONTEXT);
   private final MutableCounterLong streamOpenOperations;
   private final MutableCounterLong streamCloseOperations;
   private final MutableCounterLong streamClosed;
@@ -146,7 +182,6 @@ public class S3AInstrumentation {
       STORE_IO_THROTTLED
   };
 
-
   private static final Statistic[] GAUGES_TO_CREATE = {
       OBJECT_PUT_REQUESTS_ACTIVE,
       OBJECT_PUT_BYTES_PENDING,
@@ -157,12 +192,10 @@ public class S3AInstrumentation {
 
   public S3AInstrumentation(URI name) {
     UUID fileSystemInstanceId = UUID.randomUUID();
-    registry.tag("FileSystemId",
-        "A unique identifier for the FS ",
-        fileSystemInstanceId.toString() + "-" + name.getHost());
-    registry.tag("fsURI",
-        "URI of this filesystem",
-        name.toString());
+    registry.tag(METRIC_TAG_FILESYSTEM_ID,
+        "A unique identifier for the instance",
+        fileSystemInstanceId.toString());
+    registry.tag(METRIC_TAG_BUCKET, "Hostname from the FS URL", name.getHost());
     streamOpenOperations = streamCounter(STREAM_OPENED);
     streamCloseOperations = streamCounter(STREAM_CLOSE_OPERATIONS);
     streamClosed = streamCounter(STREAM_CLOSED);
@@ -204,6 +237,39 @@ public class S3AInstrumentation {
         "ops", "latency", interval);
     quantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
         "events", "frequency (Hz)", interval);
+
+    registerAsMetricsSource(name);
+  }
+
+  @VisibleForTesting
+  public MetricsSystem getMetricsSystem() {
+    synchronized (metricsSystemLock) {
+      if (metricsSystem == null) {
+        metricsSystem = new MetricsSystemImpl();
+        metricsSystem.init(METRICS_SYSTEM_NAME);
+      }
+    }
+    return metricsSystem;
+  }
+
+  /**
+   * Register this instance as a metrics source.
+   * @param name s3a:// URI for the associated FileSystem instance
+   */
+  private void registerAsMetricsSource(URI name) {
+    int number;
+    synchronized(metricsSystemLock) {
+      getMetricsSystem();
+
+      metricsSourceActiveCounter++;
+      number = ++metricsSourceNameCounter;
+    }
+    String msName = METRICS_SOURCE_BASENAME + number;
+    if (number > 1) {
+      msName = msName + number;
+    }
+    metricsSourceName = msName + "-" + name.getHost();
+    metricsSystem.register(metricsSourceName, "", this);
   }
 
   /**
@@ -560,6 +626,23 @@ public class S3AInstrumentation {
     streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort);
   }
 
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    registry.snapshot(collector.addRecord(registry.info().name()), true);
+  }
+
+  public void close() {
+    synchronized (metricsSystemLock) {
+      metricsSystem.unregisterSource(metricsSourceName);
+      int activeSources = --metricsSourceActiveCounter;
+      if (activeSources == 0) {
+        metricsSystem.publishMetricsNow();
+        metricsSystem.shutdown();
+        metricsSystem = null;
+      }
+    }
+  }
+
   /**
    * Statistics updated by an input stream during its actual operation.
    * These counters not thread-safe and are for use in a single instance

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index aaf9f23..a8d2e48 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -1,4 +1,3 @@
-
 <!---
   Licensed under the Apache License, Version 2.0 (the "License");
   you may not use this file except in compliance with the License.
@@ -1599,6 +1598,43 @@ basis.
 to set fadvise policies on input streams. Once implemented,
 this will become the supported mechanism used for configuring the input IO policy.
 
+##<a name="metrics"></a>Metrics
+
+S3A metrics can be monitored through Hadoop's metrics2 framework. S3A creates
+its own metrics system called s3a-file-system, and each instance of the client
+will create its own metrics source, named with a JVM-unique numerical ID.
+
+As a simple example, the following can be added to `hadoop-metrics2.properties`
+to write all S3A metrics to a log file every 10 seconds:
+
+    s3a-file-system.sink.my-metrics-config.class=org.apache.hadoop.metrics2.sink.FileSink
+    s3a-file-system.sink.my-metrics-config.filename=/var/log/hadoop-yarn/s3a-metrics.out
+    *.period=10
+
+Lines in that file will be structured like the following:
+
+    1511208770680 s3aFileSystem.s3aFileSystem: Context=s3aFileSystem, s3aFileSystemId=892b02bb-7b30-4ffe-80ca-3a9935e1d96e, bucket=bucket,
+    Hostname=hostname-1.hadoop.apache.com, files_created=1, files_copied=2, files_copied_bytes=10000, files_deleted=5, fake_directories_deleted=3,
+    directories_created=3, directories_deleted=0, ignored_errors=0, op_copy_from_local_file=0, op_exists=0, op_get_file_status=15, op_glob_status=0,
+    op_is_directory=0, op_is_file=0, op_list_files=0, op_list_located_status=0, op_list_status=3, op_mkdirs=1, op_rename=2, object_copy_requests=0,
+    object_delete_requests=6, object_list_requests=23, object_continue_list_requests=0, object_metadata_requests=46, object_multipart_aborted=0,
+    object_put_bytes=0, object_put_requests=4, object_put_requests_completed=4, stream_write_failures=0, stream_write_block_uploads=0,
+    stream_write_block_uploads_committed=0, stream_write_block_uploads_aborted=0, stream_write_total_time=0, stream_write_total_data=0,
+    s3guard_metadatastore_put_path_request=10, s3guard_metadatastore_initialization=0, object_put_requests_active=0, object_put_bytes_pending=0,
+    stream_write_block_uploads_active=0, stream_write_block_uploads_pending=0, stream_write_block_uploads_data_pending=0,
+    S3guard_metadatastore_put_path_latencyNumOps=0, S3guard_metadatastore_put_path_latency50thPercentileLatency=0,
+    S3guard_metadatastore_put_path_latency75thPercentileLatency=0, S3guard_metadatastore_put_path_latency90thPercentileLatency=0,
+    S3guard_metadatastore_put_path_latency95thPercentileLatency=0, S3guard_metadatastore_put_path_latency99thPercentileLatency=0
+
+Depending on other configuration, metrics from other systems, contexts, etc. may
+also get recorded, for example the following:
+
+    1511208770680 metricssystem.MetricsSystem: Context=metricssystem, Hostname=s3a-metrics-4.gce.cloudera.com, NumActiveSources=1, NumAllSources=1,
+    NumActiveSinks=1, NumAllSinks=0, Sink_fileNumOps=2, Sink_fileAvgTime=1.0, Sink_fileDropped=0, Sink_fileQsize=0, SnapshotNumOps=5,
+    SnapshotAvgTime=0.0, PublishNumOps=2, PublishAvgTime=0.0, DroppedPubAll=0
+
+Note that low-level metrics from the AWS SDK itself are not currently included
+in these metrics.
 
 ##<a name="further_reading"></a> Other Topics
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java
new file mode 100644
index 0000000..182990c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Test s3a performance metrics register and output.
+ */
+public class ITestS3AMetrics extends AbstractS3ATestBase {
+
+  @Test
+  public void testMetricsRegister()
+      throws IOException, InterruptedException {
+    S3AFileSystem fs = getFileSystem();
+    Path dest = new Path("newfile1");
+    ContractTestUtils.touch(fs, dest);
+
+    String targetMetricSource = "S3AMetrics1" + "-" + fs.getBucket();
+    assertNotNull("No metrics under test fs for " + targetMetricSource,
+        fs.getInstrumentation().getMetricsSystem()
+            .getSource(targetMetricSource));
+
+    MutableCounterLong fileCreated =
+        (MutableCounterLong) fs.getInstrumentation().getRegistry()
+            .get(Statistic.FILES_CREATED.getSymbol());
+    assertEquals("Metrics system should report single file created event",
+        1, fileCreated.value());
+  }
+}


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


[08/50] [abbrv] hadoop git commit: YARN-7610. Extend Distributed Shell to support launching job with opportunistic containers. Contributed by Weiwei Yang.

Posted by wa...@apache.org.
YARN-7610. Extend Distributed Shell to support launching job with opportunistic containers. Contributed by Weiwei Yang.


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

Branch: refs/heads/YARN-6592
Commit: 40b0045ebe0752cd3d1d09be00acbabdea983799
Parents: 56b1ff8
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Dec 6 17:52:41 2017 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Wed Dec 6 17:52:41 2017 +0800

----------------------------------------------------------------------
 .../distributedshell/ApplicationMaster.java     |  22 +-
 .../applications/distributedshell/Client.java   |  19 ++
 .../distributedshell/TestDistributedShell.java  |  56 +++++
 .../site/markdown/OpportunisticContainers.md    | 212 -----------------
 .../site/markdown/OpportunisticContainers.md.vm | 233 +++++++++++++++++++
 5 files changed, 329 insertions(+), 213 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/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 91dbc00..926de50 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
@@ -41,6 +41,7 @@ import java.util.Vector;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.Arrays;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
@@ -90,6 +91,8 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
@@ -241,6 +244,9 @@ public class ApplicationMaster {
   private int containerVirtualCores = DEFAULT_CONTAINER_VCORES;
   // Priority of the request
   private int requestPriority;
+  // Execution type of the containers.
+  // Default GUARANTEED.
+  private ExecutionType containerType = ExecutionType.GUARANTEED;
 
   // Resource profile for the container
   private String containerResourceProfile = "";
@@ -412,6 +418,8 @@ public class ApplicationMaster {
         "App Attempt ID. Not to be used unless for testing purposes");
     opts.addOption("shell_env", true,
         "Environment for shell script. Specified as env_key=env_val pairs");
+    opts.addOption("container_type", true,
+        "Container execution type, GUARANTEED or OPPORTUNISTIC");
     opts.addOption("container_memory", true,
         "Amount of memory in MB to be requested to run the shell command");
     opts.addOption("container_vcores", true,
@@ -558,6 +566,16 @@ public class ApplicationMaster {
       domainId = envs.get(DSConstants.DISTRIBUTEDSHELLTIMELINEDOMAIN);
     }
 
+    if (cliParser.hasOption("container_type")) {
+      String containerTypeStr = cliParser.getOptionValue("container_type");
+      if (Arrays.stream(ExecutionType.values()).noneMatch(
+          executionType -> executionType.toString()
+              .equals(containerTypeStr))) {
+        throw new IllegalArgumentException("Invalid container_type: "
+            + containerTypeStr);
+      }
+      containerType = ExecutionType.valueOf(containerTypeStr);
+    }
     containerMemory = Integer.parseInt(cliParser.getOptionValue(
         "container_memory", "-1"));
     containerVirtualCores = Integer.parseInt(cliParser.getOptionValue(
@@ -1242,7 +1260,9 @@ public class ApplicationMaster {
 
     // Set up resource type requirements
     ContainerRequest request =
-        new ContainerRequest(createProfileCapability(), null, null, pri);
+        new ContainerRequest(createProfileCapability(), null, null,
+            pri, 0, true, null,
+            ExecutionTypeRequest.newInstance(containerType));
     LOG.info("Requested container ask: " + request.toString());
     return request;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.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/Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
index 0582afe..16bf0fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Vector;
+import java.util.Arrays;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
@@ -74,6 +75,7 @@ import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
@@ -171,6 +173,8 @@ public class Client {
   // No. of containers in which the shell script needs to be executed
   private int numContainers = 1;
   private String nodeLabelExpression = null;
+  // Container type, default GUARANTEED.
+  private ExecutionType containerType = ExecutionType.GUARANTEED;
 
   // log4j.properties file 
   // if available, add to local resources and set into classpath 
@@ -282,6 +286,8 @@ public class Client {
     opts.addOption("shell_env", true,
         "Environment for shell script. Specified as env_key=env_val pairs");
     opts.addOption("shell_cmd_priority", true, "Priority for the shell command containers");
+    opts.addOption("container_type", true,
+        "Container execution type, GUARANTEED or OPPORTUNISTIC");
     opts.addOption("container_memory", true, "Amount of memory in MB to be requested to run the shell command");
     opts.addOption("container_vcores", true, "Amount of virtual cores to be requested to run the shell command");
     opts.addOption("container_resource_profile", true, "Resource profile for the shell command");
@@ -433,6 +439,16 @@ public class Client {
     }
     shellCmdPriority = Integer.parseInt(cliParser.getOptionValue("shell_cmd_priority", "0"));
 
+    if (cliParser.hasOption("container_type")) {
+      String containerTypeStr = cliParser.getOptionValue("container_type");
+      if (Arrays.stream(ExecutionType.values()).noneMatch(
+          executionType -> executionType.toString()
+          .equals(containerTypeStr))) {
+        throw new IllegalArgumentException("Invalid container_type: "
+            + containerTypeStr);
+      }
+      containerType = ExecutionType.valueOf(containerTypeStr);
+    }
     containerMemory =
         Integer.parseInt(cliParser.getOptionValue("container_memory", "-1"));
     containerVirtualCores =
@@ -740,6 +756,9 @@ public class Client {
     // Set class name 
     vargs.add(appMasterMainClass);
     // Set params for Application Master
+    if (containerType != null) {
+      vargs.add("--container_type " + String.valueOf(containerType));
+    }
     if (containerMemory > 0) {
       vargs.add("--container_memory " + String.valueOf(containerMemory));
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/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 5cf884b..d6bb8d6 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
@@ -1179,6 +1179,33 @@ public class TestDistributedShell {
           e.getMessage().contains("No shell command or shell script specified " +
           "to be executed by application master"));
     }
+
+    LOG.info("Initializing DS Client with invalid container_type argument");
+    try {
+      String[] args = {
+          "--jar",
+          APPMASTER_JAR,
+          "--num_containers",
+          "2",
+          "--master_memory",
+          "512",
+          "--master_vcores",
+          "2",
+          "--container_memory",
+          "128",
+          "--container_vcores",
+          "1",
+          "--shell_command",
+          "date",
+          "--container_type",
+          "UNSUPPORTED_TYPE"
+      };
+      client.init(args);
+      Assert.fail("Exception is expected");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue("The throw exception is not expected",
+          e.getMessage().contains("Invalid container_type: UNSUPPORTED_TYPE"));
+    }
   }
 
   @Test
@@ -1377,4 +1404,33 @@ public class TestDistributedShell {
       }
     }
   }
+
+  @Test
+  public void testDSShellWithOpportunisticContainers() throws Exception {
+    Client client = new Client(new Configuration(yarnCluster.getConfig()));
+    try {
+      String[] args = {
+          "--jar",
+          APPMASTER_JAR,
+          "--num_containers",
+          "2",
+          "--master_memory",
+          "512",
+          "--master_vcores",
+          "2",
+          "--container_memory",
+          "128",
+          "--container_vcores",
+          "1",
+          "--shell_command",
+          "date",
+          "--container_type",
+          "OPPORTUNISTIC"
+      };
+      client.init(args);
+      client.run();
+    } catch (Exception e) {
+      Assert.fail("Job execution with opportunistic containers failed.");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
deleted file mode 100644
index 83beb07..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
+++ /dev/null
@@ -1,212 +0,0 @@
-<!---
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-   http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-Opportunistic Containers
-========================
-
-<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
-
-
-<a name="Purpose"></a>Purpose
------------------------------
-
-This document introduces the notion of **opportunistic** container execution, and discusses how opportunistic containers are allocated and executed.
-
-
-<a name="Quick_Guide"></a>Quick Guide
---------------------------------------------------------------------
-
-We start by providing a brief overview of opportunistic containers, including how a user can enable this feature and run a sample job using such containers.
-
-###<a name="Main_Goal"></a>Main Goal
-
-Unlike existing YARN containers that are scheduled in a node only if there are unallocated resources, opportunistic containers can be dispatched to an NM, even if their execution at that node cannot start immediately. In such a case, opportunistic containers will be queued at that NM until resources become available. 
-The main goal of opportunistic container execution is to improve cluster resource utilization, and therefore increase task throughput. Resource utilization and task throughput improvements are more pronounced for workloads that include relatively short tasks (in the order of seconds).
-
-
-###<a name="Enabling_Opportunistic_Containers"></a>Enabling Opportunistic Containers
-
-To enable opportunistic container allocation, the following two properties have to be present in **conf/yarn-site.xml**:
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.resourcemanager.opportunistic-container-allocation.enabled` | Enables opportunistic container allocation. | `false` |
-| `yarn.nodemanager.opportunistic-containers-max-queue-length` | Determines the max number of opportunistic containers that can be queued at an NM. | `0` |
-
-The first parameter above has to be set to `true`. The second one has to be set to a positive value to allow queuing of opportunistic containers at the NM. A value of `10` can be used to start experimenting with opportunistic containers. The optimal value depends on the jobs characteristics, the cluster configuration and the target utilization.
-
-By default, allocation of opportunistic containers is performed centrally through the RM. However, a user can choose to enable distributed allocation of opportunistic containers, which can further improve allocation latency for short tasks. Distributed scheduling can be enabling by setting to `true` the following parameter (note that non-opportunistic containers will continue being scheduled through the RM):
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.nodemanager.distributed-scheduling.enabled` | Enables distributed scheduling. | `false` |
-
-
-###<a name="Running_a_Sample_Job"></a>Running a Sample Job
-
-The following command can be used to run a sample pi map-reduce job, executing 40% of mappers using opportunistic containers (substitute `3.0.0-alpha2-SNAPSHOT` below with the version of Hadoop you are using):
-
-```
-$ hadoop jar hadoop-3.0.0-alpha2-SNAPSHOT/share/hadoop/mapreduce/hadoop-mapreduce-examples-3.0.0-alpha2-SNAPSHOT.jar pi -Dmapreduce.job.num-opportunistic-maps-percent="40" 50 100
-```
-
-By changing the value of `mapreduce.job.num-opportunistic-maps-percent` in the above command, we can specify the percentage of mappers that can be executed through opportunistic containers.
-
-
-###<a name="Opportunistic_Containers_in_Web_UI"></a>Opportunistic Containers in Web UI
-
-When opportunistic container allocation is enabled, the following new columns can be observed in the Nodes page of the Web UI (`rm-address:8088/cluster/nodes`):
-
-* Running Containers (O): number of running opportunistic containers on each node;
-* Mem Used (O): Total memory used by opportunistic containers on each node;
-* VCores Used (O): Total CPU virtual cores used by opportunistic containers on each node;
-* Queued Containers: Number of containers queued at each node.
-
-When clicking on a specific container running on a node, the execution type of the container is also shown.
-
-In the rest of the document, we provide an in-depth description of opportunistic containers, including details about their allocation and execution.
-
-
-Overview <a name="Overview"></a>
---------------------------------
-
-The existing schedulers in YARN (Fair and Capacity Scheduler) allocate containers to a node only if there are unallocated resources at that node at the moment of scheduling the containers. This **guaranteed** type of execution has the advantage that once the AM dispatches a container to a node, the container execution will start immediately, since it is guaranteed that there will be available resources. Moreover, unless fairness or capacity constraints are violated, containers are guaranteed to run to completion without being preempted. 
-
-Although this design offers a more predictable task execution, it has two main drawbacks that can lead to suboptimal cluster resource utilization:
-
-* **Feedback delays.** When a container finishes its execution at a node, the RM gets notified that there are available resources through the next NM-RM heartbeat, then the RM schedules a new container at that node, the AM gets notified through the next AM-RM heartbeat, and finally the AM launches the new container at the node. These delays result in idle node resources, which in turn lead to lower resource utilization, especially when workloads involve tasks whose duration is relatively short.
-* **Allocated vs. utilized resources.** The RM allocates containers based on the *allocated* resources at each node, which might be significantly higher than the actually *utilized* resources (e.g., think of a container for which 4GB memory have been allocated, but only 2GB are being utilized). This lowers effective resource utilization, and can be avoided if the RM takes into account the utilized resources during scheduling. However, this has to be done in a way that allows resources to be reclaimed in case the utilized resources of a running container increase.
-
-To mitigate the above problems, in addition to the existing containers (which we term **guaranteed** containers hereafter), we introduce the notion of **opportunistic** containers. An opportunistic container can be dispatched to an NM, even if there are no available (unallocated) resources for it at the moment of scheduling. In such a case, the opportunistic container will be queued at the NM, waiting for resources to become available for its execution to start. The opportunistic containers are of lower priority than the guaranteed ones, which means that they can be preempted for guaranteed containers to start their execution. Therefore, they can be used to improve cluster resource utilization without impacting the execution of existing guaranteed containers.
-
-An additional advantage of opportunistic containers is that they introduce a notion of **execution priority at the NMs**. For instance, a lower priority job that does not require strict execution guarantees can use opportunistic containers or a mix of container execution types for its tasks.
-
-We have introduced two ways of allocating opportunistic containers: a **centralized** and a **distributed** one. In the centralized scheduling, opportunistic containers are allocated through the YARN RM, whereas in the distributed one, through local schedulers that reside at each NM. Centralized allocation allows for higher quality placement decisions and for implementing more involved sharing policies across applications (e.g., fairness). On the other hand, distributed scheduling can offer faster container allocation, which is useful for short tasks, as it avoids the round-trip to the RM. In both cases, the scheduling of guaranteed containers remains intact and happens through the YARN RM (using the existing Fair or Capacity Scheduler).
-
-Note that in the current implementation, we are allocating containers based on allocated (and not utilized) resources. Therefore, we tackle the "feedback delays" problem mentioned above, but not the "allocated vs. utilized resources" one. There is ongoing work (`YARN-1011`) that employs opportunistic containers to address the latter problem too.
-
-Below, we describe in more detail the [container execution types](#Container_Execution_Types), as well as the [execution](#Execution_of_Opportunistic_Containers) (including the container queuing at the NMs) and [allocation](#Allocation_of_Opportunistic_Containers) of opportunistic containers. Then we discuss how to fine-tune opportunistic containers through some [advanced configuration parameters](#Advanced_Configuration). Finally, we discuss open items for [future work](#Items_for_Future_Work).
-
-
-<a name="Container_Execution_Types"></a>Container Execution Types
------------------------------------------------------------------
-
-We introduce the following two types of containers:
-
-* **Guaranteed containers** correspond to the existing YARN containers. They are allocated by the Fair or Capacity Scheduler, and once dispatched to a node, it is guaranteed that there are available resources for their execution to start immediately. Moreover, these containers run to completion (as long as there are no failures). They can be preempted only in case the scheduler's queue to which they belong, violates fairness or capacity constraints.
-* **Opportunistic containers** are not guaranteed to have resources for their execution to start when they get dispatched to a node. Instead, they might be queued at the NM until resources become available. In case a guaranteed container arrives at a node and there are no resources available for it, one or more opportunistic containers will be preempted to execute the guaranteed one.
-
-When an AM submits its resource requests to the RM, it specifies the type for each container (default is guaranteed), determining the way the container will be [allocated](#Allocation_of_Opportunistic_Containers). Subsequently, when the container is launched by the AM at an NM, its type determines how it will be [executed](#Execution_of_Opportunistic_Containers) by the NM.
-
-
-<a name="Execution_of_Opportunistic_Containers"></a>Execution of Opportunistic Containers
----------------------------------------------------------------------------
-
-When a container arrives at an NM, its execution is determined by the available resources at the NM and the container type. Guaranteed containers start their execution immediately, and if needed, the NM will kill running opportunistic containers to ensure there are sufficient resources for the guaranteed ones to start. On the other hand, opportunistic containers can be queued at the NM, if there are no resources available to start their execution when they arrive at the NM. To enable this, we extended the NM by allowing queuing of containers at each node. The NM monitors the local resources, and when there are sufficient resources available, it starts the execution of the opportunistic container that is at the head of the queue.
-
-In particular, when a container arrives at an NM, localization is performed (i.e., all required resources are downloaded), and then the container moves to a `SCHEDULED` state, in which the container is queued, waiting for its execution to begin:
-
-* If there are available resources, the execution of the container starts immediately, irrespective of its execution type.
-* If there are no available resources:
-    * If the container is guaranteed, we kill as many running opportunistic containers as required for the guaranteed container to be executed, and then start its execution.
-    * If the container is opportunistic, it remains at the queue until resources become available.
-* When a container (guaranteed or opportunistic) finishes its execution and resources get freed up, we examine the queued containers and if there are available resources we start their execution. We pick containers from the queue in a FIFO order.
-
-In the [future work items](#Items_for_Future_Work) below, we discuss different ways of prioritizing task execution (queue reordering) and of killing opportunistic containers to make space for guaranteed ones.
-
-
-<a name="Allocation_of_Opportunistic_Containers"></a>Allocation of Opportunistic Containers
------------------------------------------------------------------------------
-
-As mentioned above, we provide both a centralized and a distributed way of allocating opportunistic containers, which we describe below.
-
-###<a name="Centralized_Allocation"></a>Centralized Allocation
-
-We have introduced a new service at the RM, namely the `OpportunisticContainerAllocatorAMService`, which extends the `ApplicationMasterService`. When the centralized opportunistic allocation is enabled, the resource requests from the AMs are served at the RM side by the `OpportunisticContainerAllocatorAMService`, which splits them into two sets of resource requests: 
-
-* The guaranteed set is forwarded to the existing `ApplicationMasterService` and is subsequently handled by the Fair or Capacity Scheduler.
-* The opportunistic set is handled by the new `OpportunisticContainerAllocator`, which performs the scheduling of opportunistic containers to nodes.
-
-The `OpportunisticContainerAllocator` maintains a list with the [least loaded nodes](#Determining_Nodes_for_Allocation) of the cluster at each moment, and assigns containers to them in a round-robin fashion. Note that in the current implementation, we purposely do not take into account node locality constraints. Since an opportunistic container (unlike the guaranteed ones) might wait at the queue of an NM before its execution starts, it is more important to allocate it at a node that is less loaded (i.e., where queuing delay will be smaller) rather than respect its locality constraints. Moreover, we do not take into account sharing (fairness/capacity) constraints for opportunistic containers at the moment. Support for both locality and sharing constraints can be added in the future if required.
-
-
-###<a name="Distributed_Allocation"></a>Distributed Allocation
-
-In order to enable distributed scheduling of opportunistic containers, we have introduced a new service at each NM, called `AMRMProxyService`. The `AMRMProxyService` implements the `ApplicationMasterService` protocol, and acts as a proxy between the AMs running at that node and the RM. When the `AMRMProxyService` is enabled (through a parameter), we force all AMs running at a particular node to communicate with the `AMRMProxyService` of the same node, instead of going directly to the RM. Moreover, to ensure that the AMs will not talk directly with the RM, when a new AM gets initialized, we replace its `AMRMToken` with a token signed by the `AMRMProxyService`.
-
-A chain of interceptors can be registered with the `AMRMProxyService`. One of these interceptors is the `DistributedScheduler` that is responsible for allocating opportunistic containers in a distributed way, without needing to contact the RM. This modular design makes the `AMRMProxyService` instrumental in other scenarios too, such as YARN federation (`YARN-2915`) or throttling down misbehaving AMs, which can be enabled simply by adding additional interceptors at the interceptor chain.
-
-When distributed opportunistic scheduling is enabled, each AM sends its resource requests to the `AMRMProxyService` running at the same node. The `AMRMProxyService` splits the resource requests into two sets:
-
-* The guaranteed set is forwarded to the RM. In this case the `AMRMProxyService` simply acts as a proxy between the AM and the RM, and the container allocation remains intact (using the Fair or Capacity Scheduler).
-* The opportunistic set is not forwarded to the RM. Instead, it is handled by the `DistributedScheduler` that is running locally at the node. In particular, the `DistributedScheduler` maintains a list with the least loaded nodes in the cluster, and allocates containers to them in a round-robin fashion. The RM informs the `DistributedScheduler` about the least loaded nodes at regular intervals through the NM-RM heartbeats.
-
-The above procedure is similar to the one performed by the `OpportunisticContainerAllocatorAMService` in the case of centralized opportunistic scheduling described above. The main difference is that in the distributed case, the splitting of requests into guaranteed and opportunistic happens locally at the node, and only the guaranteed requests are forwarded to the RM, while the opportunistic ones are handled without contacting the RM.
-
-
-###<a name="Determining_Nodes_for_Allocation"></a>Determining Nodes for Allocation
-
-Each NM informs the RM periodically through the NM-RM heartbeats about the number of running guaranteed and opportunistic containers, as well as the number of queued opportunistic containers. The RM gathers this information from all nodes and determines the least loaded ones.
-
-In the case of centralized allocation of opportunistic containers, this information is immediately available, since the allocation happens centrally. In the case of distributed scheduling, the list with the least loaded nodes is propagated to all NMs (and thus becomes available to the `DistributedSchedulers`) through the heartbeat responses from the RM to the NMs. The number of least loaded nodes sent to the NMs is configurable.
-
-At the moment, we take into account only the number of queued opportunistic containers at each node in order to estimate the time an opportunistic container would have to wait if sent to that node and, thus, determine the least loaded nodes. If the AM provided us with information about the estimated task durations, we could take them into account in order to have better estimates of the queue waiting times.
-
-
-###<a name="Rebalancing_Node_Load"></a>Rebalancing Node Load
-
-Occasionally poor placement choices for opportunistic containers may be made (due to stale queue length estimates), which can lead to load imbalance between nodes. The problem is more pronounced under high cluster load, and also in the case of distributed scheduling (multiple `DistributedSchedulers` may place containers at the same NM, since they do not coordinate with each other). To deal with this load imbalance between the NM queues, we perform load shedding to dynamically re-balance the load between NMs. In particular, while aggregating at the RM the queue time estimates published by each NM, we construct a distribution and find a targeted maximal value for the length of the NM queues (based on the mean and standard deviation of the distribution). Then the RM disseminates this value to the various NMs through the heartbeat responses. Subsequently, using this information, an NM on a node whose queue length is above the threshold discards opportunistic containers to meet this maxi
 mal value. This forces the associated individual AMs to reschedule those containers elsewhere.
-
-
-<a name="Advanced_Configuration"></a>Advanced Configuration
---------------------------------------------------
-
-The main properties for enabling opportunistic container allocation and choosing between centralized and distributed allocation were described in the [quick guide](#Quick_Guide) in the beginning of this document. Here we present more advanced configuration. Note that using default values for those parameters should be sufficient in most cases. All parameters below have to be defined in the **conf/yarn-site.xml** file.
-
-To determine the number of [least loaded nodes](#Determining_Nodes_for_Allocation) that will be used when scheduling opportunistic containers and how often this list will be refreshed, we use the following parameters:
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.resourcemanager.opportunistic-container-allocation.nodes-used` | Number of least loaded nodes to be used by the Opportunistic Container allocator for dispatching containers during container allocation. A higher value can improve load balance in large clusters. | `10` |
-| `yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms` | Frequency for computing least loaded nodes. | `1000` |
-
-
-As discussed in the [node load rebalancing](#Rebalancing_Node_Load) section above, at regular intervals, the RM gathers all NM queue lengths and computes their mean value (`avg`) and standard deviation (`stdev`), as well as the value `avg + k*stdev` (where `k` a float). This value gets propagated through the NM-RM heartbeats to all NMs, who should respect that value by dequeuing containers (if required), as long as their current queue length is between a `queue_min_length` and a `queue_max_length` value (these values are used to avoid dequeuing tasks from very short queues and to aggressively dequeue tasks from long queues, respectively). 
-The parameters `k`, `queue_min_length` and `queue_max_length` can be specified as follows:
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.resourcemanager.nm-container-queuing.queue-limit-stdev` | The `k` parameter. | `1.0f` |
-| `yarn.resourcemanager.nm-container-queuing.min-queue-length` | The `queue_min_length` parameter. | `5` |
-| `yarn.resourcemanager.nm-container-queuing.max-queue-length` | The `queue_max_length` parameter. | `15` |
-
-
-Finally, two more properties can further tune the `AMRMProxyService` in case distributed scheduling is used:
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.nodemanager.amrmproxy.address` | The address/port to which the `AMRMProxyService` is bound to. | `0.0.0.0:8049` |
-| `yarn.nodemanager.amrmproxy.client.thread-count` | The number of threads that are used at each NM for serving the interceptors register to the `AMRMProxyService` by different jobs. | `3` |
-
-
-<a name="Items_for_Future_Work"></a>Items for Future Work
------------------------------------------------
-
-Here we describe multiple ways in which we can extend/enhance the allocation and execution of opportunistic containers. We also provide the JIRAs that track each item.
-
-* **Resource overcommitment** (`YARN-1011`). As already discussed, in order to further improve the cluster resource utilization, we can schedule containers not based on the allocated resources but on the actually utilized ones. When over-committing resources, there is the risk of running out of resources in case we have an increase in the utilized resources of the already running containers. Therefore, opportunistic execution should be used for containers whose allocation goes beyond the capacity of a node. This way, we can choose opportunistic containers to kill for reclaiming resources.
-* **NM Queue reordering** (`YARN-5886`). Instead of executing queued containers in a FIFO order, we can employ reordering strategies that dynamically determine which opportunistic container will be executed next. For example, we can prioritize containers that are expected to be short-running or which belong to applications that are close to completion.
-* **Out of order killing at NMs** (`YARN-5887`). As described above, when we need to free up resources for a guaranteed container to start its execution, we kill opportunistic containers in reverse order of arrival (first the most recently started ones). This might not always be the right decision. For example, we might want to minimize the number of containers killed or to refrain from killing containers of jobs that are very close to completion.
-* **Container pausing** (`YARN-5292`): At the moment we kill opportunistic containers to make room for guaranteed in case of resource contention. In busy clusters this can lower the effective cluster utilization: whenever we kill a running opportunistic container, it has to be restarted, and thus we lose work. To this end, we can instead pause running opportunistic containers. Note that this will require support from the container executor (e.g., the container technology used) and from the application.
-* **Container promotion** (`YARN-5085`). There are cases where changing the execution type of a container during its execution can be beneficial. For instance, an application might submit a container as opportunistic, and when its execution starts, it can request its promotion to a guaranteed container to avoid it getting killed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
new file mode 100644
index 0000000..7882b87
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
@@ -0,0 +1,233 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+#set ( $H3 = '###' )
+#set ( $H4 = '####' )
+#set ( $H5 = '#####' )
+
+Opportunistic Containers
+========================
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+
+Purpose
+-------
+
+This document introduces the notion of **opportunistic** container execution, and discusses how opportunistic containers are allocated and executed.
+
+
+Quick Guide
+-----------
+
+We start by providing a brief overview of opportunistic containers, including how a user can enable this feature and run a sample job using such containers.
+
+$H3 Main Goal
+
+Unlike existing YARN containers that are scheduled in a node only if there are unallocated resources, opportunistic containers can be dispatched to an NM, even if their execution at that node cannot start immediately. In such a case, opportunistic containers will be queued at that NM until resources become available. 
+The main goal of opportunistic container execution is to improve cluster resource utilization, and therefore increase task throughput. Resource utilization and task throughput improvements are more pronounced for workloads that include relatively short tasks (in the order of seconds).
+
+
+$H3 Enabling Opportunistic Containers
+
+To enable opportunistic container allocation, the following two properties have to be present in **conf/yarn-site.xml**:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanager.opportunistic-container-allocation.enabled` | Enables opportunistic container allocation. | `false` |
+| `yarn.nodemanager.opportunistic-containers-max-queue-length` | Determines the max number of opportunistic containers that can be queued at an NM. | `0` |
+
+The first parameter above has to be set to `true`. The second one has to be set to a positive value to allow queuing of opportunistic containers at the NM. A value of `10` can be used to start experimenting with opportunistic containers. The optimal value depends on the jobs characteristics, the cluster configuration and the target utilization.
+
+By default, allocation of opportunistic containers is performed centrally through the RM. However, a user can choose to enable distributed allocation of opportunistic containers, which can further improve allocation latency for short tasks. Distributed scheduling can be enabling by setting to `true` the following parameter (note that non-opportunistic containers will continue being scheduled through the RM):
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.nodemanager.distributed-scheduling.enabled` | Enables distributed scheduling. | `false` |
+
+In order to submit jobs to a cluster that has AMRMProxy turned on, one must create a separate set of configs for the client from which jobs will be submitted. In these, the **conf/yarn-site.xml** should have the following additional configurations:
+
+| Property | Value | Description |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanger.scheduler.address` | `localhost:8049` | Redirects jobs to the Node Manager's AMRMProxy port.|
+
+
+$H3 Running a Sample Job
+
+$H4 MapReduce PI
+
+The following command can be used to run a sample pi map-reduce job, executing 40% of mappers using opportunistic containers:
+
+```
+$ hadoop jar share/hadoop/mapreduce/hadoop-mapreduce-examples-${project.version}.jar pi -Dmapreduce.job.num-opportunistic-maps-percent="40" 50 100
+```
+
+By changing the value of `mapreduce.job.num-opportunistic-maps-percent` in the above command, we can specify the percentage of mappers that can be executed through opportunistic containers.
+
+$H4 Distributed Shell
+
+Another sample job is the distributed shell, it allows us to run a given shell command on a set of containers. The following command can be used to run `sleep 10` command in 10 opportunistic containers:
+
+```
+$ yarn org.apache.hadoop.yarn.applications.distributedshell.Client -jar share/hadoop/yarn/hadoop-yarn-applications-distributedshell-${project.version}.jar.jar -shell_command sleep -shell_args 10 -num_containers 10 -container_type OPPORTUNISTIC
+```
+
+By change the value of `container_type` to `OPPORTUNISTIC` or `GUARANTEED` in the above command, we can specify the tasks to be running in opportunistic or guaranteed containers. The default type is `GUARANTEED`.
+
+$H3 Opportunistic Containers in Web UI
+
+When opportunistic container allocation is enabled, the following new columns can be observed in the Nodes page of the Web UI (`rm-address:8088/cluster/nodes`):
+
+* Running Containers (O): number of running opportunistic containers on each node;
+* Mem Used (O): Total memory used by opportunistic containers on each node;
+* VCores Used (O): Total CPU virtual cores used by opportunistic containers on each node;
+* Queued Containers: Number of containers queued at each node.
+
+When clicking on a specific container running on a node, the execution type of the container is also shown.
+
+In the rest of the document, we provide an in-depth description of opportunistic containers, including details about their allocation and execution.
+
+
+Overview
+--------
+
+The existing schedulers in YARN (Fair and Capacity Scheduler) allocate containers to a node only if there are unallocated resources at that node at the moment of scheduling the containers. This **guaranteed** type of execution has the advantage that once the AM dispatches a container to a node, the container execution will start immediately, since it is guaranteed that there will be available resources. Moreover, unless fairness or capacity constraints are violated, containers are guaranteed to run to completion without being preempted. 
+
+Although this design offers a more predictable task execution, it has two main drawbacks that can lead to suboptimal cluster resource utilization:
+
+* **Feedback delays.** When a container finishes its execution at a node, the RM gets notified that there are available resources through the next NM-RM heartbeat, then the RM schedules a new container at that node, the AM gets notified through the next AM-RM heartbeat, and finally the AM launches the new container at the node. These delays result in idle node resources, which in turn lead to lower resource utilization, especially when workloads involve tasks whose duration is relatively short.
+* **Allocated vs. utilized resources.** The RM allocates containers based on the *allocated* resources at each node, which might be significantly higher than the actually *utilized* resources (e.g., think of a container for which 4GB memory have been allocated, but only 2GB are being utilized). This lowers effective resource utilization, and can be avoided if the RM takes into account the utilized resources during scheduling. However, this has to be done in a way that allows resources to be reclaimed in case the utilized resources of a running container increase.
+
+To mitigate the above problems, in addition to the existing containers (which we term **guaranteed** containers hereafter), we introduce the notion of **opportunistic** containers. An opportunistic container can be dispatched to an NM, even if there are no available (unallocated) resources for it at the moment of scheduling. In such a case, the opportunistic container will be queued at the NM, waiting for resources to become available for its execution to start. The opportunistic containers are of lower priority than the guaranteed ones, which means that they can be preempted for guaranteed containers to start their execution. Therefore, they can be used to improve cluster resource utilization without impacting the execution of existing guaranteed containers.
+
+An additional advantage of opportunistic containers is that they introduce a notion of **execution priority at the NMs**. For instance, a lower priority job that does not require strict execution guarantees can use opportunistic containers or a mix of container execution types for its tasks.
+
+We have introduced two ways of allocating opportunistic containers: a **centralized** and a **distributed** one. In the centralized scheduling, opportunistic containers are allocated through the YARN RM, whereas in the distributed one, through local schedulers that reside at each NM. Centralized allocation allows for higher quality placement decisions and for implementing more involved sharing policies across applications (e.g., fairness). On the other hand, distributed scheduling can offer faster container allocation, which is useful for short tasks, as it avoids the round-trip to the RM. In both cases, the scheduling of guaranteed containers remains intact and happens through the YARN RM (using the existing Fair or Capacity Scheduler).
+
+Note that in the current implementation, we are allocating containers based on allocated (and not utilized) resources. Therefore, we tackle the "feedback delays" problem mentioned above, but not the "allocated vs. utilized resources" one. There is ongoing work (`YARN-1011`) that employs opportunistic containers to address the latter problem too.
+
+Below, we describe in more detail the [container execution types](#Container_Execution_Types), as well as the [execution](#Execution_of_Opportunistic_Containers) (including the container queuing at the NMs) and [allocation](#Allocation_of_Opportunistic_Containers) of opportunistic containers. Then we discuss how to fine-tune opportunistic containers through some [advanced configuration parameters](#Advanced_Configuration). Finally, we discuss open items for [future work](#Items_for_Future_Work).
+
+
+Container Execution Types
+-------------------------
+
+We introduce the following two types of containers:
+
+* **Guaranteed containers** correspond to the existing YARN containers. They are allocated by the Fair or Capacity Scheduler, and once dispatched to a node, it is guaranteed that there are available resources for their execution to start immediately. Moreover, these containers run to completion (as long as there are no failures). They can be preempted only in case the scheduler's queue to which they belong, violates fairness or capacity constraints.
+* **Opportunistic containers** are not guaranteed to have resources for their execution to start when they get dispatched to a node. Instead, they might be queued at the NM until resources become available. In case a guaranteed container arrives at a node and there are no resources available for it, one or more opportunistic containers will be preempted to execute the guaranteed one.
+
+When an AM submits its resource requests to the RM, it specifies the type for each container (default is guaranteed), determining the way the container will be [allocated](#Allocation_of_Opportunistic_Containers). Subsequently, when the container is launched by the AM at an NM, its type determines how it will be [executed](#Execution_of_Opportunistic_Containers) by the NM.
+
+
+Execution of Opportunistic Containers
+-------------------------------------
+
+When a container arrives at an NM, its execution is determined by the available resources at the NM and the container type. Guaranteed containers start their execution immediately, and if needed, the NM will kill running opportunistic containers to ensure there are sufficient resources for the guaranteed ones to start. On the other hand, opportunistic containers can be queued at the NM, if there are no resources available to start their execution when they arrive at the NM. To enable this, we extended the NM by allowing queuing of containers at each node. The NM monitors the local resources, and when there are sufficient resources available, it starts the execution of the opportunistic container that is at the head of the queue.
+
+In particular, when a container arrives at an NM, localization is performed (i.e., all required resources are downloaded), and then the container moves to a `SCHEDULED` state, in which the container is queued, waiting for its execution to begin:
+
+* If there are available resources, the execution of the container starts immediately, irrespective of its execution type.
+* If there are no available resources:
+    * If the container is guaranteed, we kill as many running opportunistic containers as required for the guaranteed container to be executed, and then start its execution.
+    * If the container is opportunistic, it remains at the queue until resources become available.
+* When a container (guaranteed or opportunistic) finishes its execution and resources get freed up, we examine the queued containers and if there are available resources we start their execution. We pick containers from the queue in a FIFO order.
+
+In the [future work items](#Items_for_Future_Work) below, we discuss different ways of prioritizing task execution (queue reordering) and of killing opportunistic containers to make space for guaranteed ones.
+
+
+Allocation of Opportunistic Containers
+--------------------------------------
+
+As mentioned above, we provide both a centralized and a distributed way of allocating opportunistic containers, which we describe below.
+
+$H3 Centralized Allocation
+
+We have introduced a new service at the RM, namely the `OpportunisticContainerAllocatorAMService`, which extends the `ApplicationMasterService`. When the centralized opportunistic allocation is enabled, the resource requests from the AMs are served at the RM side by the `OpportunisticContainerAllocatorAMService`, which splits them into two sets of resource requests: 
+
+* The guaranteed set is forwarded to the existing `ApplicationMasterService` and is subsequently handled by the Fair or Capacity Scheduler.
+* The opportunistic set is handled by the new `OpportunisticContainerAllocator`, which performs the scheduling of opportunistic containers to nodes.
+
+The `OpportunisticContainerAllocator` maintains a list with the [least loaded nodes](#Determining_Nodes_for_Allocation) of the cluster at each moment, and assigns containers to them in a round-robin fashion. Note that in the current implementation, we purposely do not take into account node locality constraints. Since an opportunistic container (unlike the guaranteed ones) might wait at the queue of an NM before its execution starts, it is more important to allocate it at a node that is less loaded (i.e., where queuing delay will be smaller) rather than respect its locality constraints. Moreover, we do not take into account sharing (fairness/capacity) constraints for opportunistic containers at the moment. Support for both locality and sharing constraints can be added in the future if required.
+
+
+$H3 Distributed Allocation
+
+In order to enable distributed scheduling of opportunistic containers, we have introduced a new service at each NM, called `AMRMProxyService`. The `AMRMProxyService` implements the `ApplicationMasterService` protocol, and acts as a proxy between the AMs running at that node and the RM. When the `AMRMProxyService` is enabled (through a parameter), we force all AMs running at a particular node to communicate with the `AMRMProxyService` of the same node, instead of going directly to the RM. Moreover, to ensure that the AMs will not talk directly with the RM, when a new AM gets initialized, we replace its `AMRMToken` with a token signed by the `AMRMProxyService`.
+
+A chain of interceptors can be registered with the `AMRMProxyService`. One of these interceptors is the `DistributedScheduler` that is responsible for allocating opportunistic containers in a distributed way, without needing to contact the RM. This modular design makes the `AMRMProxyService` instrumental in other scenarios too, such as YARN federation (`YARN-2915`) or throttling down misbehaving AMs, which can be enabled simply by adding additional interceptors at the interceptor chain.
+
+When distributed opportunistic scheduling is enabled, each AM sends its resource requests to the `AMRMProxyService` running at the same node. The `AMRMProxyService` splits the resource requests into two sets:
+
+* The guaranteed set is forwarded to the RM. In this case the `AMRMProxyService` simply acts as a proxy between the AM and the RM, and the container allocation remains intact (using the Fair or Capacity Scheduler).
+* The opportunistic set is not forwarded to the RM. Instead, it is handled by the `DistributedScheduler` that is running locally at the node. In particular, the `DistributedScheduler` maintains a list with the least loaded nodes in the cluster, and allocates containers to them in a round-robin fashion. The RM informs the `DistributedScheduler` about the least loaded nodes at regular intervals through the NM-RM heartbeats.
+
+The above procedure is similar to the one performed by the `OpportunisticContainerAllocatorAMService` in the case of centralized opportunistic scheduling described above. The main difference is that in the distributed case, the splitting of requests into guaranteed and opportunistic happens locally at the node, and only the guaranteed requests are forwarded to the RM, while the opportunistic ones are handled without contacting the RM.
+
+
+$H3 Determining Nodes for Allocation
+
+Each NM informs the RM periodically through the NM-RM heartbeats about the number of running guaranteed and opportunistic containers, as well as the number of queued opportunistic containers. The RM gathers this information from all nodes and determines the least loaded ones.
+
+In the case of centralized allocation of opportunistic containers, this information is immediately available, since the allocation happens centrally. In the case of distributed scheduling, the list with the least loaded nodes is propagated to all NMs (and thus becomes available to the `DistributedSchedulers`) through the heartbeat responses from the RM to the NMs. The number of least loaded nodes sent to the NMs is configurable.
+
+At the moment, we take into account only the number of queued opportunistic containers at each node in order to estimate the time an opportunistic container would have to wait if sent to that node and, thus, determine the least loaded nodes. If the AM provided us with information about the estimated task durations, we could take them into account in order to have better estimates of the queue waiting times.
+
+
+$H3 Rebalancing Node Load
+
+Occasionally poor placement choices for opportunistic containers may be made (due to stale queue length estimates), which can lead to load imbalance between nodes. The problem is more pronounced under high cluster load, and also in the case of distributed scheduling (multiple `DistributedSchedulers` may place containers at the same NM, since they do not coordinate with each other). To deal with this load imbalance between the NM queues, we perform load shedding to dynamically re-balance the load between NMs. In particular, while aggregating at the RM the queue time estimates published by each NM, we construct a distribution and find a targeted maximal value for the length of the NM queues (based on the mean and standard deviation of the distribution). Then the RM disseminates this value to the various NMs through the heartbeat responses. Subsequently, using this information, an NM on a node whose queue length is above the threshold discards opportunistic containers to meet this maxi
 mal value. This forces the associated individual AMs to reschedule those containers elsewhere.
+
+
+Advanced Configuration
+----------------------
+
+The main properties for enabling opportunistic container allocation and choosing between centralized and distributed allocation were described in the [quick guide](#Quick_Guide) in the beginning of this document. Here we present more advanced configuration. Note that using default values for those parameters should be sufficient in most cases. All parameters below have to be defined in the **conf/yarn-site.xml** file.
+
+To determine the number of [least loaded nodes](#Determining_Nodes_for_Allocation) that will be used when scheduling opportunistic containers and how often this list will be refreshed, we use the following parameters:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanager.opportunistic-container-allocation.nodes-used` | Number of least loaded nodes to be used by the Opportunistic Container allocator for dispatching containers during container allocation. A higher value can improve load balance in large clusters. | `10` |
+| `yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms` | Frequency for computing least loaded nodes. | `1000` |
+
+
+As discussed in the [node load rebalancing](#Rebalancing_Node_Load) section above, at regular intervals, the RM gathers all NM queue lengths and computes their mean value (`avg`) and standard deviation (`stdev`), as well as the value `avg + k*stdev` (where `k` a float). This value gets propagated through the NM-RM heartbeats to all NMs, who should respect that value by dequeuing containers (if required), as long as their current queue length is between a `queue_min_length` and a `queue_max_length` value (these values are used to avoid dequeuing tasks from very short queues and to aggressively dequeue tasks from long queues, respectively). 
+The parameters `k`, `queue_min_length` and `queue_max_length` can be specified as follows:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanager.nm-container-queuing.queue-limit-stdev` | The `k` parameter. | `1.0f` |
+| `yarn.resourcemanager.nm-container-queuing.min-queue-length` | The `queue_min_length` parameter. | `5` |
+| `yarn.resourcemanager.nm-container-queuing.max-queue-length` | The `queue_max_length` parameter. | `15` |
+
+
+Finally, two more properties can further tune the `AMRMProxyService` in case distributed scheduling is used:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.nodemanager.amrmproxy.address` | The address/port to which the `AMRMProxyService` is bound to. | `0.0.0.0:8049` |
+| `yarn.nodemanager.amrmproxy.client.thread-count` | The number of threads that are used at each NM for serving the interceptors register to the `AMRMProxyService` by different jobs. | `3` |
+
+
+Items for Future Work
+---------------------
+
+Here we describe multiple ways in which we can extend/enhance the allocation and execution of opportunistic containers. We also provide the JIRAs that track each item.
+
+* **Resource overcommitment** (`YARN-1011`). As already discussed, in order to further improve the cluster resource utilization, we can schedule containers not based on the allocated resources but on the actually utilized ones. When over-committing resources, there is the risk of running out of resources in case we have an increase in the utilized resources of the already running containers. Therefore, opportunistic execution should be used for containers whose allocation goes beyond the capacity of a node. This way, we can choose opportunistic containers to kill for reclaiming resources.
+* **NM Queue reordering** (`YARN-5886`). Instead of executing queued containers in a FIFO order, we can employ reordering strategies that dynamically determine which opportunistic container will be executed next. For example, we can prioritize containers that are expected to be short-running or which belong to applications that are close to completion.
+* **Out of order killing at NMs** (`YARN-5887`). As described above, when we need to free up resources for a guaranteed container to start its execution, we kill opportunistic containers in reverse order of arrival (first the most recently started ones). This might not always be the right decision. For example, we might want to minimize the number of containers killed or to refrain from killing containers of jobs that are very close to completion.
+* **Container pausing** (`YARN-5292`): At the moment we kill opportunistic containers to make room for guaranteed in case of resource contention. In busy clusters this can lower the effective cluster utilization: whenever we kill a running opportunistic container, it has to be restarted, and thus we lose work. To this end, we can instead pause running opportunistic containers. Note that this will require support from the container executor (e.g., the container technology used) and from the application.
+* **Container promotion** (`YARN-5085`). There are cases where changing the execution type of a container during its execution can be beneficial. For instance, an application might submit a container as opportunistic, and when its execution starts, it can request its promotion to a guaranteed container to avoid it getting killed.


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


[43/50] [abbrv] hadoop git commit: YARN-7591. NPE in async-scheduling mode of CapacityScheduler. (Tao Yang via wangda)

Posted by wa...@apache.org.
YARN-7591. NPE in async-scheduling mode of CapacityScheduler. (Tao Yang via wangda)

Change-Id: I46689e530550ee0a6ac7a29786aab2cc1bdf314f


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

Branch: refs/heads/YARN-6592
Commit: adca1a72e4eca2ea634551e9fb8e9b878c36cb5c
Parents: a8316df
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Dec 8 15:17:02 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Dec 8 15:17:02 2017 -0800

----------------------------------------------------------------------
 .../scheduler/capacity/CapacityScheduler.java   |  4 ++++
 .../scheduler/capacity/LeafQueue.java           | 21 +++++++++++++++++++-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  5 +++++
 3 files changed, 29 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/adca1a72/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 a5efd9f..8de3631 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
@@ -1301,6 +1301,10 @@ public class CapacityScheduler extends
     if (reservedContainer != null) {
       FiCaSchedulerApp reservedApplication = getCurrentAttemptForContainer(
           reservedContainer.getContainerId());
+      if (reservedApplication == null) {
+        LOG.error("Trying to schedule for a finished app, please double check.");
+        return null;
+      }
 
       // Try to fulfill the reservation
       LOG.info(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adca1a72/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 86fcbc9..ac1a26c 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
@@ -1201,7 +1201,14 @@ public class LeafQueue extends AbstractCSQueue {
             allocation.getSchedulingMode(), null);
 
         // Deduct resources that we can release
-        Resource usedResource = Resources.clone(getUser(username).getUsed(p));
+        User user = getUser(username);
+        if (user == null) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("User " + username + " has been removed!");
+          }
+          return false;
+        }
+        Resource usedResource = Resources.clone(user.getUsed(p));
         Resources.subtractFrom(usedResource,
             request.getTotalReleasedResource());
 
@@ -1406,6 +1413,12 @@ public class LeafQueue extends AbstractCSQueue {
       SchedulingMode schedulingMode, Resource userLimit) {
     String user = application.getUser();
     User queueUser = getUser(user);
+    if (queueUser == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("User " + user + " has been removed!");
+      }
+      return Resources.none();
+    }
 
     // Compute user limit respect requested labels,
     // TODO, need consider headroom respect labels also
@@ -1500,6 +1513,12 @@ public class LeafQueue extends AbstractCSQueue {
     try {
       readLock.lock();
       User user = getUser(userName);
+      if (user == null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("User " + userName + " has been removed!");
+        }
+        return false;
+      }
 
       currentResourceLimits.setAmountNeededUnreserve(Resources.none());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adca1a72/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 a4e0096..776a7e9 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
@@ -322,6 +322,11 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     RMContainer reservedContainerOnNode =
         schedulerContainer.getSchedulerNode().getReservedContainer();
     if (reservedContainerOnNode != null) {
+      // adding NP check as this proposal could not be allocated from reserved
+      // container in async-scheduling mode
+      if (allocation.getAllocateFromReservedContainer() == null) {
+        return false;
+      }
       RMContainer fromReservedContainer =
           allocation.getAllocateFromReservedContainer().getRmContainer();
 


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


[10/50] [abbrv] hadoop git commit: MAPREDUCE-6998. Moving logging APIs over to slf4j in hadoop-mapreduce-client-jobclient. Contributed by Gergely Novák.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
index 06e37dd..289c17e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
@@ -24,8 +24,6 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -53,6 +51,8 @@ import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Configures and starts the MR-specific components in the YARN cluster.
@@ -64,7 +64,8 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
 
   public static final String APPJAR = JarFinder.getJar(LocalContainerLauncher.class);
 
-  private static final Log LOG = LogFactory.getLog(MiniMRYarnCluster.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MiniMRYarnCluster.class);
   private JobHistoryServer historyServer;
   private JobHistoryServerWrapper historyServerWrapper;
   private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAMWithNonNormalizedCapabilities.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAMWithNonNormalizedCapabilities.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAMWithNonNormalizedCapabilities.java
index 7aaaa1b..8eb673c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAMWithNonNormalizedCapabilities.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAMWithNonNormalizedCapabilities.java
@@ -23,8 +23,6 @@ import java.io.IOException;
 
 import org.junit.Assert;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,9 +35,12 @@ import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestMRAMWithNonNormalizedCapabilities {
-  private static final Log LOG = LogFactory.getLog(TestMRAMWithNonNormalizedCapabilities.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMRAMWithNonNormalizedCapabilities.class);
   private static FileSystem localFs;
   protected static MiniMRYarnCluster mrCluster = null;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAppWithCombiner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAppWithCombiner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAppWithCombiner.java
index a96befa..0f39168 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAppWithCombiner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAppWithCombiner.java
@@ -23,8 +23,6 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.CustomOutputCommitter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -47,6 +45,8 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @SuppressWarnings("deprecation")
 public class TestMRAppWithCombiner {
@@ -54,7 +54,8 @@ public class TestMRAppWithCombiner {
   protected static MiniMRYarnCluster mrCluster;
   private static Configuration conf = new Configuration();
   private static FileSystem localFs;
-  private static final Log LOG = LogFactory.getLog(TestMRAppWithCombiner.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMRAppWithCombiner.class);
 
   static {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
index 6e280ad..c8b4003 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
@@ -36,8 +36,6 @@ import java.util.jar.JarOutputStream;
 import java.util.zip.ZipEntry;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.FailingMapper;
 import org.apache.hadoop.RandomTextWriterJob;
 import org.apache.hadoop.RandomTextWriterJob.RandomInputFormat;
@@ -104,10 +102,12 @@ import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestMRJobs {
 
-  private static final Log LOG = LogFactory.getLog(TestMRJobs.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestMRJobs.class);
   private static final EnumSet<RMAppState> TERMINAL_RM_APP_STATES =
       EnumSet.of(RMAppState.FINISHED, RMAppState.FAILED, RMAppState.KILLED);
   private static final int NUM_NODE_MGRS = 3;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
index 98a6de2..64cb253 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
@@ -26,8 +26,6 @@ import java.util.List;
 import org.junit.Assert;
 
 import org.apache.avro.AvroRemoteException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,11 +50,13 @@ import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestMRJobsWithHistoryService {
 
-  private static final Log LOG =
-    LogFactory.getLog(TestMRJobsWithHistoryService.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMRJobsWithHistoryService.class);
 
   private static final EnumSet<RMAppState> TERMINAL_RM_APP_STATES =
     EnumSet.of(RMAppState.FINISHED, RMAppState.FAILED, RMAppState.KILLED);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java
index cbf1f39..9fa8828 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java
@@ -27,8 +27,6 @@ import java.util.regex.Pattern;
 import org.junit.AfterClass;
 import org.junit.Assert;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
@@ -42,11 +40,13 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestMRJobsWithProfiler {
 
-  private static final Log LOG =
-    LogFactory.getLog(TestMRJobsWithProfiler.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMRJobsWithProfiler.class);
 
   private static final EnumSet<RMAppState> TERMINAL_RM_APP_STATES =
     EnumSet.of(RMAppState.FINISHED, RMAppState.FAILED, RMAppState.KILLED);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMROldApiJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMROldApiJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMROldApiJobs.java
index 7af5922..92c65e7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMROldApiJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMROldApiJobs.java
@@ -22,8 +22,6 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.CustomOutputCommitter;
 import org.apache.hadoop.FailMapper;
 import org.apache.hadoop.conf.Configuration;
@@ -46,10 +44,13 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestMROldApiJobs {
 
-  private static final Log LOG = LogFactory.getLog(TestMROldApiJobs.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMROldApiJobs.class);
 
   protected static MiniMRYarnCluster mrCluster;
   private static Configuration conf = new Configuration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestRMNMInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestRMNMInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestRMNMInfo.java
index 0e55529..efea709 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestRMNMInfo.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestRMNMInfo.java
@@ -26,8 +26,6 @@ import java.util.concurrent.ConcurrentMap;
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -42,10 +40,13 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.mockito.Mockito.*;
 
 public class TestRMNMInfo {
-  private static final Log LOG = LogFactory.getLog(TestRMNMInfo.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestRMNMInfo.class);
   private static final int NUMNODEMANAGERS = 4;
   protected static MiniMRYarnCluster mrCluster;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecution.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecution.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecution.java
index 17d82d6..fe21f07 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecution.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecution.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.mapreduce.v2;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -48,6 +46,8 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestSpeculativeExecution {
 
@@ -82,7 +82,8 @@ public class TestSpeculativeExecution {
     }
   }
 
-  private static final Log LOG = LogFactory.getLog(TestSpeculativeExecution.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSpeculativeExecution.class);
 
   protected static MiniMRYarnCluster mrCluster;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
index e198f99..24192ea 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.mapreduce.v2;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobCounter;
@@ -35,10 +33,12 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestUberAM extends TestMRJobs {
 
-  private static final Log LOG = LogFactory.getLog(TestUberAM.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestUberAM.class);
 
   @BeforeClass
   public static void setup() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/testjar/UserNamePermission.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/testjar/UserNamePermission.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/testjar/UserNamePermission.java
index 8ca2e16..f339edb 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/testjar/UserNamePermission.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/testjar/UserNamePermission.java
@@ -21,28 +21,23 @@ package testjar;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public  class UserNamePermission      
 {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(UserNamePermission.class);
 
-  private static final Log LOG = LogFactory.getLog(UserNamePermission.class);
   //This mapper will read the user name and pass in to the reducer
   public static class UserNameMapper extends Mapper<LongWritable,Text,Text,Text>
   {


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


[36/50] [abbrv] hadoop git commit: YARN-7274. Ability to disable elasticity at leaf queue level. (Zian Chen via wangda)

Posted by wa...@apache.org.
YARN-7274. Ability to disable elasticity at leaf queue level. (Zian Chen via wangda)

Change-Id: Ic8d43e297f0f5de788b562f7eff8106c5c35e8d2


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

Branch: refs/heads/YARN-6592
Commit: 74665e3a7d7f05644d9a5abad5a3f2d47597d6c8
Parents: ef7d334
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Dec 8 15:07:56 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Dec 8 15:07:56 2017 -0800

----------------------------------------------------------------------
 .../scheduler/capacity/CSQueueUtils.java        |  16 +-
 .../TestNodeLabelContainerAllocation.java       | 163 ++++++++++++++++++
 .../scheduler/capacity/TestQueueParsing.java    | 168 ++++++++++++++++++-
 3 files changed, 329 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/74665e3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.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/CSQueueUtils.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/CSQueueUtils.java
index 6daca51..51e5b17 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/CSQueueUtils.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/CSQueueUtils.java
@@ -66,19 +66,9 @@ class CSQueueUtils {
   private static void capacitiesSanityCheck(String queueName,
       QueueCapacities queueCapacities) {
     for (String label : queueCapacities.getExistingNodeLabels()) {
-      float capacity = queueCapacities.getCapacity(label);
-      float maximumCapacity = queueCapacities.getMaximumCapacity(label);
-      if (capacity > maximumCapacity) {
-        throw new IllegalArgumentException("Illegal queue capacity setting, "
-            + "(capacity=" + capacity + ") > (maximum-capacity="
-            + maximumCapacity + "). When label=[" + label + "]");
-      }
-     
-      // Actually, this may not needed since we have verified capacity <=
-      // maximumCapacity. And the way we compute absolute capacity (abs(x) =
-      // cap(x) * cap(x.parent) * ...) is a monotone increasing function. But
-      // just keep it here to make sure our compute abs capacity method works
-      // correctly. 
+      // The only thing we should care about is absolute capacity <=
+      // absolute max capacity otherwise the absolute max capacity is
+      // no longer an absolute maximum.
       float absCapacity = queueCapacities.getAbsoluteCapacity(label);
       float absMaxCapacity = queueCapacities.getAbsoluteMaximumCapacity(label);
       if (absCapacity > absMaxCapacity) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74665e3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.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/TestNodeLabelContainerAllocation.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/TestNodeLabelContainerAllocation.java
index 71fddfc..1836919 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/TestNodeLabelContainerAllocation.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/TestNodeLabelContainerAllocation.java
@@ -2083,4 +2083,167 @@ public class TestNodeLabelContainerAllocation {
 
     rm1.close();
   }
+
+  @Test
+  public void testQueueMetricsWithLabelsDisableElasticity() throws Exception {
+    /**
+     * Test case: have a following queue structure:
+     *
+     * <pre>
+     *
+     *          root
+     *        /      \
+     *       a        b
+     *      (x)      (x)
+     *      / \
+     *     a1 a2
+     *    (x) (x)
+     * </pre>
+     *
+     * a/b can access x, both of them has max-capacity-on-x = 50
+     *
+     * When doing non-exclusive allocation, app in a (or b) can use 100% of x
+     * resource.
+     */
+
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(
+        this.conf);
+
+    // Define top-level queues
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] { "a", "b" });
+    csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+
+    final String queueA = CapacitySchedulerConfiguration.ROOT + ".a";
+    csConf.setCapacity(queueA, 50);
+    csConf.setMaximumCapacity(queueA, 100);
+    csConf.setAccessibleNodeLabels(queueA, toSet("x"));
+    csConf.setCapacityByLabel(queueA, "x", 50);
+    csConf.setMaximumCapacityByLabel(queueA, "x", 100);
+    final String queueB = CapacitySchedulerConfiguration.ROOT + ".b";
+    csConf.setCapacity(queueB, 50);
+    csConf.setMaximumCapacity(queueB, 100);
+    csConf.setAccessibleNodeLabels(queueB, toSet("x"));
+    csConf.setCapacityByLabel(queueB, "x", 50);
+    csConf.setMaximumCapacityByLabel(queueB, "x", 100);
+
+    // Define 2nd-level queues
+    csConf.setQueues(queueA, new String[] { "a1",
+        "a2"});
+
+    final String A1 = queueA + ".a1";
+    csConf.setCapacity(A1, 20);
+    csConf.setMaximumCapacity(A1, 60);
+    csConf.setAccessibleNodeLabels(A1, toSet("x"));
+    csConf.setCapacityByLabel(A1, "x", 60);
+    csConf.setMaximumCapacityByLabel(A1, "x", 30);
+
+    final String A2 = queueA + ".a2";
+    csConf.setCapacity(A2, 80);
+    csConf.setMaximumCapacity(A2, 40);
+    csConf.setAccessibleNodeLabels(A2, toSet("x"));
+    csConf.setCapacityByLabel(A2, "x", 40);
+    csConf.setMaximumCapacityByLabel(A2, "x", 20);
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(
+        ImmutableSet.of(NodeLabel.newInstance("x", false)));
+    mgr.addLabelsToNode(
+        ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 20 * GB); // label = x
+    // app1 -> a1
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a1", "x");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // app1 asks for 6 partition=x containers
+    am1.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>(), "x");
+
+    // NM1 do 50 heartbeats
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    doNMHeartbeat(rm1, nm1.getNodeId(), 50);
+    checkNumOfContainersInAnAppOnGivenNode(6, nm1.getNodeId(),
+        cs.getApplicationAttempt(am1.getApplicationAttemptId()));
+
+    SchedulerNodeReport reportNm1 = rm1.getResourceScheduler()
+        .getNodeReport(nm1.getNodeId());
+    Assert.assertEquals(6 * GB, reportNm1.getUsedResource().getMemorySize());
+    Assert.assertEquals(14 * GB,
+        reportNm1.getAvailableResource().getMemorySize());
+
+    // Try to launch app2 in a2, asked 2GB, should success
+    // app2 -> a2
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "a2", "x");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    // app2 asks for 4 partition=x containers
+    am2.allocate("*", 1 * GB, 4, new ArrayList<ContainerId>(), "x");
+    // NM1 do 50 heartbeats
+
+    doNMHeartbeat(rm1, nm1.getNodeId(), 50);
+    checkNumOfContainersInAnAppOnGivenNode(4, nm1.getNodeId(),
+        cs.getApplicationAttempt(am2.getApplicationAttemptId()));
+
+    reportNm1 = rm1.getResourceScheduler()
+        .getNodeReport(nm1.getNodeId());
+    Assert.assertEquals(10 * GB, reportNm1.getUsedResource().getMemorySize());
+    Assert.assertEquals(10 * GB,
+        reportNm1.getAvailableResource().getMemorySize());
+
+    // Kill all apps in queue a2
+    cs.killAllAppsInQueue("a2");
+    rm1.waitForState(app2.getApplicationId(), RMAppState.KILLED);
+    rm1.waitForAppRemovedFromScheduler(app2.getApplicationId());
+
+    // Try to launch app3 in a2, asked 6GB, should fail
+    // app3 -> a2
+    RMApp app3 = rm1.submitApp(1 * GB, "app", "user", null, "a2", "x");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm1);
+
+    am3.allocate("*", 1 * GB, 6, new ArrayList<ContainerId>(), "x");
+    // NM1 do 50 heartbeats
+    doNMHeartbeat(rm1, nm1.getNodeId(), 50);
+    // app3 cannot preempt more resources restricted by disable elasticity
+    checkNumOfContainersInAnAppOnGivenNode(4, nm1.getNodeId(),
+        cs.getApplicationAttempt(am3.getApplicationAttemptId()));
+
+    Assert.assertEquals(10 * GB, reportNm1.getUsedResource().getMemorySize());
+    Assert.assertEquals(10 * GB,
+        reportNm1.getAvailableResource().getMemorySize());
+
+    // Kill all apps in queue a1
+    cs.killAllAppsInQueue("a1");
+    rm1.waitForState(app1.getApplicationId(), RMAppState.KILLED);
+    rm1.waitForAppRemovedFromScheduler(app1.getApplicationId());
+
+    // app4 -> a1, try to allocate more than 6GB resource, should fail
+    RMApp app4 = rm1.submitApp(1 * GB, "app", "user", null, "a1", "x");
+    MockAM am4 = MockRM.launchAndRegisterAM(app4, rm1, nm1);
+
+    // app3 asks for 7 partition=x containers
+    am4.allocate("*", 1 * GB, 7, new ArrayList<ContainerId>(), "x");
+    // NM1 do 50 heartbeats
+    doNMHeartbeat(rm1, nm1.getNodeId(), 50);
+
+    // app4 should only gets 6GB resource in partition=x
+    // since elasticity is disabled
+    checkNumOfContainersInAnAppOnGivenNode(6, nm1.getNodeId(),
+        cs.getApplicationAttempt(am4.getApplicationAttemptId()));
+
+    Assert.assertEquals(10 * GB, reportNm1.getUsedResource().getMemorySize());
+    Assert.assertEquals(10 * GB,
+        reportNm1.getAvailableResource().getMemorySize());
+
+    rm1.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74665e3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.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/TestQueueParsing.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/TestQueueParsing.java
index 5d167c7..add14ab 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/TestQueueParsing.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/TestQueueParsing.java
@@ -365,7 +365,76 @@ public class TestQueueParsing {
     conf.setCapacityByLabel(B3, "red", 25);
     conf.setCapacityByLabel(B3, "blue", 25);
   }
-  
+
+  private void setupQueueConfigurationWithLabelsAndReleaseCheck
+      (CapacitySchedulerConfiguration conf) {
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "red", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "blue", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    // The cap <= max-cap check is not needed
+    conf.setCapacity(A, 50);
+    conf.setMaximumCapacity(A, 100);
+
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 50);
+    conf.setMaximumCapacity(B, 100);
+
+    LOG.info("Setup top-level queues");
+
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    final String A2 = A + ".a2";
+    conf.setQueues(A, new String[] {"a1", "a2"});
+    conf.setAccessibleNodeLabels(A, ImmutableSet.of("red", "blue"));
+    conf.setCapacityByLabel(A, "red", 50);
+    conf.setMaximumCapacityByLabel(A, "red", 100);
+    conf.setCapacityByLabel(A, "blue", 30);
+    conf.setMaximumCapacityByLabel(A, "blue", 50);
+
+    conf.setCapacity(A1, 60);
+    conf.setMaximumCapacity(A1, 60);
+    conf.setCapacityByLabel(A1, "red", 60);
+    conf.setMaximumCapacityByLabel(A1, "red", 30);
+    conf.setCapacityByLabel(A1, "blue", 100);
+    conf.setMaximumCapacityByLabel(A1, "blue", 100);
+
+    conf.setCapacity(A2, 40);
+    conf.setMaximumCapacity(A2, 85);
+    conf.setAccessibleNodeLabels(A2, ImmutableSet.of("red"));
+    conf.setCapacityByLabel(A2, "red", 40);
+    conf.setMaximumCapacityByLabel(A2, "red", 60);
+
+    final String B1 = B + ".b1";
+    final String B2 = B + ".b2";
+    final String B3 = B + ".b3";
+    conf.setQueues(B, new String[] {"b1", "b2", "b3"});
+    conf.setAccessibleNodeLabels(B, ImmutableSet.of("red", "blue"));
+    conf.setCapacityByLabel(B, "red", 50);
+    conf.setMaximumCapacityByLabel(B, "red", 100);
+    conf.setCapacityByLabel(B, "blue", 70);
+    conf.setMaximumCapacityByLabel(B, "blue", 100);
+
+    conf.setCapacity(B1, 10);
+    conf.setMaximumCapacity(B1, 10);
+    conf.setCapacityByLabel(B1, "red", 60);
+    conf.setMaximumCapacityByLabel(B1, "red", 30);
+    conf.setCapacityByLabel(B1, "blue", 50);
+    conf.setMaximumCapacityByLabel(B1, "blue", 100);
+
+    conf.setCapacity(B2, 80);
+    conf.setMaximumCapacity(B2, 40);
+    conf.setCapacityByLabel(B2, "red", 30);
+    conf.setCapacityByLabel(B2, "blue", 25);
+
+    conf.setCapacity(B3, 10);
+    conf.setMaximumCapacity(B3, 25);
+    conf.setCapacityByLabel(B3, "red", 10);
+    conf.setCapacityByLabel(B3, "blue", 25);
+  }
+
   private void setupQueueConfigurationWithLabelsInherit(
       CapacitySchedulerConfiguration conf) {
     // Define top-level queues
@@ -472,7 +541,7 @@ public class TestQueueParsing {
     // queue-B2 inherits "red"/"blue"
     Assert.assertTrue(capacityScheduler.getQueue("b2")
         .getAccessibleNodeLabels().containsAll(ImmutableSet.of("red", "blue")));
-    
+
     // check capacity of A2
     CSQueue qA2 = capacityScheduler.getQueue("a2");
     Assert.assertEquals(0.7, qA2.getCapacity(), DELTA);
@@ -481,7 +550,7 @@ public class TestQueueParsing {
     Assert.assertEquals(0.25, qA2.getQueueCapacities().getAbsoluteCapacity("red"), DELTA);
     Assert.assertEquals(0.1275, qA2.getAbsoluteMaximumCapacity(), DELTA);
     Assert.assertEquals(0.3, qA2.getQueueCapacities().getAbsoluteMaximumCapacity("red"), DELTA);
-    
+
     // check capacity of B3
     CSQueue qB3 = capacityScheduler.getQueue("b3");
     Assert.assertEquals(0.18, qB3.getAbsoluteCapacity(), DELTA);
@@ -489,7 +558,71 @@ public class TestQueueParsing {
     Assert.assertEquals(0.35, qB3.getAbsoluteMaximumCapacity(), DELTA);
     Assert.assertEquals(1, qB3.getQueueCapacities().getAbsoluteMaximumCapacity("red"), DELTA);
   }
-  
+
+  private void checkQueueLabelsWithLeafQueueDisableElasticity
+      (CapacityScheduler capacityScheduler) {
+    // queue-A is red, blue
+    Assert.assertTrue(capacityScheduler.getQueue("a").getAccessibleNodeLabels()
+        .containsAll(ImmutableSet.of("red", "blue")));
+
+    // queue-A1 inherits A's configuration
+    Assert.assertTrue(capacityScheduler.getQueue("a1")
+        .getAccessibleNodeLabels().containsAll(ImmutableSet.of("red", "blue")));
+
+    // queue-A2 is "red"
+    Assert.assertEquals(1, capacityScheduler.getQueue("a2")
+        .getAccessibleNodeLabels().size());
+    Assert.assertTrue(capacityScheduler.getQueue("a2")
+        .getAccessibleNodeLabels().contains("red"));
+
+    // queue-B is "red"/"blue"
+    Assert.assertTrue(capacityScheduler.getQueue("b").getAccessibleNodeLabels()
+        .containsAll(ImmutableSet.of("red", "blue")));
+
+    // queue-B2 inherits "red"/"blue"
+    Assert.assertTrue(capacityScheduler.getQueue("b2")
+        .getAccessibleNodeLabels().containsAll(ImmutableSet.of("red", "blue")));
+
+    // check capacity of A2
+    CSQueue qA2 = capacityScheduler.getQueue("a2");
+    Assert.assertEquals(0.4, qA2.getCapacity(), DELTA);
+    Assert.assertEquals(0.4, qA2.getQueueCapacities()
+        .getCapacity("red"), DELTA);
+    Assert.assertEquals(0.2, qA2.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.2, qA2.getQueueCapacities()
+        .getAbsoluteCapacity("red"), DELTA);
+    Assert.assertEquals(0.85, qA2.getAbsoluteMaximumCapacity(), DELTA);
+    Assert.assertEquals(0.6, qA2.getQueueCapacities()
+        .getAbsoluteMaximumCapacity("red"), DELTA);
+
+    // check disable elasticity at leaf queue level without label
+    CSQueue qB2 = capacityScheduler.getQueue("b2");
+    Assert.assertEquals(0.4, qB2.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.4, qB2.getAbsoluteMaximumCapacity(), DELTA);
+
+    // check disable elasticity at leaf queue level with label
+    CSQueue qA1 = capacityScheduler.getQueue("a1");
+    Assert.assertEquals(0.3, qA1.getQueueCapacities().
+        getAbsoluteCapacity("red"), DELTA);
+    Assert.assertEquals(0.3, qA1.getQueueCapacities().
+        getAbsoluteMaximumCapacity("red"), DELTA);
+
+    CSQueue qB1 = capacityScheduler.getQueue("b1");
+    Assert.assertEquals(0.3, qB1.getQueueCapacities()
+        .getAbsoluteCapacity("red"), DELTA);
+    Assert.assertEquals(0.3, qB1.getQueueCapacities()
+        .getAbsoluteMaximumCapacity("red"), DELTA);
+
+    // check capacity of B3
+    CSQueue qB3 = capacityScheduler.getQueue("b3");
+    Assert.assertEquals(0.05, qB3.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.175, qB3.getQueueCapacities()
+        .getAbsoluteCapacity("blue"), DELTA);
+    Assert.assertEquals(0.25, qB3.getAbsoluteMaximumCapacity(), DELTA);
+    Assert.assertEquals(1, qB3.getQueueCapacities()
+        .getAbsoluteMaximumCapacity("blue"), DELTA);
+  }
+
   private void
       checkQueueLabelsInheritConfig(CapacityScheduler capacityScheduler) {
     // queue-A is red, blue
@@ -514,7 +647,7 @@ public class TestQueueParsing {
   @Test
   public void testQueueParsingWithLabels() throws IOException {
     nodeLabelManager.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("red", "blue"));
-    
+
     YarnConfiguration conf = new YarnConfiguration();
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration(conf);
@@ -534,6 +667,31 @@ public class TestQueueParsing {
     checkQueueLabels(capacityScheduler);
     ServiceOperations.stopQuietly(capacityScheduler);
   }
+
+  @Test
+  public void testQueueParsingWithLeafQueueDisableElasticity()
+      throws IOException {
+    nodeLabelManager.addToCluserNodeLabelsWithDefaultExclusivity
+        (ImmutableSet.of("red", "blue"));
+
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(conf);
+    setupQueueConfigurationWithLabelsAndReleaseCheck(csConf);
+    CapacityScheduler capacityScheduler = new CapacityScheduler();
+    RMContextImpl rmContext =
+        new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(csConf),
+            new NMTokenSecretManagerInRM(csConf),
+            new ClientToAMTokenSecretManagerInRM(), null);
+    rmContext.setNodeLabelManager(nodeLabelManager);
+    capacityScheduler.setConf(csConf);
+    capacityScheduler.setRMContext(rmContext);
+    capacityScheduler.init(csConf);
+    capacityScheduler.start();
+    checkQueueLabelsWithLeafQueueDisableElasticity(capacityScheduler);
+    ServiceOperations.stopQuietly(capacityScheduler);
+  }
   
   @Test
   public void testQueueParsingWithLabelsInherit() throws IOException {


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


[23/50] [abbrv] hadoop git commit: YARN-6471. Support to add min/max resource configuration for a queue. (Sunil G via wangda)

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/policy/PriorityUtilizationQueueOrderingPolicy.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/policy/PriorityUtilizationQueueOrderingPolicy.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/policy/PriorityUtilizationQueueOrderingPolicy.java
index 0544387..4985a1a 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/policy/PriorityUtilizationQueueOrderingPolicy.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/policy/PriorityUtilizationQueueOrderingPolicy.java
@@ -20,9 +20,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -121,6 +123,15 @@ public class PriorityUtilizationQueueOrderingPolicy implements QueueOrderingPoli
       // For queue with same used ratio / priority, queue with higher configured
       // capacity goes first
       if (0 == rc) {
+        Resource minEffRes1 = q1.getQueueResourceQuotas()
+            .getConfiguredMinResource(p);
+        Resource minEffRes2 = q2.getQueueResourceQuotas()
+            .getConfiguredMinResource(p);
+        if (!minEffRes1.equals(Resources.none())
+            && !minEffRes2.equals(Resources.none())) {
+          return minEffRes2.compareTo(minEffRes1);
+        }
+
         float abs1 = q1.getQueueCapacities().getAbsoluteCapacity(p);
         float abs2 = q2.getQueueCapacities().getAbsoluteCapacity(p);
         return Float.compare(abs2, abs1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
index 22705cc..86b2fea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
@@ -62,6 +62,8 @@ public class CapacitySchedulerQueueInfo {
   protected long pendingContainers;
   protected QueueCapacitiesInfo capacities;
   protected ResourcesInfo resources;
+  protected ResourceInfo minEffectiveCapacity;
+  protected ResourceInfo maxEffectiveCapacity;
 
   CapacitySchedulerQueueInfo() {
   };
@@ -105,6 +107,11 @@ public class CapacitySchedulerQueueInfo {
 
     ResourceUsage queueResourceUsage = q.getQueueResourceUsage();
     populateQueueResourceUsage(queueResourceUsage);
+
+    minEffectiveCapacity = new ResourceInfo(
+        q.getQueueResourceQuotas().getEffectiveMinResource());
+    maxEffectiveCapacity = new ResourceInfo(
+        q.getQueueResourceQuotas().getEffectiveMaxResource());
   }
 
   protected void populateQueueResourceUsage(ResourceUsage queueResourceUsage) {
@@ -200,4 +207,12 @@ public class CapacitySchedulerQueueInfo {
   public ResourcesInfo getResources() {
     return resources;
   }
+
+  public ResourceInfo getMinEffectiveCapacity(){
+    return minEffectiveCapacity;
+  }
+
+  public ResourceInfo getMaxEffectiveCapacity(){
+    return maxEffectiveCapacity;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.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/MockNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
index b5c20f4..909d023 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
@@ -142,6 +143,13 @@ public class MockNM {
     return this.registeringCollectors;
   }
 
+  public void unRegisterNode() throws Exception {
+    UnRegisterNodeManagerRequest request = Records
+        .newRecord(UnRegisterNodeManagerRequest.class);
+    request.setNodeId(nodeId);
+    resourceTracker.unRegisterNodeManager(request);
+  }
+
   public RegisterNodeManagerResponse registerNode() throws Exception {
     return registerNode(null, null);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index 711f008..19ca6d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -823,6 +823,12 @@ public class MockRM extends ResourceManager {
     return rmApp;
   }
 
+  public MockNM unRegisterNode(MockNM nm) throws Exception {
+    nm.unRegisterNode();
+    drainEventsImplicitly();
+    return nm;
+  }
+
   public MockNM registerNode(String nodeIdStr, int memory) throws Exception {
     MockNM nm = new MockNM(nodeIdStr, memory, getResourceTrackerService());
     nm.registerNode();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.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/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
index 0bc5cb5..1dda8a6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.QueueOrderingPolicy;
@@ -644,9 +645,11 @@ public class ProportionalCapacityPreemptionPolicyMockFramework {
 
     QueueCapacities qc = new QueueCapacities(0 == myLevel);
     ResourceUsage ru = new ResourceUsage();
+    QueueResourceQuotas qr  = new QueueResourceQuotas();
 
     when(queue.getQueueCapacities()).thenReturn(qc);
     when(queue.getQueueResourceUsage()).thenReturn(ru);
+    when(queue.getQueueResourceQuotas()).thenReturn(qr);
 
     LOG.debug("Setup queue, name=" + queue.getQueueName() + " path="
         + queue.getQueuePath());
@@ -679,7 +682,17 @@ public class ProportionalCapacityPreemptionPolicyMockFramework {
       qc.setAbsoluteMaximumCapacity(partitionName, absMax);
       qc.setAbsoluteUsedCapacity(partitionName, absUsed);
       qc.setUsedCapacity(partitionName, used);
+      qr.setEffectiveMaxResource(parseResourceFromString(values[1].trim()));
+      qr.setEffectiveMinResource(parseResourceFromString(values[0].trim()));
+      qr.setEffectiveMaxResource(partitionName,
+          parseResourceFromString(values[1].trim()));
+      qr.setEffectiveMinResource(partitionName,
+          parseResourceFromString(values[0].trim()));
       when(queue.getUsedCapacity()).thenReturn(used);
+      when(queue.getEffectiveCapacity(partitionName))
+          .thenReturn(parseResourceFromString(values[0].trim()));
+      when(queue.getEffectiveMaxCapacity(partitionName))
+          .thenReturn(parseResourceFromString(values[1].trim()));
       ru.setPending(partitionName, pending);
       // Setup reserved resource if it contained by input config
       Resource reserved = Resources.none();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.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/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index f0ca466..22a241f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.QueueOrderingPolicy;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
@@ -437,8 +438,8 @@ public class TestProportionalCapacityPreemptionPolicy {
     policy.editSchedule();
     // queueF(appD) wants resources, Verify that resources come from queueE(appC)
     // because it's a sibling and queueB(appA) because queueA is over capacity.
-    verify(mDisp, times(28)).handle(argThat(new IsPreemptionRequestFor(appA)));
-    verify(mDisp, times(22)).handle(argThat(new IsPreemptionRequestFor(appC)));
+    verify(mDisp, times(27)).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(23)).handle(argThat(new IsPreemptionRequestFor(appC)));
 
     // Need to call setup() again to reset mDisp
     setup();
@@ -1204,6 +1205,17 @@ public class TestProportionalCapacityPreemptionPolicy {
     when(root.getQueuePath()).thenReturn(CapacitySchedulerConfiguration.ROOT);
     boolean preemptionDisabled = mockPreemptionStatus("root");
     when(root.getPreemptionDisabled()).thenReturn(preemptionDisabled);
+    QueueResourceQuotas rootQr = new QueueResourceQuotas();
+    rootQr.setEffectiveMaxResource(Resource.newInstance(maxCap[0], 0));
+    rootQr.setEffectiveMinResource(abs[0]);
+    rootQr.setEffectiveMaxResource(RMNodeLabelsManager.NO_LABEL,
+        Resource.newInstance(maxCap[0], 0));
+    rootQr.setEffectiveMinResource(RMNodeLabelsManager.NO_LABEL, abs[0]);
+    when(root.getQueueResourceQuotas()).thenReturn(rootQr);
+    when(root.getEffectiveCapacity(RMNodeLabelsManager.NO_LABEL))
+        .thenReturn(abs[0]);
+    when(root.getEffectiveMaxCapacity(RMNodeLabelsManager.NO_LABEL))
+        .thenReturn(Resource.newInstance(maxCap[0], 0));
 
     for (int i = 1; i < queues.length; ++i) {
       final CSQueue q;
@@ -1234,6 +1246,18 @@ public class TestProportionalCapacityPreemptionPolicy {
       qc.setAbsoluteMaximumCapacity(maxCap[i] / (float) tot.getMemorySize());
       when(q.getQueueCapacities()).thenReturn(qc);
 
+      QueueResourceQuotas qr = new QueueResourceQuotas();
+      qr.setEffectiveMaxResource(Resource.newInstance(maxCap[i], 0));
+      qr.setEffectiveMinResource(abs[i]);
+      qr.setEffectiveMaxResource(RMNodeLabelsManager.NO_LABEL,
+          Resource.newInstance(maxCap[i], 0));
+      qr.setEffectiveMinResource(RMNodeLabelsManager.NO_LABEL, abs[i]);
+      when(q.getQueueResourceQuotas()).thenReturn(qr);
+      when(q.getEffectiveCapacity(RMNodeLabelsManager.NO_LABEL))
+          .thenReturn(abs[i]);
+      when(q.getEffectiveMaxCapacity(RMNodeLabelsManager.NO_LABEL))
+          .thenReturn(Resource.newInstance(maxCap[i], 0));
+
       String parentPathName = p.getQueuePath();
       parentPathName = (parentPathName == null) ? "root" : parentPathName;
       String queuePathName = (parentPathName + "." + queueName).replace("/",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.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/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java
index 7784549..a1d89d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java
@@ -67,9 +67,9 @@ public class TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF
     conf.set(CapacitySchedulerConfiguration.INTRAQUEUE_PREEMPTION_ORDER_POLICY,
         "priority_first");
 
-    String labelsConfig = "=100:200,true;";
+    String labelsConfig = "=100:50,true;";
     String nodesConfig = // n1 has no label
-        "n1= res=100:200";
+        "n1= res=100:50";
     String queuesConfig =
         // guaranteed,max,used,pending,reserved
         "root(=[100:50 100:50 80:40 120:60 0]);" + // root
@@ -105,7 +105,7 @@ public class TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF
     verify(mDisp, times(1)).handle(argThat(
         new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
             getAppAttemptId(4))));
-    verify(mDisp, times(7)).handle(argThat(
+    verify(mDisp, times(3)).handle(argThat(
         new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
             getAppAttemptId(3))));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.java
new file mode 100644
index 0000000..5a66281
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAbsoluteResourceConfiguration.java
@@ -0,0 +1,516 @@
+/**
+* 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.scheduler.capacity;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestAbsoluteResourceConfiguration {
+
+  private static final int GB = 1024;
+
+  private static final String QUEUEA = "queueA";
+  private static final String QUEUEB = "queueB";
+  private static final String QUEUEC = "queueC";
+  private static final String QUEUEA1 = "queueA1";
+  private static final String QUEUEA2 = "queueA2";
+  private static final String QUEUEB1 = "queueB1";
+
+  private static final String QUEUEA_FULL = CapacitySchedulerConfiguration.ROOT
+      + "." + QUEUEA;
+  private static final String QUEUEB_FULL = CapacitySchedulerConfiguration.ROOT
+      + "." + QUEUEB;
+  private static final String QUEUEC_FULL = CapacitySchedulerConfiguration.ROOT
+      + "." + QUEUEC;
+  private static final String QUEUEA1_FULL = QUEUEA_FULL + "." + QUEUEA1;
+  private static final String QUEUEA2_FULL = QUEUEA_FULL + "." + QUEUEA2;
+  private static final String QUEUEB1_FULL = QUEUEB_FULL + "." + QUEUEB1;
+
+  private static final Resource QUEUE_A_MINRES = Resource.newInstance(100 * GB,
+      10);
+  private static final Resource QUEUE_A_MAXRES = Resource.newInstance(200 * GB,
+      30);
+  private static final Resource QUEUE_A1_MINRES = Resource.newInstance(50 * GB,
+      5);
+  private static final Resource QUEUE_A2_MINRES = Resource.newInstance(50 * GB,
+      5);
+  private static final Resource QUEUE_B_MINRES = Resource.newInstance(50 * GB,
+      10);
+  private static final Resource QUEUE_B1_MINRES = Resource.newInstance(40 * GB,
+      10);
+  private static final Resource QUEUE_B_MAXRES = Resource.newInstance(150 * GB,
+      30);
+  private static final Resource QUEUE_C_MINRES = Resource.newInstance(50 * GB,
+      10);
+  private static final Resource QUEUE_C_MAXRES = Resource.newInstance(150 * GB,
+      20);
+  private static final Resource QUEUEA_REDUCED = Resource.newInstance(64000, 6);
+  private static final Resource QUEUEB_REDUCED = Resource.newInstance(32000, 6);
+  private static final Resource QUEUEC_REDUCED = Resource.newInstance(32000, 6);
+  private static final Resource QUEUEMAX_REDUCED = Resource.newInstance(128000,
+      20);
+
+  private static Set<String> resourceTypes = new HashSet<>(
+      Arrays.asList("memory", "vcores"));
+
+  private CapacitySchedulerConfiguration setupSimpleQueueConfiguration(
+      boolean isCapacityNeeded) {
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[]{QUEUEA, QUEUEB, QUEUEC});
+
+    // Set default capacities like normal configuration.
+    if (isCapacityNeeded) {
+      csConf.setCapacity(QUEUEA_FULL, 50f);
+      csConf.setCapacity(QUEUEB_FULL, 25f);
+      csConf.setCapacity(QUEUEC_FULL, 25f);
+    }
+
+    return csConf;
+  }
+
+  private CapacitySchedulerConfiguration setupComplexQueueConfiguration(
+      boolean isCapacityNeeded) {
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[]{QUEUEA, QUEUEB, QUEUEC});
+    csConf.setQueues(QUEUEA_FULL, new String[]{QUEUEA1, QUEUEA2});
+    csConf.setQueues(QUEUEB_FULL, new String[]{QUEUEB1});
+
+    // Set default capacities like normal configuration.
+    if (isCapacityNeeded) {
+      csConf.setCapacity(QUEUEA_FULL, 50f);
+      csConf.setCapacity(QUEUEB_FULL, 25f);
+      csConf.setCapacity(QUEUEC_FULL, 25f);
+      csConf.setCapacity(QUEUEA1_FULL, 50f);
+      csConf.setCapacity(QUEUEA2_FULL, 50f);
+      csConf.setCapacity(QUEUEB1_FULL, 100f);
+    }
+
+    return csConf;
+  }
+
+  private CapacitySchedulerConfiguration setupMinMaxResourceConfiguration(
+      CapacitySchedulerConfiguration csConf) {
+    // Update min/max resource to queueA/B/C
+    csConf.setMinimumResourceRequirement("", QUEUEA_FULL, QUEUE_A_MINRES);
+    csConf.setMinimumResourceRequirement("", QUEUEB_FULL, QUEUE_B_MINRES);
+    csConf.setMinimumResourceRequirement("", QUEUEC_FULL, QUEUE_C_MINRES);
+
+    csConf.setMaximumResourceRequirement("", QUEUEA_FULL, QUEUE_A_MAXRES);
+    csConf.setMaximumResourceRequirement("", QUEUEB_FULL, QUEUE_B_MAXRES);
+    csConf.setMaximumResourceRequirement("", QUEUEC_FULL, QUEUE_C_MAXRES);
+
+    return csConf;
+  }
+
+  private CapacitySchedulerConfiguration setupComplexMinMaxResourceConfig(
+      CapacitySchedulerConfiguration csConf) {
+    // Update min/max resource to queueA/B/C
+    csConf.setMinimumResourceRequirement("", QUEUEA_FULL, QUEUE_A_MINRES);
+    csConf.setMinimumResourceRequirement("", QUEUEB_FULL, QUEUE_B_MINRES);
+    csConf.setMinimumResourceRequirement("", QUEUEC_FULL, QUEUE_C_MINRES);
+    csConf.setMinimumResourceRequirement("", QUEUEA1_FULL, QUEUE_A1_MINRES);
+    csConf.setMinimumResourceRequirement("", QUEUEA2_FULL, QUEUE_A2_MINRES);
+    csConf.setMinimumResourceRequirement("", QUEUEB1_FULL, QUEUE_B1_MINRES);
+
+    csConf.setMaximumResourceRequirement("", QUEUEA_FULL, QUEUE_A_MAXRES);
+    csConf.setMaximumResourceRequirement("", QUEUEB_FULL, QUEUE_B_MAXRES);
+    csConf.setMaximumResourceRequirement("", QUEUEC_FULL, QUEUE_C_MAXRES);
+
+    return csConf;
+  }
+
+  @Test
+  public void testSimpleMinMaxResourceConfigurartionPerQueue() {
+
+    CapacitySchedulerConfiguration csConf = setupSimpleQueueConfiguration(true);
+    setupMinMaxResourceConfiguration(csConf);
+
+    Assert.assertEquals("Min resource configured for QUEUEA is not correct",
+        QUEUE_A_MINRES,
+        csConf.getMinimumResourceRequirement("", QUEUEA_FULL, resourceTypes));
+    Assert.assertEquals("Max resource configured for QUEUEA is not correct",
+        QUEUE_A_MAXRES,
+        csConf.getMaximumResourceRequirement("", QUEUEA_FULL, resourceTypes));
+    Assert.assertEquals("Min resource configured for QUEUEB is not correct",
+        QUEUE_B_MINRES,
+        csConf.getMinimumResourceRequirement("", QUEUEB_FULL, resourceTypes));
+    Assert.assertEquals("Max resource configured for QUEUEB is not correct",
+        QUEUE_B_MAXRES,
+        csConf.getMaximumResourceRequirement("", QUEUEB_FULL, resourceTypes));
+    Assert.assertEquals("Min resource configured for QUEUEC is not correct",
+        QUEUE_C_MINRES,
+        csConf.getMinimumResourceRequirement("", QUEUEC_FULL, resourceTypes));
+    Assert.assertEquals("Max resource configured for QUEUEC is not correct",
+        QUEUE_C_MAXRES,
+        csConf.getMaximumResourceRequirement("", QUEUEC_FULL, resourceTypes));
+  }
+
+  @Test
+  public void testEffectiveMinMaxResourceConfigurartionPerQueue()
+      throws Exception {
+    // create conf with basic queue configuration.
+    CapacitySchedulerConfiguration csConf = setupSimpleQueueConfiguration(
+        false);
+    setupMinMaxResourceConfiguration(csConf);
+
+    csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    @SuppressWarnings("resource")
+    MockRM rm = new MockRM(csConf);
+    rm.start();
+
+    // Add few nodes
+    rm.registerNode("127.0.0.1:1234", 250 * GB, 40);
+
+    // Get queue object to verify min/max resource configuration.
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    LeafQueue qA = (LeafQueue) cs.getQueue(QUEUEA);
+    Assert.assertNotNull(qA);
+    Assert.assertEquals("Min resource configured for QUEUEA is not correct",
+        QUEUE_A_MINRES, qA.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEA is not correct",
+        QUEUE_A_MAXRES, qA.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEA is not correct",
+        QUEUE_A_MINRES, qA.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA is not correct",
+        QUEUE_A_MAXRES, qA.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qB = (LeafQueue) cs.getQueue(QUEUEB);
+    Assert.assertNotNull(qB);
+    Assert.assertEquals("Min resource configured for QUEUEB is not correct",
+        QUEUE_B_MINRES, qB.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEB is not correct",
+        QUEUE_B_MAXRES, qB.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEB is not correct",
+        QUEUE_B_MINRES, qB.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEB is not correct",
+        QUEUE_B_MAXRES, qB.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qC = (LeafQueue) cs.getQueue(QUEUEC);
+    Assert.assertNotNull(qC);
+    Assert.assertEquals("Min resource configured for QUEUEC is not correct",
+        QUEUE_C_MINRES, qC.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEC is not correct",
+        QUEUE_C_MAXRES, qC.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEC is not correct",
+        QUEUE_C_MINRES, qC.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEC is not correct",
+        QUEUE_C_MAXRES, qC.queueResourceQuotas.getEffectiveMaxResource());
+
+    rm.stop();
+  }
+
+  @Test
+  public void testSimpleValidateAbsoluteResourceConfig() throws Exception {
+    /**
+     * Queue structure is as follows. root / | \ a b c / \ | a1 a2 b1
+     *
+     * Test below cases 1) Configure percentage based capacity and absolute
+     * resource together. 2) As per above tree structure, ensure all values
+     * could be retrieved. 3) Validate whether min resource cannot be more than
+     * max resources. 4) Validate whether max resource of queue cannot be more
+     * than its parent max resource.
+     */
+    // create conf with basic queue configuration.
+    CapacitySchedulerConfiguration csConf = setupSimpleQueueConfiguration(
+        false);
+    setupMinMaxResourceConfiguration(csConf);
+    csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    @SuppressWarnings("resource")
+    MockRM rm = new MockRM(csConf);
+    rm.start();
+
+    // Add few nodes
+    rm.registerNode("127.0.0.1:1234", 250 * GB, 40);
+
+    // Get queue object to verify min/max resource configuration.
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    // 1. Create a new config with capcity and min/max together. Ensure an
+    // exception is thrown.
+    CapacitySchedulerConfiguration csConf1 = setupSimpleQueueConfiguration(
+        true);
+    setupMinMaxResourceConfiguration(csConf1);
+
+    try {
+      cs.reinitialize(csConf1, rm.getRMContext());
+      Assert.fail();
+    } catch (IOException e) {
+      Assert.assertTrue(e instanceof IOException);
+      Assert.assertEquals(
+          "Failed to re-init queues : Queue 'queueA' should use either"
+              + " percentage based capacity configuration or absolute resource.",
+          e.getMessage());
+    }
+    rm.stop();
+
+    // 2. Create a new config with min/max alone with a complex queue config.
+    // Check all values could be fetched correctly.
+    CapacitySchedulerConfiguration csConf2 = setupComplexQueueConfiguration(
+        false);
+    setupComplexMinMaxResourceConfig(csConf2);
+
+    rm = new MockRM(csConf2);
+    rm.start();
+    rm.registerNode("127.0.0.1:1234", 250 * GB, 40);
+    cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    LeafQueue qA1 = (LeafQueue) cs.getQueue(QUEUEA1);
+    Assert.assertEquals("Effective Min resource for QUEUEA1 is not correct",
+        QUEUE_A1_MINRES, qA1.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA1 is not correct",
+        QUEUE_A_MAXRES, qA1.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qA2 = (LeafQueue) cs.getQueue(QUEUEA2);
+    Assert.assertEquals("Effective Min resource for QUEUEA2 is not correct",
+        QUEUE_A2_MINRES, qA2.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA2 is not correct",
+        QUEUE_A_MAXRES, qA2.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qB1 = (LeafQueue) cs.getQueue(QUEUEB1);
+    Assert.assertNotNull(qB1);
+    Assert.assertEquals("Min resource configured for QUEUEB1 is not correct",
+        QUEUE_B1_MINRES, qB1.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEB1 is not correct",
+        QUEUE_B_MAXRES, qB1.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEB1 is not correct",
+        QUEUE_B1_MINRES, qB1.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEB1 is not correct",
+        QUEUE_B_MAXRES, qB1.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qC = (LeafQueue) cs.getQueue(QUEUEC);
+    Assert.assertNotNull(qC);
+    Assert.assertEquals("Min resource configured for QUEUEC is not correct",
+        QUEUE_C_MINRES, qC.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEC is not correct",
+        QUEUE_C_MAXRES, qC.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEC is not correct",
+        QUEUE_C_MINRES, qC.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEC is not correct",
+        QUEUE_C_MAXRES, qC.queueResourceQuotas.getEffectiveMaxResource());
+
+    // 3. Create a new config and make sure one queue's min resource is more
+    // than its max resource configured.
+    CapacitySchedulerConfiguration csConf3 = setupComplexQueueConfiguration(
+        false);
+    setupComplexMinMaxResourceConfig(csConf3);
+
+    csConf3.setMinimumResourceRequirement("", QUEUEB1_FULL, QUEUE_B_MAXRES);
+    csConf3.setMaximumResourceRequirement("", QUEUEB1_FULL, QUEUE_B1_MINRES);
+
+    try {
+      cs.reinitialize(csConf3, rm.getRMContext());
+      Assert.fail();
+    } catch (IOException e) {
+      Assert.assertTrue(e instanceof IOException);
+      Assert.assertEquals(
+          "Failed to re-init queues : Min resource configuration "
+              + "<memory:153600, vCores:30> is greater than its "
+              + "max value:<memory:40960, vCores:10> in queue:queueB1",
+          e.getMessage());
+    }
+
+    // 4. Create a new config and make sure one queue's max resource is more
+    // than its preant's max resource configured.
+    CapacitySchedulerConfiguration csConf4 = setupComplexQueueConfiguration(
+        false);
+    setupComplexMinMaxResourceConfig(csConf4);
+
+    csConf4.setMaximumResourceRequirement("", QUEUEB1_FULL, QUEUE_A_MAXRES);
+
+    try {
+      cs.reinitialize(csConf4, rm.getRMContext());
+      Assert.fail();
+    } catch (IOException e) {
+      Assert.assertTrue(e instanceof IOException);
+      Assert
+          .assertEquals(
+              "Failed to re-init queues : Max resource configuration "
+                  + "<memory:204800, vCores:30> is greater than parents max value:"
+                  + "<memory:153600, vCores:30> in queue:queueB1",
+              e.getMessage());
+    }
+    rm.stop();
+  }
+
+  @Test
+  public void testComplexValidateAbsoluteResourceConfig() throws Exception {
+    /**
+     * Queue structure is as follows. root / | \ a b c / \ | a1 a2 b1
+     *
+     * Test below cases: 1) Parent and its child queues must use either
+     * percentage based or absolute resource configuration. 2) Parent's min
+     * resource must be more than sum of child's min resource.
+     */
+
+    // create conf with basic queue configuration.
+    CapacitySchedulerConfiguration csConf = setupComplexQueueConfiguration(
+        false);
+    setupComplexMinMaxResourceConfig(csConf);
+    csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    @SuppressWarnings("resource")
+    MockRM rm = new MockRM(csConf);
+    rm.start();
+
+    // Add few nodes
+    rm.registerNode("127.0.0.1:1234", 250 * GB, 40);
+
+    // 1. Explicitly set percentage based config for parent queues. This will
+    // make Queue A,B and C with percentage based and A1,A2 or B1 with absolute
+    // resource.
+    csConf.setCapacity(QUEUEA_FULL, 50f);
+    csConf.setCapacity(QUEUEB_FULL, 25f);
+    csConf.setCapacity(QUEUEC_FULL, 25f);
+
+    // Also unset resource based config.
+    csConf.setMinimumResourceRequirement("", QUEUEA_FULL, Resources.none());
+    csConf.setMinimumResourceRequirement("", QUEUEB_FULL, Resources.none());
+    csConf.setMinimumResourceRequirement("", QUEUEC_FULL, Resources.none());
+
+    // Get queue object to verify min/max resource configuration.
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    try {
+      cs.reinitialize(csConf, rm.getRMContext());
+      Assert.fail();
+    } catch (IOException e) {
+      Assert.assertTrue(e instanceof IOException);
+      Assert.assertEquals(
+          "Failed to re-init queues : Parent queue 'queueA' "
+              + "and child queue 'queueA1' should use either percentage based"
+              + " capacity configuration or absolute resource together.",
+          e.getMessage());
+    }
+
+    // 2. Create a new config and make sure one queue's min resource is more
+    // than its max resource configured.
+    CapacitySchedulerConfiguration csConf1 = setupComplexQueueConfiguration(
+        false);
+    setupComplexMinMaxResourceConfig(csConf1);
+
+    // Configure QueueA with lesser resource than its children.
+    csConf1.setMinimumResourceRequirement("", QUEUEA_FULL, QUEUE_A1_MINRES);
+
+    try {
+      cs.reinitialize(csConf1, rm.getRMContext());
+      Assert.fail();
+    } catch (IOException e) {
+      Assert.assertTrue(e instanceof IOException);
+      Assert.assertEquals("Failed to re-init queues : Parent Queues capacity: "
+          + "<memory:51200, vCores:5> is less than to its children:"
+          + "<memory:102400, vCores:10> for queue:queueA", e.getMessage());
+    }
+  }
+
+  @Test
+  public void testEffectiveResourceAfterReducingClusterResource()
+      throws Exception {
+    // create conf with basic queue configuration.
+    CapacitySchedulerConfiguration csConf = setupSimpleQueueConfiguration(
+        false);
+    setupMinMaxResourceConfiguration(csConf);
+
+    csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    @SuppressWarnings("resource")
+    MockRM rm = new MockRM(csConf);
+    rm.start();
+
+    // Add few nodes
+    MockNM nm1 = rm.registerNode("127.0.0.1:1234", 125 * GB, 20);
+    rm.registerNode("127.0.0.2:1234", 125 * GB, 20);
+
+    // Get queue object to verify min/max resource configuration.
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    LeafQueue qA = (LeafQueue) cs.getQueue(QUEUEA);
+    Assert.assertNotNull(qA);
+    Assert.assertEquals("Min resource configured for QUEUEA is not correct",
+        QUEUE_A_MINRES, qA.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEA is not correct",
+        QUEUE_A_MAXRES, qA.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEA is not correct",
+        QUEUE_A_MINRES, qA.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA is not correct",
+        QUEUE_A_MAXRES, qA.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qB = (LeafQueue) cs.getQueue(QUEUEB);
+    Assert.assertNotNull(qB);
+    Assert.assertEquals("Min resource configured for QUEUEB is not correct",
+        QUEUE_B_MINRES, qB.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEB is not correct",
+        QUEUE_B_MAXRES, qB.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEB is not correct",
+        QUEUE_B_MINRES, qB.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEB is not correct",
+        QUEUE_B_MAXRES, qB.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qC = (LeafQueue) cs.getQueue(QUEUEC);
+    Assert.assertNotNull(qC);
+    Assert.assertEquals("Min resource configured for QUEUEC is not correct",
+        QUEUE_C_MINRES, qC.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEC is not correct",
+        QUEUE_C_MAXRES, qC.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEC is not correct",
+        QUEUE_C_MINRES, qC.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEC is not correct",
+        QUEUE_C_MAXRES, qC.queueResourceQuotas.getEffectiveMaxResource());
+
+    // unregister one NM.
+    rm.unRegisterNode(nm1);
+
+    // After loosing one NM, effective min res of queueA will become just
+    // above half. Hence A's min will be 60Gi and 6 cores and max will be
+    // 128GB and 20 cores.
+    Assert.assertEquals("Effective Min resource for QUEUEA is not correct",
+        QUEUEA_REDUCED, qA.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA is not correct",
+        QUEUEMAX_REDUCED, qA.queueResourceQuotas.getEffectiveMaxResource());
+
+    Assert.assertEquals("Effective Min resource for QUEUEB is not correct",
+        QUEUEB_REDUCED, qB.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEB is not correct",
+        QUEUEMAX_REDUCED, qB.queueResourceQuotas.getEffectiveMaxResource());
+
+    Assert.assertEquals("Effective Min resource for QUEUEC is not correct",
+        QUEUEC_REDUCED, qC.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEC is not correct",
+        QUEUEMAX_REDUCED, qC.queueResourceQuotas.getEffectiveMaxResource());
+
+    rm.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 8aca235..24ae244 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
@@ -60,6 +60,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
@@ -86,6 +87,7 @@ public class TestApplicationLimits {
   final static int GB = 1024;
 
   LeafQueue queue;
+  CSQueue root;
   
   private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
 
@@ -100,7 +102,7 @@ public class TestApplicationLimits {
     setupQueueConfiguration(csConf);
     
     rmContext = TestUtils.getMockRMContext();
-
+    Resource clusterResource = Resources.createResource(10 * 16 * GB, 10 * 32);
 
     CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);
     when(csContext.getConfiguration()).thenReturn(csConf);
@@ -110,10 +112,11 @@ public class TestApplicationLimits {
     when(csContext.getMaximumResourceCapability()).
         thenReturn(Resources.createResource(16*GB, 32));
     when(csContext.getClusterResource()).
-        thenReturn(Resources.createResource(10 * 16 * GB, 10 * 32));
+        thenReturn(clusterResource);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     
     RMContainerTokenSecretManager containerTokenSecretManager =
         new RMContainerTokenSecretManager(conf);
@@ -122,13 +125,17 @@ public class TestApplicationLimits {
         containerTokenSecretManager);
 
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
-    CSQueue root = CapacitySchedulerQueueManager
+    root = CapacitySchedulerQueueManager
         .parseQueue(csContext, csConf, null, "root",
             queues, queues,
             TestUtils.spyHook);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
-    
     queue = spy(new LeafQueue(csContext, A, root, null));
+    QueueResourceQuotas queueResourceQuotas = ((LeafQueue) queues.get(A))
+        .getQueueResourceQuotas();
+    doReturn(queueResourceQuotas).when(queue).getQueueResourceQuotas();
 
     // Stub out ACL checks
     doReturn(true).
@@ -189,6 +196,8 @@ public class TestApplicationLimits {
     // when there is only 1 user, and drops to 2G (the userlimit) when there
     // is a second user
     Resource clusterResource = Resource.newInstance(80 * GB, 40);
+    root.updateClusterResource(clusterResource, new ResourceLimits(
+        clusterResource));
     queue.updateClusterResource(clusterResource, new ResourceLimits(
         clusterResource));
     
@@ -287,6 +296,8 @@ public class TestApplicationLimits {
     CSQueue root = 
         CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
             "root", queues, queues, TestUtils.spyHook);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     LeafQueue queue = (LeafQueue)queues.get(A);
     
@@ -357,6 +368,8 @@ public class TestApplicationLimits {
         csContext, csConf, null, "root",
         queues, queues, TestUtils.spyHook);
     clusterResource = Resources.createResource(100 * 16 * GB);
+    root.updateClusterResource(clusterResource, new ResourceLimits(
+        clusterResource));
 
     queue = (LeafQueue)queues.get(A);
 
@@ -378,6 +391,8 @@ public class TestApplicationLimits {
     root = CapacitySchedulerQueueManager.parseQueue(
         csContext, csConf, null, "root",
         queues, queues, TestUtils.spyHook);
+    root.updateClusterResource(clusterResource, new ResourceLimits(
+        clusterResource));
 
     queue = (LeafQueue)queues.get(A);
     assertEquals(9999, (int)csConf.getMaximumApplicationsPerQueue(queue.getQueuePath()));
@@ -393,7 +408,7 @@ public class TestApplicationLimits {
     final String user_0 = "user_0";
     final String user_1 = "user_1";
     final String user_2 = "user_2";
-    
+
     assertEquals(Resource.newInstance(16 * GB, 1),
         queue.calculateAndGetAMResourceLimit());
     assertEquals(Resource.newInstance(8 * GB, 1),
@@ -578,6 +593,7 @@ public class TestApplicationLimits {
         thenReturn(Resources.createResource(16*GB));
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     when(csContext.getRMContext()).thenReturn(rmContext);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
     
     // Say cluster has 100 nodes of 16G each
     Resource clusterResource = Resources.createResource(100 * 16 * GB);
@@ -586,6 +602,8 @@ public class TestApplicationLimits {
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(csContext,
         csConf, null, "root", queues, queues, TestUtils.spyHook);
+    rootQueue.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     ResourceUsage queueCapacities = rootQueue.getQueueResourceUsage();
     when(csContext.getClusterResourceUsage())
@@ -693,6 +711,8 @@ public class TestApplicationLimits {
 
     // Now reduce cluster size and check for the smaller headroom
     clusterResource = Resources.createResource(90*16*GB);
+    rootQueue.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Any change is cluster resource needs to enforce user-limit recomputation.
     // In existing code, LeafQueue#updateClusterResource handled this. However

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimitsByPartition.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/TestApplicationLimitsByPartition.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/TestApplicationLimitsByPartition.java
index 0aac2ef..d73f1c8 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/TestApplicationLimitsByPartition.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/TestApplicationLimitsByPartition.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@@ -600,6 +601,7 @@ public class TestApplicationLimitsByPartition {
     RMContext spyRMContext = spy(rmContext);
     when(spyRMContext.getNodeLabelManager()).thenReturn(mgr);
     when(csContext.getRMContext()).thenReturn(spyRMContext);
+    when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager());
 
     mgr.activateNode(NodeId.newInstance("h0", 0),
         Resource.newInstance(160 * GB, 16)); // default Label
@@ -615,6 +617,8 @@ public class TestApplicationLimitsByPartition {
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(csContext,
         csConf, null, "root", queues, queues, TestUtils.spyHook);
+    rootQueue.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     ResourceUsage queueResUsage = rootQueue.getQueueResourceUsage();
     when(csContext.getClusterResourceUsage())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 e91f734..23a371a 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
@@ -4307,7 +4307,7 @@ public class TestCapacityScheduler {
           null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
     }
-    assertEquals("P2 Used Resource should be 8 GB", 8 * GB,
+    assertEquals("P2 Used Resource should be 7 GB", 7 * GB,
         cs.getQueue("p2").getUsedResources().getMemorySize());
 
     //Free a container from X1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 0fcc86d..fcfa0dd 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
@@ -242,6 +242,8 @@ public class TestChildQueueOrder {
       Resources.createResource(numNodes * (memoryPerNode*GB), 
           numNodes * coresPerNode);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Start testing
     CSQueue a = queues.get(A);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 a32352b..96f8970 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
@@ -191,6 +191,8 @@ public class TestLeafQueue {
             CapacitySchedulerConfiguration.ROOT, 
             queues, queues, 
             TestUtils.spyHook);
+    root.updateClusterResource(Resources.createResource(100 * 16 * GB, 100 * 32),
+        new ResourceLimits(Resources.createResource(100 * 16 * GB, 100 * 32)));
 
     ResourceUsage queueResUsage = root.getQueueResourceUsage();
     when(csContext.getClusterResourceUsage())
@@ -307,13 +309,11 @@ public class TestLeafQueue {
     // Verify the value for getAMResourceLimit for queues with < .1 maxcap
     Resource clusterResource = Resource.newInstance(50 * GB, 50);
 
-    a.updateClusterResource(clusterResource,
+    root.updateClusterResource(clusterResource,
         new ResourceLimits(clusterResource));
     assertEquals(Resource.newInstance(1 * GB, 1),
         a.calculateAndGetAMResourceLimit());
 
-    b.updateClusterResource(clusterResource,
-        new ResourceLimits(clusterResource));
     assertEquals(Resource.newInstance(5 * GB, 1),
         b.calculateAndGetAMResourceLimit());
   }
@@ -358,6 +358,8 @@ public class TestLeafQueue {
     Resource clusterResource = 
         Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     Priority priority = TestUtils.createMockPriority(1);
@@ -556,6 +558,8 @@ public class TestLeafQueue {
     Resource clusterResource = 
         Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     Priority priority = TestUtils.createMockPriority(1);
@@ -630,6 +634,8 @@ public class TestLeafQueue {
     // Test max-capacity
     // Now - no more allocs since we are at max-cap
     a.setMaxCapacity(0.5f);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     applyCSAssignment(clusterResource,
         a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource),
@@ -699,6 +705,8 @@ public class TestLeafQueue {
     Resource clusterResource =
         Resources.createResource(numNodes * (80 * GB), numNodes * 100);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Set user-limit. Need a small queue within a large cluster.
     b.setUserLimit(50);
@@ -779,6 +787,8 @@ public class TestLeafQueue {
         Resources.createResource(numNodes * (8 * GB), numNodes * 100);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getClusterResource()).thenReturn(clusterResource);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests so that one application is memory dominant
     // and other application is vcores dominant
@@ -891,6 +901,8 @@ public class TestLeafQueue {
     Resource clusterResource = 
         Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
  
     // Setup resource-requests
     Priority priority = TestUtils.createMockPriority(1);
@@ -915,6 +927,8 @@ public class TestLeafQueue {
     // Set user-limit
     a.setUserLimit(50);
     a.setUserLimitFactor(2);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     
     // There're two active users
     assertEquals(2, a.getAbstractUsersManager().getNumActiveUsers());
@@ -940,7 +954,7 @@ public class TestLeafQueue {
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
 
     // Allocate one container to app_0, before allocating this container,
-    // user-limit = ceil((4 + 1) / 2) = 3G. app_0's used resource (3G) <=
+    // user-limit = floor((5 + 1) / 2) = 3G. app_0's used resource (3G) <=
     // user-limit.
     applyCSAssignment(clusterResource,
         a.assignContainers(clusterResource, node_1,
@@ -1068,15 +1082,9 @@ public class TestLeafQueue {
         a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource),
         SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps);
-    assertEquals(9*GB, a.getUsedResources().getMemorySize());
-    assertEquals(8*GB, app_0.getCurrentConsumption().getMemorySize());
-    assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
-
-    assertEquals(4*GB,
-        app_0.getTotalPendingRequestsPerPartition().get("").getMemorySize());
-
-    assertEquals(1*GB,
-        app_1.getTotalPendingRequestsPerPartition().get("").getMemorySize());
+    assertEquals(12*GB, a.getUsedResources().getMemorySize());
+    assertEquals(12*GB, app_0.getCurrentConsumption().getMemorySize());
+    assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
   }
 
   @SuppressWarnings({ "unchecked", "rawtypes" })
@@ -1100,6 +1108,8 @@ public class TestLeafQueue {
     final int numNodes = 2;
     Resource clusterResource = Resources.createResource(numNodes * (8*GB), 1);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     CapacitySchedulerQueueManager mockCapacitySchedulerQueueManager
         = mock(CapacitySchedulerQueueManager.class);
@@ -1122,6 +1132,8 @@ public class TestLeafQueue {
     qb.setUserLimit(100);
     qb.setUserLimitFactor(1);
 
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     final ApplicationAttemptId appAttemptId_0 =
               TestUtils.getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 =
@@ -1256,106 +1268,6 @@ public class TestLeafQueue {
   }
 
   @Test
-  public void testUserHeadroomMultiApp() throws Exception {
-    // Mock the queue
-    LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
-    //unset maxCapacity
-    a.setMaxCapacity(1.0f);
-
-    // Users
-    final String user_0 = "user_0";
-    final String user_1 = "user_1";
-
-    // Submit applications
-    final ApplicationAttemptId appAttemptId_0 =
-        TestUtils.getMockApplicationAttemptId(0, 0);
-    FiCaSchedulerApp app_0 =
-        new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-            a.getAbstractUsersManager(), spyRMContext);
-    a.submitApplicationAttempt(app_0, user_0);
-
-    final ApplicationAttemptId appAttemptId_1 =
-        TestUtils.getMockApplicationAttemptId(1, 0);
-    FiCaSchedulerApp app_1 =
-        new FiCaSchedulerApp(appAttemptId_1, user_0, a,
-            a.getAbstractUsersManager(), spyRMContext);
-    a.submitApplicationAttempt(app_1, user_0);  // same user
-
-    final ApplicationAttemptId appAttemptId_2 =
-        TestUtils.getMockApplicationAttemptId(2, 0);
-    FiCaSchedulerApp app_2 =
-        new FiCaSchedulerApp(appAttemptId_2, user_1, a,
-            a.getAbstractUsersManager(), spyRMContext);
-    a.submitApplicationAttempt(app_2, user_1);
-
-    // Setup some nodes
-    String host_0 = "127.0.0.1";
-    FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 
-      0, 16*GB);
-    String host_1 = "127.0.0.2";
-    FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 
-      0, 16*GB);
-
-    Map<ApplicationAttemptId, FiCaSchedulerApp> apps = ImmutableMap.of(
-        app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(),
-        app_1, app_2.getApplicationAttemptId(), app_2);
-    Map<NodeId, FiCaSchedulerNode> nodes = ImmutableMap.of(node_0.getNodeID(),
-        node_0, node_1.getNodeID(), node_1);
-
-    final int numNodes = 2;
-    Resource clusterResource = Resources.createResource(numNodes * (16*GB), 1);
-    when(csContext.getNumClusterNodes()).thenReturn(numNodes);
-
-    Priority priority = TestUtils.createMockPriority(1);
-
-    app_0.updateResourceRequests(Collections.singletonList(
-            TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, true,
-                priority, recordFactory)));
-
-    applyCSAssignment(clusterResource,
-        a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource),
-        SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps);
-    assertEquals(1*GB, a.getUsedResources().getMemorySize());
-    assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
-    assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
-    //Now, headroom is the same for all apps for a given user + queue combo
-    //and a change to any app's headroom is reflected for all the user's apps
-    //once those apps are active/have themselves calculated headroom for 
-    //allocation at least one time
-    assertEquals(2*GB, app_0.getHeadroom().getMemorySize());
-    assertEquals(0*GB, app_1.getHeadroom().getMemorySize());//not yet active
-    assertEquals(0*GB, app_2.getHeadroom().getMemorySize());//not yet active
-
-    app_1.updateResourceRequests(Collections.singletonList(
-        TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 2, true,
-            priority, recordFactory)));
-
-    applyCSAssignment(clusterResource,
-        a.assignContainers(clusterResource, node_0,
-        new ResourceLimits(clusterResource),
-        SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps);
-    assertEquals(2*GB, a.getUsedResources().getMemorySize());
-    assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
-    assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
-    assertEquals(1*GB, app_0.getHeadroom().getMemorySize());
-    assertEquals(1*GB, app_1.getHeadroom().getMemorySize());//now active
-    assertEquals(0*GB, app_2.getHeadroom().getMemorySize());//not yet active
-
-    //Complete container and verify that headroom is updated, for both apps 
-    //for the user
-    RMContainer rmContainer = app_0.getLiveContainers().iterator().next();
-    a.completedContainer(clusterResource, app_0, node_0, rmContainer,
-    ContainerStatus.newInstance(rmContainer.getContainerId(),
-	ContainerState.COMPLETE, "",
-	ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
-    RMContainerEventType.KILL, null, true);
-
-    assertEquals(2*GB, app_0.getHeadroom().getMemorySize());
-    assertEquals(2*GB, app_1.getHeadroom().getMemorySize());
-  }
-
-  @Test
   public void testHeadroomWithMaxCap() throws Exception {
     // Mock the queue
     LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
@@ -1403,7 +1315,12 @@ public class TestLeafQueue {
     final int numNodes = 2;
     Resource clusterResource = Resources.createResource(numNodes * (8*GB), 1);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
- 
+
+    ParentQueue root = (ParentQueue) queues
+        .get(CapacitySchedulerConfiguration.ROOT);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
+
     // Setup resource-requests
     Priority priority = TestUtils.createMockPriority(1);
     app_0.updateResourceRequests(Collections.singletonList(
@@ -1454,6 +1371,8 @@ public class TestLeafQueue {
     
     // Submit requests for app_1 and set max-cap
     a.setMaxCapacity(.1f);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     app_2.updateResourceRequests(Collections.singletonList(
         TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, true,
             priority, recordFactory)));
@@ -1542,6 +1461,8 @@ public class TestLeafQueue {
         Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getClusterResource()).thenReturn(clusterResource);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     Priority priority = TestUtils.createMockPriority(1);
@@ -1624,6 +1545,8 @@ public class TestLeafQueue {
     // Test max-capacity
     // Now - no more allocs since we are at max-cap
     a.setMaxCapacity(0.5f);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     applyCSAssignment(clusterResource,
         a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource),
@@ -1638,6 +1561,8 @@ public class TestLeafQueue {
     // Now, allocations should goto app_3 since it's under user-limit 
     a.setMaxCapacity(1.0f);
     a.setUserLimitFactor(1);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     applyCSAssignment(clusterResource,
         a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource),
@@ -1743,6 +1668,8 @@ public class TestLeafQueue {
     Resource clusterResource = 
         Resources.createResource(numNodes * (4*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     
     // Setup resource-requests
     Priority priority = TestUtils.createMockPriority(1);
@@ -1880,6 +1807,8 @@ public class TestLeafQueue {
     final int numNodes = 3;
     Resource clusterResource = 
         Resources.createResource(numNodes * (4*GB), numNodes * 16);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getMaximumResourceCapability()).thenReturn(
         Resources.createResource(4*GB, 16));
@@ -2051,6 +1980,8 @@ public class TestLeafQueue {
     Resource clusterResource = 
         Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     
     // Setup resource-requests and submit
     Priority priority = TestUtils.createMockPriority(1);
@@ -2237,11 +2168,10 @@ public class TestLeafQueue {
     CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext,
         csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues,
         TestUtils.spyHook);
-    queues = newQueues;
     root.reinitialize(newRoot, cs.getClusterResource());
 
     // Manipulate queue 'b'
-    LeafQueue a = stubLeafQueue((LeafQueue) queues.get(B));
+    LeafQueue a = stubLeafQueue((LeafQueue) newQueues.get(B));
 
     // Check locality parameters.
     assertEquals(2, a.getNodeLocalityDelay());
@@ -2277,6 +2207,8 @@ public class TestLeafQueue {
     Resource clusterResource =
         Resources.createResource(numNodes * (8 * GB), numNodes * 16);
     when(spyRMContext.getScheduler().getNumClusterNodes()).thenReturn(numNodes);
+    newRoot.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests and submit
     Priority priority = TestUtils.createMockPriority(1);
@@ -2412,6 +2344,8 @@ public class TestLeafQueue {
     Resource clusterResource = 
         Resources.createResource(numNodes * (8*GB), 1);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     
     // Setup resource-requests and submit
     List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
@@ -2545,6 +2479,8 @@ public class TestLeafQueue {
     Resource clusterResource = Resources.createResource(
         numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests and submit
     Priority priority = TestUtils.createMockPriority(1);
@@ -2660,17 +2596,14 @@ public class TestLeafQueue {
     assertEquals(2, e.getNumActiveApplications());
     assertEquals(1, e.getNumPendingApplications());
 
-    csConf.setDouble(CapacitySchedulerConfiguration
-        .MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT,
-        CapacitySchedulerConfiguration
-        .DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT * 2);
+    csConf.setDouble(
+        CapacitySchedulerConfiguration.MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT,
+        CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT
+            * 2);
     Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
-    CSQueue newRoot =
-        CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
-            CapacitySchedulerConfiguration.ROOT,
-            newQueues, queues,
-            TestUtils.spyHook);
-    queues = newQueues;
+    CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext,
+        csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues,
+        TestUtils.spyHook);
     root.reinitialize(newRoot, csContext.getClusterResource());
 
     // after reinitialization
@@ -2697,7 +2630,6 @@ public class TestLeafQueue {
             CapacitySchedulerConfiguration.ROOT,
             newQueues, queues,
             TestUtils.spyHook);
-    queues = newQueues;
     root.reinitialize(newRoot, cs.getClusterResource());
 
     // after reinitialization
@@ -2745,7 +2677,7 @@ public class TestLeafQueue {
     assertEquals(1, e.getNumPendingApplications());
 
     Resource clusterResource = Resources.createResource(200 * 16 * GB, 100 * 32); 
-    e.updateClusterResource(clusterResource,
+    root.updateClusterResource(clusterResource,
         new ResourceLimits(clusterResource));
 
     // after updating cluster resource
@@ -2837,6 +2769,9 @@ public class TestLeafQueue {
         numNodes * (8*GB), numNodes * 1);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
+
     // Setup resource-requests
     // resourceName: <priority, memory, #containers, relaxLocality>
     // host_0_0: < 1, 1GB, 1, true >
@@ -3036,36 +2971,44 @@ public class TestLeafQueue {
   @Test
   public void testMaxAMResourcePerQueuePercentAfterQueueRefresh()
       throws Exception {
+    Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
-    Resource clusterResource = Resources
-        .createResource(100 * 16 * GB, 100 * 32);
+    final String newRootName = "root" + System.currentTimeMillis();
+    setupQueueConfiguration(csConf, newRootName);
+
+    Resource clusterResource = Resources.createResource(100 * 16 * GB,
+        100 * 32);
     CapacitySchedulerContext csContext = mockCSContext(csConf, clusterResource);
     when(csContext.getRMContext()).thenReturn(rmContext);
-    csConf.setFloat(CapacitySchedulerConfiguration.
-        MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.1f);
-    ParentQueue root = new ParentQueue(csContext, 
-        CapacitySchedulerConfiguration.ROOT, null, null);
-    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 80);
-    LeafQueue a = new LeafQueue(csContext, A, root, null);
-    assertEquals(0.1f, a.getMaxAMResourcePerQueuePercent(), 1e-3f);
-    assertEquals(a.calculateAndGetAMResourceLimit(),
-        Resources.createResource(160 * GB, 1));
-    
-    csConf.setFloat(CapacitySchedulerConfiguration.
-        MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.2f);
-    LeafQueue newA = new LeafQueue(csContext, A, root, null);
-    a.reinitialize(newA, clusterResource);
-    assertEquals(0.2f, a.getMaxAMResourcePerQueuePercent(), 1e-3f);
-    assertEquals(a.calculateAndGetAMResourceLimit(),
-        Resources.createResource(320 * GB, 1));
+    csConf.setFloat(
+        CapacitySchedulerConfiguration.MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT,
+        0.1f);
+
+    CSQueue root;
+    root = CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
+        CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
-    Resource newClusterResource = Resources.createResource(100 * 20 * GB,
-        100 * 32);
-    a.updateClusterResource(newClusterResource, 
-        new ResourceLimits(newClusterResource));
-    //  100 * 20 * 0.2 = 400
-    assertEquals(a.calculateAndGetAMResourceLimit(),
-        Resources.createResource(400 * GB, 1));
+    // Manipulate queue 'a'
+    LeafQueue b = stubLeafQueue((LeafQueue) queues.get(B));
+    assertEquals(0.1f, b.getMaxAMResourcePerQueuePercent(), 1e-3f);
+    assertEquals(b.calculateAndGetAMResourceLimit(),
+        Resources.createResource(159 * GB, 1));
+
+    csConf.setFloat(
+        CapacitySchedulerConfiguration.MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT,
+        0.2f);
+    clusterResource = Resources.createResource(100 * 20 * GB, 100 * 32);
+    Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
+    CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext,
+        csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues,
+        TestUtils.spyHook);
+    root.reinitialize(newRoot, clusterResource);
+
+    b = stubLeafQueue((LeafQueue) newQueues.get(B));
+    assertEquals(b.calculateAndGetAMResourceLimit(),
+        Resources.createResource(320 * GB, 1));
   }
   
   @Test
@@ -3142,6 +3085,8 @@ public class TestLeafQueue {
     Resource clusterResource = Resources.createResource(numNodes * (16 * GB),
         numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     String user_0 = "user_0";
 
@@ -3308,6 +3253,8 @@ public class TestLeafQueue {
     Resource clusterResource = Resources.createResource(
         numNodes * (16*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     String user_0 = "user_0";
 
@@ -3435,6 +3382,8 @@ public class TestLeafQueue {
     Resource clusterResource = 
         Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
     
     // Setup resource-requests and submit
     // App0 has node local request for host_0/host_1, and app1 has node local
@@ -3533,6 +3482,8 @@ public class TestLeafQueue {
     Resource clusterResource =
         Resources.createResource(numNodes * (100*GB), numNodes * 128);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Pending resource requests for app_0 and app_1 total 5GB.
     Priority priority = TestUtils.createMockPriority(1);
@@ -3699,6 +3650,8 @@ public class TestLeafQueue {
     Resource clusterResource =
         Resources.createResource(numNodes * (100*GB), numNodes * 128);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Pending resource requests for user_0: app_0 and app_1 total 3GB.
     Priority priority = TestUtils.createMockPriority(1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 a9196d1..25a9774 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
@@ -249,6 +249,8 @@ public class TestParentQueue {
         Resources.createResource(numNodes * (memoryPerNode*GB),
             numNodes * coresPerNode);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Start testing
     LeafQueue a = (LeafQueue)queues.get(A);
@@ -494,6 +496,8 @@ public class TestParentQueue {
         Resources.createResource(numNodes * (memoryPerNode*GB), 
             numNodes * coresPerNode);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Start testing
     CSQueue a = queues.get(A);
@@ -710,6 +714,8 @@ public class TestParentQueue {
         Resources.createResource(numNodes * (memoryPerNode*GB),
             numNodes * coresPerNode);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Start testing
     LeafQueue a = (LeafQueue)queues.get(A);
@@ -790,6 +796,8 @@ public class TestParentQueue {
         Resources.createResource(numNodes * (memoryPerNode*GB),
             numNodes * coresPerNode);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Start testing
     LeafQueue b3 = (LeafQueue)queues.get(B3);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 32f022f..0855720 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
@@ -267,6 +267,8 @@ public class TestReservations {
     final int numNodes = 3;
     Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     Priority priorityAM = TestUtils.createMockPriority(1);
@@ -454,6 +456,8 @@ public class TestReservations {
     final int numNodes = 3;
     Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     Priority priorityAM = TestUtils.createMockPriority(1);
@@ -600,6 +604,8 @@ public class TestReservations {
     final int numNodes = 3;
     Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     Priority priorityAM = TestUtils.createMockPriority(1);
@@ -782,6 +788,8 @@ public class TestReservations {
     final int numNodes = 2;
     Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     Priority priorityAM = TestUtils.createMockPriority(1);
@@ -896,6 +904,10 @@ 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);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
 
     // Setup resource-requests
@@ -1068,6 +1080,8 @@ public class TestReservations {
     final int numNodes = 2;
     Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     Priority priorityAM = TestUtils.createMockPriority(1);
@@ -1256,6 +1270,8 @@ public class TestReservations {
     final int numNodes = 2;
     Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
 
     // Setup resource-requests
     Priority priorityAM = TestUtils.createMockPriority(1);
@@ -1418,6 +1434,9 @@ public class TestReservations {
     final int numNodes = 3;
     Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+    root.updateClusterResource(clusterResource,
+        new ResourceLimits(clusterResource));
+
 
     // Setup resource-requests
     Priority priorityAM = TestUtils.createMockPriority(1);


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


[29/50] [abbrv] hadoop git commit: YARN-7538. Fix performance regression introduced by Capacity Scheduler absolute min/max resource refactoring. (Sunil G via wangda)

Posted by wa...@apache.org.
YARN-7538. Fix performance regression introduced by Capacity Scheduler absolute min/max resource refactoring. (Sunil G via wangda)

Change-Id: Ic9bd7e599c56970fe01cb0e1bba6df7d1f77eb29


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

Branch: refs/heads/YARN-6592
Commit: b7b8cd53242da8d47ba4a6d99d906bdb2a1a3494
Parents: 7462c38
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Nov 23 19:52:05 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../scheduler/AbstractResourceUsage.java        | 68 +++++++++++---------
 .../scheduler/capacity/AbstractCSQueue.java     |  4 +-
 .../scheduler/capacity/CSQueueUtils.java        |  8 +--
 .../scheduler/capacity/ParentQueue.java         |  4 +-
 .../capacity/TestCapacityScheduler.java         |  2 +-
 5 files changed, 46 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7b8cd53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.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/AbstractResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java
index c295323..3203543 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicReferenceArray;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -38,17 +39,20 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 public class AbstractResourceUsage {
   protected ReadLock readLock;
   protected WriteLock writeLock;
-  protected Map<String, UsageByLabel> usages;
+  protected final Map<String, UsageByLabel> usages;
+  private final UsageByLabel noLabelUsages;
   // short for no-label :)
-  private static final String NL = CommonNodeLabelsManager.NO_LABEL;
 
   public AbstractResourceUsage() {
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     readLock = lock.readLock();
     writeLock = lock.writeLock();
 
-    usages = new HashMap<String, UsageByLabel>();
-    usages.put(NL, new UsageByLabel(NL));
+    usages = new HashMap<>();
+
+    // For default label, avoid map for faster access.
+    noLabelUsages = new UsageByLabel();
+    usages.put(CommonNodeLabelsManager.NO_LABEL, noLabelUsages);
   }
 
   // Usage enum here to make implement cleaner
@@ -57,41 +61,40 @@ public class AbstractResourceUsage {
     // be written by ordering policies
     USED(0), PENDING(1), AMUSED(2), RESERVED(3), CACHED_USED(4), CACHED_PENDING(
         5), AMLIMIT(6), MIN_RESOURCE(7), MAX_RESOURCE(8), EFF_MIN_RESOURCE(
-            9), EFF_MAX_RESOURCE(
-                10), EFF_MIN_RESOURCE_UP(11), EFF_MAX_RESOURCE_UP(12);
+            9), EFF_MAX_RESOURCE(10);
 
     private int idx;
 
-    private ResourceType(int value) {
+    ResourceType(int value) {
       this.idx = value;
     }
   }
 
   public static class UsageByLabel {
     // usage by label, contains all UsageType
-    private Resource[] resArr;
+    private final AtomicReferenceArray<Resource> resArr;
 
-    public UsageByLabel(String label) {
-      resArr = new Resource[ResourceType.values().length];
-      for (int i = 0; i < resArr.length; i++) {
-        resArr[i] = Resource.newInstance(0, 0);
-      };
+    public UsageByLabel() {
+      resArr = new AtomicReferenceArray<>(ResourceType.values().length);
+      for (int i = 0; i < resArr.length(); i++) {
+        resArr.set(i, Resource.newInstance(0, 0));
+      }
     }
 
     public Resource getUsed() {
-      return resArr[ResourceType.USED.idx];
+      return resArr.get(ResourceType.USED.idx);
     }
 
     @Override
     public String toString() {
       StringBuilder sb = new StringBuilder();
-      sb.append("{used=" + resArr[0] + "%, ");
-      sb.append("pending=" + resArr[1] + "%, ");
-      sb.append("am_used=" + resArr[2] + "%, ");
-      sb.append("reserved=" + resArr[3] + "%}");
-      sb.append("min_eff=" + resArr[9] + "%, ");
-      sb.append("max_eff=" + resArr[10] + "%}");
-      sb.append("min_effup=" + resArr[11] + "%, ");
+      sb.append("{used=" + resArr.get(ResourceType.USED.idx) + ", ");
+      sb.append("pending=" + resArr.get(ResourceType.PENDING.idx) + ", ");
+      sb.append("am_used=" + resArr.get(ResourceType.AMUSED.idx) + ", ");
+      sb.append("reserved=" + resArr.get(ResourceType.RESERVED.idx) + ", ");
+      sb.append("min_eff=" + resArr.get(ResourceType.EFF_MIN_RESOURCE.idx) + ", ");
+      sb.append(
+          "max_eff=" + resArr.get(ResourceType.EFF_MAX_RESOURCE.idx) + "}");
       return sb.toString();
     }
   }
@@ -104,8 +107,8 @@ public class AbstractResourceUsage {
   }
 
   protected Resource _get(String label, ResourceType type) {
-    if (label == null) {
-      label = RMNodeLabelsManager.NO_LABEL;
+    if (label == null || label.equals(RMNodeLabelsManager.NO_LABEL)) {
+      return normalize(noLabelUsages.resArr.get(type.idx));
     }
 
     try {
@@ -114,7 +117,7 @@ public class AbstractResourceUsage {
       if (null == usage) {
         return Resources.none();
       }
-      return normalize(usage.resArr[type.idx]);
+      return normalize(usage.resArr.get(type.idx));
     } finally {
       readLock.unlock();
     }
@@ -126,7 +129,7 @@ public class AbstractResourceUsage {
       Resource allOfType = Resources.createResource(0);
       for (Map.Entry<String, UsageByLabel> usageEntry : usages.entrySet()) {
         //all usages types are initialized
-        Resources.addTo(allOfType, usageEntry.getValue().resArr[type.idx]);
+        Resources.addTo(allOfType, usageEntry.getValue().resArr.get(type.idx));
       }
       return allOfType;
     } finally {
@@ -135,11 +138,12 @@ public class AbstractResourceUsage {
   }
 
   private UsageByLabel getAndAddIfMissing(String label) {
-    if (label == null) {
-      label = RMNodeLabelsManager.NO_LABEL;
+    if (label == null || label.equals(RMNodeLabelsManager.NO_LABEL)) {
+      return noLabelUsages;
     }
+
     if (!usages.containsKey(label)) {
-      UsageByLabel u = new UsageByLabel(label);
+      UsageByLabel u = new UsageByLabel();
       usages.put(label, u);
       return u;
     }
@@ -151,7 +155,7 @@ public class AbstractResourceUsage {
     try {
       writeLock.lock();
       UsageByLabel usage = getAndAddIfMissing(label);
-      usage.resArr[type.idx] = res;
+      usage.resArr.set(type.idx, res);
     } finally {
       writeLock.unlock();
     }
@@ -161,7 +165,8 @@ public class AbstractResourceUsage {
     try {
       writeLock.lock();
       UsageByLabel usage = getAndAddIfMissing(label);
-      Resources.addTo(usage.resArr[type.idx], res);
+      usage.resArr.set(type.idx,
+          Resources.add(usage.resArr.get(type.idx), res));
     } finally {
       writeLock.unlock();
     }
@@ -171,7 +176,8 @@ public class AbstractResourceUsage {
     try {
       writeLock.lock();
       UsageByLabel usage = getAndAddIfMissing(label);
-      Resources.subtractFrom(usage.resArr[type.idx], res);
+      usage.resArr.set(type.idx,
+          Resources.subtract(usage.resArr.get(type.idx), res));
     } finally {
       writeLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7b8cd53/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 9caf589..140ea5d 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
@@ -932,7 +932,7 @@ public abstract class AbstractCSQueue implements CSQueue {
     queueUsage.incUsed(nodeLabel, resourceToInc);
     CSQueueUtils.updateUsedCapacity(resourceCalculator,
         labelManager.getResourceByLabel(nodeLabel, Resources.none()),
-        Resources.none(), nodeLabel, this);
+        nodeLabel, this);
     if (null != parent) {
       parent.incUsedResource(nodeLabel, resourceToInc, null);
     }
@@ -948,7 +948,7 @@ public abstract class AbstractCSQueue implements CSQueue {
     queueUsage.decUsed(nodeLabel, resourceToDec);
     CSQueueUtils.updateUsedCapacity(resourceCalculator,
         labelManager.getResourceByLabel(nodeLabel, Resources.none()),
-        Resources.none(), nodeLabel, this);
+        nodeLabel, this);
     if (null != parent) {
       parent.decUsedResource(nodeLabel, resourceToDec, null);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7b8cd53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.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/CSQueueUtils.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/CSQueueUtils.java
index 81dec80..6daca51 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/CSQueueUtils.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/CSQueueUtils.java
@@ -180,8 +180,8 @@ class CSQueueUtils {
    * used resource for all partitions of this queue.
    */
   public static void updateUsedCapacity(final ResourceCalculator rc,
-      final Resource totalPartitionResource, Resource clusterResource,
-      String nodePartition, AbstractCSQueue childQueue) {
+      final Resource totalPartitionResource, String nodePartition,
+      AbstractCSQueue childQueue) {
     QueueCapacities queueCapacities = childQueue.getQueueCapacities();
     CSQueueMetrics queueMetrics = childQueue.getMetrics();
     ResourceUsage queueResourceUsage = childQueue.getQueueResourceUsage();
@@ -287,11 +287,11 @@ class CSQueueUtils {
       for (String partition : Sets.union(queueCapacities.getNodePartitionsSet(),
           queueResourceUsage.getNodePartitionsSet())) {
         updateUsedCapacity(rc, nlm.getResourceByLabel(partition, cluster),
-            cluster, partition, childQueue);
+            partition, childQueue);
       }
     } else {
       updateUsedCapacity(rc, nlm.getResourceByLabel(nodePartition, cluster),
-          cluster, nodePartition, childQueue);
+          nodePartition, childQueue);
     }
 
     // Update queue metrics w.r.t node labels. In a generic way, we can

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7b8cd53/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 a427fb1..c198d13 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
@@ -739,8 +739,8 @@ public class ParentQueue extends AbstractCSQueue {
         child.getQueueResourceUsage().getUsed(nodePartition));
 
     // Get child's max resource
-    Resource childConfiguredMaxResource = getEffectiveMaxCapacityDown(
-        nodePartition, minimumAllocation);
+    Resource childConfiguredMaxResource = child
+        .getEffectiveMaxCapacityDown(nodePartition, minimumAllocation);
 
     // Child's limit should be capped by child configured max resource
     childLimit =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7b8cd53/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 23a371a..e91f734 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
@@ -4307,7 +4307,7 @@ public class TestCapacityScheduler {
           null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
     }
-    assertEquals("P2 Used Resource should be 7 GB", 7 * GB,
+    assertEquals("P2 Used Resource should be 8 GB", 8 * GB,
         cs.getQueue("p2").getUsedResources().getMemorySize());
 
     //Free a container from X1


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


[40/50] [abbrv] hadoop git commit: YARN-7473. Implement Framework and policy for capacity management of auto created queues. (Suma Shivaprasad via wangda)

Posted by wa...@apache.org.
YARN-7473. Implement Framework and policy for capacity management of auto created queues. (Suma Shivaprasad via wangda)

Change-Id: Icca7805fe12f6f7fb335effff4b121b6f7f6337b


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

Branch: refs/heads/YARN-6592
Commit: b38643c9a8dd2c53024ae830b9565a550d0ec39c
Parents: 74665e3
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Dec 8 15:10:16 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Dec 8 15:10:16 2017 -0800

----------------------------------------------------------------------
 .../monitor/SchedulingMonitor.java              |   4 +-
 .../ProportionalCapacityPreemptionPolicy.java   |   7 +-
 .../monitor/capacity/TempQueuePerPartition.java |   8 +
 .../CapacitySchedulerPlanFollower.java          |  12 +-
 .../scheduler/AbstractYarnScheduler.java        |   2 +-
 .../scheduler/YarnScheduler.java                |   2 +-
 .../capacity/AbstractAutoCreatedLeafQueue.java  | 113 +++
 .../scheduler/capacity/AbstractCSQueue.java     | 101 ++-
 .../capacity/AbstractManagedParentQueue.java    | 162 ++--
 .../capacity/AutoCreatedLeafQueue.java          | 136 ++--
 .../capacity/AutoCreatedLeafQueueConfig.java    |  66 ++
 .../AutoCreatedQueueManagementPolicy.java       |  64 ++
 .../scheduler/capacity/CSQueue.java             |   2 +-
 .../scheduler/capacity/CSQueueUtils.java        |  33 +-
 .../scheduler/capacity/CapacityScheduler.java   | 104 +--
 .../CapacitySchedulerConfiguration.java         | 112 ++-
 .../capacity/CapacitySchedulerContext.java      |   8 +
 .../capacity/CapacitySchedulerQueueManager.java |   6 +-
 .../scheduler/capacity/LeafQueue.java           |  75 +-
 .../scheduler/capacity/ManagedParentQueue.java  | 294 +++++++-
 .../scheduler/capacity/PlanQueue.java           | 150 +++-
 .../capacity/QueueManagementChange.java         | 148 ++++
 .../QueueManagementDynamicEditPolicy.java       | 272 +++++++
 .../scheduler/capacity/ReservationQueue.java    |  91 +++
 .../GuaranteedOrZeroCapacityOverTimePolicy.java | 745 +++++++++++++++++++
 .../scheduler/common/QueueEntitlement.java      |  22 +
 .../event/QueueManagementChangeEvent.java       |  49 ++
 .../scheduler/event/SchedulerEventType.java     |   5 +-
 .../capacity/TestAutoCreatedLeafQueue.java      | 113 ---
 ...stCapacitySchedulerAutoCreatedQueueBase.java | 579 ++++++++++++++
 .../TestCapacitySchedulerAutoQueueCreation.java | 611 ++++++---------
 .../TestCapacitySchedulerDynamicBehavior.java   |  32 +-
 ...tGuaranteedOrZeroCapacityOverTimePolicy.java |  40 +
 .../scheduler/capacity/TestLeafQueue.java       | 125 +++-
 .../TestQueueManagementDynamicEditPolicy.java   | 121 +++
 .../capacity/TestReservationQueue.java          | 114 +++
 36 files changed, 3629 insertions(+), 899 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.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/monitor/SchedulingMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
index 09edb98..d1cc850 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
@@ -79,7 +79,7 @@ public class SchedulingMonitor extends AbstractService {
   }
 
   private void schedulePreemptionChecker() {
-    handler = ses.scheduleAtFixedRate(new PreemptionChecker(),
+    handler = ses.scheduleAtFixedRate(new PolicyInvoker(),
         0, monitorInterval, TimeUnit.MILLISECONDS);
   }
 
@@ -99,7 +99,7 @@ public class SchedulingMonitor extends AbstractService {
     scheduleEditPolicy.editSchedule();
   }
 
-  private class PreemptionChecker implements Runnable {
+  private class PolicyInvoker implements Runnable {
     @Override
     public void run() {
       try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.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/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 8327cb9..304d204 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -37,6 +37,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuot
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .ManagedParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptableQueue;
@@ -377,7 +380,9 @@ public class ProportionalCapacityPreemptionPolicy
   }
 
   private Set<String> getLeafQueueNames(TempQueuePerPartition q) {
-    if (q.children == null || q.children.isEmpty()) {
+    // If its a ManagedParentQueue, it might not have any children
+    if ((q.children == null || q.children.isEmpty())
+        && !(q.parentQueue instanceof ManagedParentQueue)) {
       return ImmutableSet.of(q.queueName);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.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/monitor/capacity/TempQueuePerPartition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
index 4d71223..fdeee52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
@@ -21,6 +21,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .ParentQueue;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -56,6 +59,7 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
   final ArrayList<TempQueuePerPartition> children;
   private Collection<TempAppPerPartition> apps;
   LeafQueue leafQueue;
+  ParentQueue parentQueue;
   boolean preemptionDisabled;
 
   protected Resource pendingDeductReserved;
@@ -90,6 +94,10 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
       pendingDeductReserved = Resources.createResource(0);
     }
 
+    if (ParentQueue.class.isAssignableFrom(queue.getClass())) {
+      parentQueue = (ParentQueue) queue;
+    }
+
     this.normalizedGuarantee = new double[ResourceUtils
         .getNumberOfKnownResourceTypes()];
     this.children = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
index 2e16689..7962d8e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
@@ -28,10 +28,12 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedLeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .ReservationQueue;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
@@ -92,8 +94,8 @@ public class CapacitySchedulerPlanFollower extends AbstractSchedulerPlanFollower
       String planQueueName, Queue queue, String currResId) {
     PlanQueue planQueue = (PlanQueue)queue;
     try {
-      AutoCreatedLeafQueue resQueue =
-          new AutoCreatedLeafQueue(cs, currResId, planQueue);
+      ReservationQueue resQueue =
+          new ReservationQueue(cs, currResId, planQueue);
       cs.addQueue(resQueue);
     } catch (SchedulerDynamicEditException e) {
       LOG.warn(
@@ -112,8 +114,8 @@ public class CapacitySchedulerPlanFollower extends AbstractSchedulerPlanFollower
     PlanQueue planQueue = (PlanQueue)queue;
     if (cs.getQueue(defReservationId) == null) {
       try {
-        AutoCreatedLeafQueue defQueue =
-            new AutoCreatedLeafQueue(cs, defReservationId, planQueue);
+        ReservationQueue defQueue =
+            new ReservationQueue(cs, defReservationId, planQueue);
         cs.addQueue(defQueue);
       } catch (SchedulerDynamicEditException e) {
         LOG.warn(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index d94efb1..cf5e13b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -457,7 +457,7 @@ public abstract class AbstractYarnScheduler
   }
 
   @Override
-  public void addQueue(Queue newQueue) throws YarnException {
+  public void addQueue(Queue newQueue) throws YarnException, IOException {
     throw new YarnException(getClass().getSimpleName()
         + " does not support this operation");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.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/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index 111998b..93ca7c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -272,7 +272,7 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
    * @param newQueue the queue being added.
    * @throws YarnException
    */
-  void addQueue(Queue newQueue) throws YarnException;
+  void addQueue(Queue newQueue) throws YarnException, IOException;
 
   /**
    * This method increase the entitlement for current queue (must respect

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractAutoCreatedLeafQueue.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/AbstractAutoCreatedLeafQueue.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/AbstractAutoCreatedLeafQueue.java
new file mode 100644
index 0000000..ac97d72
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractAutoCreatedLeafQueue.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.resourcemanager.scheduler.capacity;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common
+    .QueueEntitlement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
+    .NO_LABEL;
+
+/**
+ * Abstract class for dynamic auto created queues managed by an implementation
+ * of AbstractManagedParentQueue
+ */
+public class AbstractAutoCreatedLeafQueue extends LeafQueue {
+
+  protected AbstractManagedParentQueue parent;
+
+  public AbstractAutoCreatedLeafQueue(CapacitySchedulerContext cs,
+      String queueName, AbstractManagedParentQueue parent, CSQueue old)
+      throws IOException {
+    super(cs, queueName, parent, old);
+    this.parent = parent;
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbstractAutoCreatedLeafQueue.class);
+
+  public AbstractAutoCreatedLeafQueue(CapacitySchedulerContext cs,
+      CapacitySchedulerConfiguration leafQueueConfigs, String queueName,
+      AbstractManagedParentQueue parent, CSQueue old) throws IOException {
+    super(cs, leafQueueConfigs, queueName, parent, old);
+    this.parent = parent;
+  }
+
+  /**
+   * This methods to change capacity for a queue and adjusts its
+   * absoluteCapacity
+   *
+   * @param entitlement the new entitlement for the queue (capacity,
+   *                    maxCapacity, etc..)
+   * @throws SchedulerDynamicEditException
+   */
+  public void setEntitlement(QueueEntitlement entitlement)
+      throws SchedulerDynamicEditException {
+     setEntitlement(NO_LABEL, entitlement);
+  }
+
+  /**
+   * This methods to change capacity for a queue and adjusts its
+   * absoluteCapacity
+   *
+   * @param entitlement the new entitlement for the queue (capacity,
+   *                    maxCapacity, etc..)
+   * @throws SchedulerDynamicEditException
+   */
+  public void setEntitlement(String nodeLabel, QueueEntitlement entitlement)
+      throws SchedulerDynamicEditException {
+    try {
+      writeLock.lock();
+      float capacity = entitlement.getCapacity();
+      if (capacity < 0 || capacity > 1.0f) {
+        throw new SchedulerDynamicEditException(
+            "Capacity demand is not in the [0,1] range: " + capacity);
+      }
+      setCapacity(nodeLabel, capacity);
+      setAbsoluteCapacity(nodeLabel,
+          getParent().getQueueCapacities().
+              getAbsoluteCapacity(nodeLabel)
+              * getQueueCapacities().getCapacity(nodeLabel));
+      // note: we currently set maxCapacity to capacity
+      // this might be revised later
+      setMaxCapacity(nodeLabel, entitlement.getMaxCapacity());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("successfully changed to " + capacity + " for queue " + this
+            .getQueueName());
+      }
+
+      //update queue used capacity etc
+      CSQueueUtils.updateQueueStatistics(resourceCalculator,
+          csContext.getClusterResource(),
+          this, labelManager, nodeLabel);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  protected void setupConfigurableCapacities(QueueCapacities queueCapacities) {
+    CSQueueUtils.updateAndCheckCapacitiesByLabel(getQueuePath(),
+        queueCapacities, parent == null ? null : parent.getQueueCapacities());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/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 140ea5d..4df4cf2 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
@@ -128,27 +128,34 @@ public abstract class AbstractCSQueue implements CSQueue {
 
   public AbstractCSQueue(CapacitySchedulerContext cs,
       String queueName, CSQueue parent, CSQueue old) throws IOException {
+    this(cs, cs.getConfiguration(), queueName, parent, old);
+  }
+
+  public AbstractCSQueue(CapacitySchedulerContext cs,
+      CapacitySchedulerConfiguration configuration, String queueName,
+      CSQueue parent, CSQueue old) {
+
     this.labelManager = cs.getRMContext().getNodeLabelManager();
     this.parent = parent;
     this.queueName = queueName;
-    this.queuePath =
-      ((parent == null) ? "" : (parent.getQueuePath() + ".")) + this.queueName;
+    this.queuePath = ((parent == null) ? "" : (parent.getQueuePath() + "."))
+        + this.queueName;
     this.resourceCalculator = cs.getResourceCalculator();
     this.activitiesManager = cs.getActivitiesManager();
-    
+
     // must be called after parent and queueName is set
-    this.metrics =
-        old != null ? (CSQueueMetrics) old.getMetrics() : CSQueueMetrics
-            .forQueue(getQueuePath(), parent, cs.getConfiguration()
-                .getEnableUserMetrics(), cs.getConf());
+    this.metrics = old != null ?
+        (CSQueueMetrics) old.getMetrics() :
+        CSQueueMetrics.forQueue(getQueuePath(), parent,
+            configuration.getEnableUserMetrics(), cs.getConf());
 
     this.csContext = cs;
     this.minimumAllocation = csContext.getMinimumResourceCapability();
-    
+
     // initialize ResourceUsage
     queueUsage = new ResourceUsage();
     queueEntity = new PrivilegedEntity(EntityType.QUEUE, getQueuePath());
-    
+
     // initialize QueueCapacities
     queueCapacities = new QueueCapacities(parent == null);
 
@@ -159,11 +166,16 @@ public abstract class AbstractCSQueue implements CSQueue {
     readLock = lock.readLock();
     writeLock = lock.writeLock();
   }
-  
+
   protected void setupConfigurableCapacities() {
+    setupConfigurableCapacities(csContext.getConfiguration());
+  }
+
+  protected void setupConfigurableCapacities(
+      CapacitySchedulerConfiguration configuration) {
     CSQueueUtils.loadUpdateAndCheckCapacities(
         getQueuePath(),
-        csContext.getConfiguration(), 
+        configuration,
         queueCapacities,
         parent == null ? null : parent.getQueueCapacities());
   }
@@ -275,6 +287,29 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
   }
 
+  /**
+   * Set maximum capacity
+   * @param maximumCapacity new max capacity
+   */
+  void setMaxCapacity(String nodeLabel, float maximumCapacity) {
+    try {
+      writeLock.lock();
+      // Sanity check
+      CSQueueUtils.checkMaxCapacity(getQueueName(),
+          queueCapacities.getCapacity(nodeLabel), maximumCapacity);
+      float absMaxCapacity = CSQueueUtils.computeAbsoluteMaximumCapacity(
+          maximumCapacity, parent);
+      CSQueueUtils.checkAbsoluteCapacity(getQueueName(),
+          queueCapacities.getAbsoluteCapacity(nodeLabel), absMaxCapacity);
+
+      queueCapacities.setMaximumCapacity(maximumCapacity);
+      queueCapacities.setAbsoluteMaximumCapacity(absMaxCapacity);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+
   @Override
   public String getDefaultNodeLabelExpression() {
     return defaultLabelExpression;
@@ -282,13 +317,20 @@ public abstract class AbstractCSQueue implements CSQueue {
   
   void setupQueueConfigs(Resource clusterResource)
       throws IOException {
+    setupQueueConfigs(clusterResource, csContext.getConfiguration());
+  }
+
+  protected void setupQueueConfigs(Resource clusterResource,
+      CapacitySchedulerConfiguration configuration) throws
+      IOException {
+
     try {
       writeLock.lock();
       // get labels
       this.accessibleLabels =
-          csContext.getConfiguration().getAccessibleNodeLabels(getQueuePath());
+          configuration.getAccessibleNodeLabels(getQueuePath());
       this.defaultLabelExpression =
-          csContext.getConfiguration().getDefaultNodeLabelExpression(
+          configuration.getDefaultNodeLabelExpression(
               getQueuePath());
       this.resourceTypes = new HashSet<String>();
       for (AbsoluteResourceType type : AbsoluteResourceType.values()) {
@@ -308,7 +350,7 @@ public abstract class AbstractCSQueue implements CSQueue {
       }
 
       // After we setup labels, we can setup capacities
-      setupConfigurableCapacities();
+      setupConfigurableCapacities(configuration);
 
       // Also fetch minimum/maximum resource constraint for this queue if
       // configured.
@@ -316,20 +358,20 @@ public abstract class AbstractCSQueue implements CSQueue {
       updateConfigurableResourceRequirement(getQueuePath(), clusterResource);
 
       this.maximumAllocation =
-          csContext.getConfiguration().getMaximumAllocationPerQueue(
+          configuration.getMaximumAllocationPerQueue(
               getQueuePath());
 
       // initialized the queue state based on previous state, configured state
       // and its parent state.
       QueueState previous = getState();
-      QueueState configuredState = csContext.getConfiguration()
+      QueueState configuredState = configuration
           .getConfiguredState(getQueuePath());
       QueueState parentState = (parent == null) ? null : parent.getState();
       initializeQueueState(previous, configuredState, parentState);
 
       authorizer = YarnAuthorizationProvider.getInstance(csContext.getConf());
 
-      this.acls = csContext.getConfiguration().getAcls(getQueuePath());
+      this.acls = configuration.getAcls(getQueuePath());
 
       // Update metrics
       CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
@@ -361,18 +403,21 @@ public abstract class AbstractCSQueue implements CSQueue {
       this.reservationsContinueLooking =
           csContext.getConfiguration().getReservationContinueLook();
 
-      this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(this);
+      this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(this,
+          configuration);
 
-      this.priority = csContext.getConfiguration().getQueuePriority(
+      this.priority = configuration.getQueuePriority(
           getQueuePath());
 
-      this.userWeights = getUserWeightsFromHierarchy();
+      this.userWeights = getUserWeightsFromHierarchy(configuration);
     } finally {
       writeLock.unlock();
     }
   }
 
-  private Map<String, Float> getUserWeightsFromHierarchy() throws IOException {
+  private Map<String, Float> getUserWeightsFromHierarchy
+      (CapacitySchedulerConfiguration configuration) throws
+      IOException {
     Map<String, Float> unionInheritedWeights = new HashMap<String, Float>();
     CSQueue parentQ = getParent();
     if (parentQ != null) {
@@ -381,9 +426,8 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
     // Insert this queue's user's weights, overriding parent's user's weights if
     // there is overlap.
-    CapacitySchedulerConfiguration csConf = csContext.getConfiguration();
     unionInheritedWeights.putAll(
-        csConf.getAllUserWeightsForQueue(getQueuePath()));
+        configuration.getAllUserWeightsForQueue(getQueuePath()));
     return unionInheritedWeights;
   }
 
@@ -720,10 +764,11 @@ public abstract class AbstractCSQueue implements CSQueue {
    * 
    * @return true if queue has preemption disabled, false otherwise
    */
-  private boolean isQueueHierarchyPreemptionDisabled(CSQueue q) {
-    CapacitySchedulerConfiguration csConf = csContext.getConfiguration();
+  private boolean isQueueHierarchyPreemptionDisabled(CSQueue q,
+      CapacitySchedulerConfiguration configuration) {
     boolean systemWidePreemption =
-        csConf.getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
+        csContext.getConfiguration()
+            .getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
                        YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS);
     CSQueue parentQ = q.getParent();
 
@@ -735,14 +780,14 @@ public abstract class AbstractCSQueue implements CSQueue {
     // on, then q does not have preemption disabled (default=false, below)
     // unless the preemption_disabled property is explicitly set.
     if (parentQ == null) {
-      return csConf.getPreemptionDisabled(q.getQueuePath(), false);
+      return configuration.getPreemptionDisabled(q.getQueuePath(), false);
     }
 
     // If this is not the root queue, inherit the default value for the
     // preemption_disabled property from the parent. Preemptability will be
     // inherited from the parent's hierarchy unless explicitly overridden at
     // this level.
-    return csConf.getPreemptionDisabled(q.getQueuePath(),
+    return configuration.getPreemptionDisabled(q.getQueuePath(),
                                         parentQ.getPreemptionDisabled());
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractManagedParentQueue.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/AbstractManagedParentQueue.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/AbstractManagedParentQueue.java
index 46f5cf1..9d38f79 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/AbstractManagedParentQueue.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/AbstractManagedParentQueue.java
@@ -17,13 +17,21 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common
+    .QueueEntitlement;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.Comparator;
 import java.util.Iterator;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
 
 /**
  * A container class for automatically created child leaf queues.
@@ -35,13 +43,12 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
   private static final Logger LOG = LoggerFactory.getLogger(
       AbstractManagedParentQueue.class);
 
-  protected AutoCreatedLeafQueueTemplate leafQueueTemplate;
+  protected AutoCreatedLeafQueueConfig leafQueueTemplate;
+  protected AutoCreatedQueueManagementPolicy queueManagementPolicy = null;
 
   public AbstractManagedParentQueue(CapacitySchedulerContext cs,
       String queueName, CSQueue parent, CSQueue old) throws IOException {
     super(cs, queueName, parent, old);
-
-    super.setupQueueConfigs(csContext.getClusterResource());
   }
 
   @Override
@@ -53,52 +60,18 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
       // Set new configs
       setupQueueConfigs(clusterResource);
 
-      // run reinitialize on each existing queue, to trigger absolute cap
-      // recomputations
-      for (CSQueue res : this.getChildQueues()) {
-        res.reinitialize(res, clusterResource);
-      }
     } finally {
       writeLock.unlock();
     }
   }
 
   /**
-   * Initialize leaf queue configs from template configurations specified on
-   * parent queue.
-   */
-  protected AutoCreatedLeafQueueTemplate.Builder initializeLeafQueueConfigs
-    (String queuePath) {
-
-    CapacitySchedulerConfiguration conf = csContext.getConfiguration();
-
-    AutoCreatedLeafQueueTemplate.Builder leafQueueTemplateBuilder = new
-        AutoCreatedLeafQueueTemplate.Builder();
-    int maxApps = conf.getMaximumApplicationsPerQueue(queuePath);
-    if (maxApps < 0) {
-      maxApps = (int) (
-          CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS
-              * getAbsoluteCapacity());
-    }
-
-    int userLimit = conf.getUserLimit(queuePath);
-    float userLimitFactor = conf.getUserLimitFactor(queuePath);
-    leafQueueTemplateBuilder.userLimit(userLimit)
-          .userLimitFactor(userLimitFactor)
-          .maxApps(maxApps)
-          .maxAppsPerUser(
-              (int) (maxApps * (userLimit / 100.0f) * userLimitFactor));
-
-    return leafQueueTemplateBuilder;
-  }
-
-  /**
    * Add the specified child queue.
    * @param childQueue reference to the child queue to be added
    * @throws SchedulerDynamicEditException
    */
   public void addChildQueue(CSQueue childQueue)
-      throws SchedulerDynamicEditException {
+      throws SchedulerDynamicEditException, IOException {
     try {
       writeLock.lock();
       if (childQueue.getCapacity() > 0) {
@@ -193,84 +166,69 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
     }
   }
 
-  public static class AutoCreatedLeafQueueTemplate {
-
-    private QueueCapacities queueCapacities;
-
-    private int maxApps;
-    private int maxAppsPerUser;
-    private int userLimit;
-    private float userLimitFactor;
-
-    AutoCreatedLeafQueueTemplate(Builder builder) {
-      this.maxApps = builder.maxApps;
-      this.maxAppsPerUser = builder.maxAppsPerUser;
-      this.userLimit = builder.userLimit;
-      this.userLimitFactor = builder.userLimitFactor;
-      this.queueCapacities = builder.queueCapacities;
-    }
-
-    public static class Builder {
-      private int maxApps;
-      private int maxAppsPerUser;
+  public AutoCreatedLeafQueueConfig getLeafQueueTemplate() {
+    return leafQueueTemplate;
+  }
 
-      private int userLimit;
-      private float userLimitFactor;
+  public AutoCreatedQueueManagementPolicy
+  getAutoCreatedQueueManagementPolicy() {
+    return queueManagementPolicy;
+  }
 
-      private QueueCapacities queueCapacities;
+  protected SortedMap<String, String> getConfigurationsWithPrefix
+      (SortedMap<String, String> sortedConfigs, String prefix) {
+    return sortedConfigs.subMap( prefix, prefix + Character.MAX_VALUE );
+  }
 
-      Builder maxApps(int maxApplications) {
-        this.maxApps =  maxApplications;
-        return this;
-      }
+  protected SortedMap<String, String> sortCSConfigurations() {
+    SortedMap<String, String> sortedConfigs = new TreeMap(
+        new Comparator<String>() {
+          public int compare(String s1, String s2) {
+            return s1.compareToIgnoreCase(s2);
+          }
 
-      Builder maxAppsPerUser(int maxApplicationsPerUser) {
-        this.maxAppsPerUser = maxApplicationsPerUser;
-        return this;
-      }
+        });
 
-      Builder userLimit(int usrLimit) {
-        this.userLimit = usrLimit;
-        return this;
-      }
+    for (final Iterator<Map.Entry<String, String>> iterator =
+         csContext.getConfiguration().iterator(); iterator.hasNext(); ) {
+      final Map.Entry<String, String> confKeyValuePair = iterator.next();
+      sortedConfigs.put(confKeyValuePair.getKey(), confKeyValuePair.getValue());
+    }
+    return sortedConfigs;
+  }
 
-      Builder userLimitFactor(float ulf) {
-        this.userLimitFactor = ulf;
-        return this;
-      }
+  protected CapacitySchedulerConfiguration initializeLeafQueueConfigs(String
+      configPrefix) {
 
-      Builder capacities(QueueCapacities capacities) {
-        this.queueCapacities = capacities;
-        return this;
-      }
+    CapacitySchedulerConfiguration leafQueueConfigs = new
+        CapacitySchedulerConfiguration(new Configuration(false), false);
 
-      AutoCreatedLeafQueueTemplate build() {
-        return new AutoCreatedLeafQueueTemplate(this);
-      }
-    }
+    SortedMap<String, String> sortedConfigs = sortCSConfigurations();
+    SortedMap<String, String> templateConfigs = getConfigurationsWithPrefix
+        (sortedConfigs, configPrefix);
 
-    public int getUserLimit() {
-      return userLimit;
+    for (final Iterator<Map.Entry<String, String>> iterator =
+         templateConfigs.entrySet().iterator(); iterator.hasNext(); ) {
+      Map.Entry<String, String> confKeyValuePair = iterator.next();
+      leafQueueConfigs.set(confKeyValuePair.getKey(),
+          confKeyValuePair.getValue());
     }
 
-    public float getUserLimitFactor() {
-      return userLimitFactor;
-    }
+    return leafQueueConfigs;
+  }
 
-    public QueueCapacities getQueueCapacities() {
-      return queueCapacities;
-    }
+  protected void validateQueueEntitlementChange(AbstractAutoCreatedLeafQueue
+      leafQueue, QueueEntitlement entitlement)
+      throws SchedulerDynamicEditException {
 
-    public int getMaxApps() {
-      return maxApps;
-    }
+    float sumChilds = sumOfChildCapacities();
+    float newChildCap =
+        sumChilds - leafQueue.getCapacity() + entitlement.getCapacity();
 
-    public int getMaxAppsPerUser() {
-      return maxAppsPerUser;
+    if (!(newChildCap >= 0 && newChildCap < 1.0f + CSQueueUtils.EPSILON)) {
+      throw new SchedulerDynamicEditException(
+          "Sum of child queues should exceed 100% for auto creating parent "
+              + "queue : " + queueName);
     }
   }
-
-  public AutoCreatedLeafQueueTemplate getLeafQueueTemplate() {
-    return leafQueueTemplate;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedLeafQueue.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/AutoCreatedLeafQueue.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/AutoCreatedLeafQueue.java
index bc206d4..1d796ad 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/AutoCreatedLeafQueue.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/AutoCreatedLeafQueue.java
@@ -21,36 +21,27 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
-    .AbstractManagedParentQueue.AutoCreatedLeafQueueTemplate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
 /**
- * Leaf queues which are auto created by an underkying implementation of
+ * Leaf queues which are auto created by an underlying implementation of
  * AbstractManagedParentQueue. Eg: PlanQueue for reservations or
  * ManagedParentQueue for auto created dynamic queues
  */
-public class AutoCreatedLeafQueue extends LeafQueue {
+public class AutoCreatedLeafQueue extends AbstractAutoCreatedLeafQueue {
 
   private static final Logger LOG = LoggerFactory
       .getLogger(AutoCreatedLeafQueue.class);
 
-  private AbstractManagedParentQueue parent;
-
   public AutoCreatedLeafQueue(CapacitySchedulerContext cs, String queueName,
-      AbstractManagedParentQueue parent) throws IOException {
-    super(cs, queueName, parent, null);
-
-    AutoCreatedLeafQueueTemplate leafQueueTemplate =
-        parent.getLeafQueueTemplate();
-    updateApplicationAndUserLimits(leafQueueTemplate.getUserLimit(),
-        leafQueueTemplate.getUserLimitFactor(),
-        leafQueueTemplate.getMaxApps(),
-        leafQueueTemplate.getMaxAppsPerUser());
-    this.parent = parent;
+      ManagedParentQueue parent) throws IOException {
+    super(cs, parent.getLeafQueueConfigs(queueName),
+        queueName,
+        parent, null);
+    updateCapacitiesToZero();
   }
 
   @Override
@@ -61,48 +52,75 @@ public class AutoCreatedLeafQueue extends LeafQueue {
 
       validate(newlyParsedQueue);
 
-      super.reinitialize(newlyParsedQueue, clusterResource);
-      CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
-          this, labelManager, null);
+      ManagedParentQueue managedParentQueue = (ManagedParentQueue) parent;
 
-      AutoCreatedLeafQueueTemplate leafQueueTemplate =
-          parent.getLeafQueueTemplate();
-      updateApplicationAndUserLimits(leafQueueTemplate.getUserLimit(),
-          leafQueueTemplate.getUserLimitFactor(),
-          leafQueueTemplate.getMaxApps(),
-          leafQueueTemplate.getMaxAppsPerUser());
+      super.reinitialize(newlyParsedQueue, clusterResource, managedParentQueue
+          .getLeafQueueConfigs(newlyParsedQueue.getQueueName()));
+
+      //Reset capacities to 0 since reinitialize above
+      // queueCapacities to initialize to configured capacity which might
+      // overcommit resources from parent queue
+      updateCapacitiesToZero();
 
     } finally {
       writeLock.unlock();
     }
   }
 
-  /**
-   * This methods to change capacity for a queue and adjusts its
-   * absoluteCapacity.
-   *
-   * @param entitlement the new entitlement for the queue (capacity,
-   *                    maxCapacity)
-   * @throws SchedulerDynamicEditException
-   */
-  public void setEntitlement(QueueEntitlement entitlement)
-      throws SchedulerDynamicEditException {
+  public void reinitializeFromTemplate(AutoCreatedLeafQueueConfig
+      leafQueueTemplate) throws SchedulerDynamicEditException, IOException {
+
     try {
       writeLock.lock();
-      float capacity = entitlement.getCapacity();
+
+      // TODO:
+      // reinitialize only capacities for now since 0 capacity updates
+      // can cause
+      // abs capacity related config computations to be incorrect if we go
+      // through reinitialize
+      QueueCapacities capacities = leafQueueTemplate.getQueueCapacities();
+
+      //update abs capacities
+      setupConfigurableCapacities(capacities);
+
+      //reset capacities for the leaf queue
+      mergeCapacities(capacities);
+
+      //update queue used capacity for all the node labels
+      CSQueueUtils.updateQueueStatistics(resourceCalculator,
+          csContext.getClusterResource(),
+          this, labelManager, null);
+
+      //activate applications if any are pending
+      activateApplications();
+
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private void mergeCapacities(QueueCapacities capacities) {
+    for ( String nodeLabel : capacities.getExistingNodeLabels()) {
+      this.queueCapacities.setCapacity(nodeLabel,
+          capacities.getCapacity(nodeLabel));
+      this.queueCapacities.setAbsoluteCapacity(nodeLabel, capacities
+          .getAbsoluteCapacity(nodeLabel));
+      this.queueCapacities.setMaximumCapacity(nodeLabel, capacities
+          .getMaximumCapacity(nodeLabel));
+      this.queueCapacities.setAbsoluteMaximumCapacity(nodeLabel, capacities
+          .getAbsoluteMaximumCapacity(nodeLabel));
+    }
+  }
+
+  public void validateConfigurations(AutoCreatedLeafQueueConfig template)
+      throws SchedulerDynamicEditException {
+    QueueCapacities capacities = template.getQueueCapacities();
+    for (String label : capacities.getExistingNodeLabels()) {
+      float capacity = capacities.getCapacity(label);
       if (capacity < 0 || capacity > 1.0f) {
         throw new SchedulerDynamicEditException(
             "Capacity demand is not in the [0,1] range: " + capacity);
       }
-      setCapacity(capacity);
-      setAbsoluteCapacity(getParent().getAbsoluteCapacity() * getCapacity());
-      setMaxCapacity(entitlement.getMaxCapacity());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("successfully changed to " + capacity + " for queue " + this
-            .getQueueName());
-      }
-    } finally {
-      writeLock.unlock();
     }
   }
 
@@ -113,22 +131,20 @@ public class AutoCreatedLeafQueue extends LeafQueue {
           "Error trying to reinitialize " + getQueuePath() + " from "
               + newlyParsedQueue.getQueuePath());
     }
-
-  }
-
-  @Override
-  protected void setupConfigurableCapacities() {
-    CSQueueUtils.updateAndCheckCapacitiesByLabel(getQueuePath(),
-        queueCapacities, parent == null ? null : parent.getQueueCapacities());
   }
 
-  private void updateApplicationAndUserLimits(int userLimit,
-      float userLimitFactor,
-      int maxAppsForAutoCreatedQueues,
-      int maxAppsPerUserForAutoCreatedQueues) {
-    setUserLimit(userLimit);
-    setUserLimitFactor(userLimitFactor);
-    setMaxApplications(maxAppsForAutoCreatedQueues);
-    setMaxApplicationsPerUser(maxAppsPerUserForAutoCreatedQueues);
+  private void updateCapacitiesToZero() throws IOException {
+    try {
+      for( String nodeLabel : parent.getQueueCapacities().getExistingNodeLabels
+          ()) {
+        //TODO - update to use getMaximumCapacity(nodeLabel) in YARN-7574
+        setEntitlement(nodeLabel, new QueueEntitlement(0.0f,
+            parent.getLeafQueueTemplate()
+                .getQueueCapacities()
+                .getMaximumCapacity()));
+      }
+    } catch (SchedulerDynamicEditException e) {
+      throw new IOException(e);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedLeafQueueConfig.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/AutoCreatedLeafQueueConfig.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/AutoCreatedLeafQueueConfig.java
new file mode 100644
index 0000000..5952250
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedLeafQueueConfig.java
@@ -0,0 +1,66 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scheduler.capacity;
+
+/**
+ * Auto Created Leaf queue configurations, capacity
+ */
+public class AutoCreatedLeafQueueConfig {
+
+  /**
+   * Template queue capacities - contains configured and derived capacities
+   * like abs capacity which are used by auto queue creation policy to manage
+   * leaf queue capacities
+   */
+  private QueueCapacities queueCapacities;
+
+  private CapacitySchedulerConfiguration leafQueueConfigs;
+
+  public AutoCreatedLeafQueueConfig(Builder builder) {
+    this.queueCapacities = builder.queueCapacities;
+    this.leafQueueConfigs = builder.leafQueueConfigs;
+  }
+
+  public static class Builder {
+
+    private QueueCapacities queueCapacities;
+    private CapacitySchedulerConfiguration leafQueueConfigs;
+
+    public Builder capacities(QueueCapacities capacities) {
+      this.queueCapacities = capacities;
+      return this;
+    }
+
+    public Builder configuration(CapacitySchedulerConfiguration conf) {
+      this.leafQueueConfigs = conf;
+      return this;
+    }
+
+    public AutoCreatedLeafQueueConfig build() {
+      return new AutoCreatedLeafQueueConfig(this);
+    }
+  }
+
+  public QueueCapacities getQueueCapacities() {
+    return queueCapacities;
+  }
+
+  public CapacitySchedulerConfiguration getLeafQueueConfigs() {
+    return leafQueueConfigs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedQueueManagementPolicy.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/AutoCreatedQueueManagementPolicy.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/AutoCreatedQueueManagementPolicy.java
new file mode 100644
index 0000000..f7a4bbd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedQueueManagementPolicy.java
@@ -0,0 +1,64 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scheduler.capacity;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import java.util.List;
+
+public interface AutoCreatedQueueManagementPolicy {
+
+  /**
+   * Initialize policy
+   * @param schedulerContext Capacity Scheduler context
+   */
+  void init(CapacitySchedulerContext schedulerContext, ParentQueue parentQueue);
+
+  /**
+   * Reinitialize policy state ( if required )
+   * @param schedulerContext Capacity Scheduler context
+   */
+  void reinitialize(CapacitySchedulerContext schedulerContext,
+      ParentQueue parentQueue);
+
+  /**
+   * Get initial template for the specified leaf queue
+   * @param leafQueue the leaf queue
+   * @return initial leaf queue template configurations and capacities for
+   * auto created queue
+   */
+  AutoCreatedLeafQueueConfig getInitialLeafQueueConfiguration(
+      AbstractAutoCreatedLeafQueue leafQueue)
+      throws SchedulerDynamicEditException;
+
+  /**
+   * Compute/Adjust child queue capacities
+   * for auto created leaf queues
+   *
+   * @return returns a list of suggested QueueEntitlementChange(s) which may
+   * or may not be be enforced by the scheduler
+   */
+  List<QueueManagementChange> computeQueueManagementChanges()
+      throws SchedulerDynamicEditException;
+
+  /**
+   * Commit/Update state for the specified queue management changes.
+   */
+  void commitQueueManagementChanges(
+      List<QueueManagementChange> queueManagementChanges)
+      throws SchedulerDynamicEditException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/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 6d79b6a..5dd307c 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
@@ -230,7 +230,7 @@ public interface CSQueue extends SchedulerQueue<CSQueue> {
    * @param newlyParsedQueue new queue to re-initalize from
    * @param clusterResource resources in the cluster
    */
-  public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) 
+  public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
   throws IOException;
 
    /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.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/CSQueueUtils.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/CSQueueUtils.java
index 51e5b17..3901398 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/CSQueueUtils.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/CSQueueUtils.java
@@ -29,9 +29,9 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.collect.Sets;
 
-class CSQueueUtils {
+public class CSQueueUtils {
 
-  final static float EPSILON = 0.0001f;
+  public final static float EPSILON = 0.0001f;
   
   /*
    * Used only by tests
@@ -123,12 +123,12 @@ class CSQueueUtils {
 
     for (String label : configuredNodelabels) {
       if (label.equals(CommonNodeLabelsManager.NO_LABEL)) {
-        queueCapacities.setCapacity(CommonNodeLabelsManager.NO_LABEL,
+        queueCapacities.setCapacity(label,
             csConf.getNonLabeledQueueCapacity(queuePath) / 100);
-        queueCapacities.setMaximumCapacity(CommonNodeLabelsManager.NO_LABEL,
+        queueCapacities.setMaximumCapacity(label,
             csConf.getNonLabeledQueueMaximumCapacity(queuePath) / 100);
         queueCapacities.setMaxAMResourcePercentage(
-            CommonNodeLabelsManager.NO_LABEL,
+            label,
             csConf.getMaximumAMResourcePercentPerPartition(queuePath, label));
       } else {
         queueCapacities.setCapacity(label,
@@ -183,9 +183,32 @@ class CSQueueUtils {
 
     if (Resources.greaterThan(rc, totalPartitionResource,
         totalPartitionResource, Resources.none())) {
+
       Resource queueGuranteedResource = childQueue
           .getEffectiveCapacity(nodePartition);
 
+      //TODO : Modify below code to support Absolute Resource configurations
+      // (YARN-5881) for AutoCreatedLeafQueue
+      if (Float.compare(queueCapacities.getAbsoluteCapacity
+              (nodePartition), 0f) == 0
+          && childQueue instanceof AutoCreatedLeafQueue) {
+
+        //If absolute capacity is 0 for a leaf queue (could be a managed leaf
+        // queue, then use the leaf queue's template capacity to compute
+        // guaranteed resource for used capacity)
+
+        // queueGuaranteed = totalPartitionedResource *
+        // absolute_capacity(partition)
+        ManagedParentQueue parentQueue = (ManagedParentQueue)
+            childQueue.getParent();
+        QueueCapacities leafQueueTemplateCapacities = parentQueue
+            .getLeafQueueTemplate()
+            .getQueueCapacities();
+        queueGuranteedResource = Resources.multiply(totalPartitionResource,
+            leafQueueTemplateCapacities.getAbsoluteCapacity
+                (nodePartition));
+      }
+
       // make queueGuranteed >= minimum_allocation to avoid divided by 0.
       queueGuranteedResource =
           Resources.max(rc, totalPartitionResource, queueGuranteedResource,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/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 de93a6a..a5efd9f 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
@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
+
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -129,6 +130,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemoved
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
+    .QueueManagementChangeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ReleaseContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@@ -138,6 +141,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimpleC
 import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.Lock;
+import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -175,6 +179,8 @@ public class CapacityScheduler extends
 
   private CSConfigurationProvider csConfProvider;
 
+  protected Clock monotonicClock;
+
   @Override
   public void setConf(Configuration conf) {
       yarnConf = conf;
@@ -1501,7 +1507,7 @@ public class CapacityScheduler extends
     {
       NodeLabelsUpdateSchedulerEvent labelUpdateEvent =
           (NodeLabelsUpdateSchedulerEvent) event;
-      
+
       updateNodeLabelsAndQueueResource(labelUpdateEvent);
     }
     break;
@@ -1613,6 +1619,25 @@ public class CapacityScheduler extends
       }
     }
     break;
+    case MANAGE_QUEUE:
+    {
+      QueueManagementChangeEvent queueManagementChangeEvent =
+          (QueueManagementChangeEvent) event;
+      ParentQueue parentQueue = queueManagementChangeEvent.getParentQueue();
+      try {
+        final List<QueueManagementChange> queueManagementChanges =
+            queueManagementChangeEvent.getQueueManagementChanges();
+        ((ManagedParentQueue) parentQueue)
+            .validateAndApplyQueueManagementChanges(queueManagementChanges);
+      } catch (SchedulerDynamicEditException sde) {
+        LOG.error("Queue Management Change event cannot be applied for "
+            + "parent queue : " + parentQueue.getQueueName(), sde);
+      } catch (IOException ioe) {
+        LOG.error("Queue Management Change event cannot be applied for "
+            + "parent queue : " + parentQueue.getQueueName(), ioe);
+      }
+    }
+    break;
     default:
       LOG.error("Invalid eventtype " + event.getType() + ". Ignoring!");
     }
@@ -1976,12 +2001,14 @@ public class CapacityScheduler extends
       writeLock.lock();
       LOG.info("Removing queue: " + queueName);
       CSQueue q = this.getQueue(queueName);
-      if (!(q instanceof AutoCreatedLeafQueue)) {
+      if (!(AbstractAutoCreatedLeafQueue.class.isAssignableFrom(
+          q.getClass()))) {
         throw new SchedulerDynamicEditException(
             "The queue that we are asked " + "to remove (" + queueName
-                + ") is not a AutoCreatedLeafQueue");
+                + ") is not a AutoCreatedLeafQueue or ReservationQueue");
       }
-      AutoCreatedLeafQueue disposableLeafQueue = (AutoCreatedLeafQueue) q;
+      AbstractAutoCreatedLeafQueue disposableLeafQueue =
+          (AbstractAutoCreatedLeafQueue) q;
       // at this point we should have no more apps
       if (disposableLeafQueue.getNumApplications() > 0) {
         throw new SchedulerDynamicEditException(
@@ -1994,8 +2021,8 @@ public class CapacityScheduler extends
       ((AbstractManagedParentQueue) disposableLeafQueue.getParent())
           .removeChildQueue(q);
       this.queueManager.removeQueue(queueName);
-      LOG.info("Removal of AutoCreatedLeafQueue "
-          + queueName + " has succeeded");
+      LOG.info(
+          "Removal of AutoCreatedLeafQueue " + queueName + " has succeeded");
     } finally {
       writeLock.unlock();
     }
@@ -2003,22 +2030,27 @@ public class CapacityScheduler extends
 
   @Override
   public void addQueue(Queue queue)
-      throws SchedulerDynamicEditException {
+      throws SchedulerDynamicEditException, IOException {
     try {
       writeLock.lock();
-      if (!(queue instanceof AutoCreatedLeafQueue)) {
+      if (queue == null) {
+        throw new SchedulerDynamicEditException(
+            "Queue specified is null. Should be an implementation of "
+                + "AbstractAutoCreatedLeafQueue");
+      } else if (!(AbstractAutoCreatedLeafQueue.class
+          .isAssignableFrom(queue.getClass()))) {
         throw new SchedulerDynamicEditException(
-            "Queue " + queue.getQueueName() + " is not a AutoCreatedLeafQueue");
+            "Queue is not an implementation of "
+                + "AbstractAutoCreatedLeafQueue : " + queue.getClass());
       }
 
-      AutoCreatedLeafQueue newQueue = (AutoCreatedLeafQueue) queue;
+      AbstractAutoCreatedLeafQueue newQueue =
+          (AbstractAutoCreatedLeafQueue) queue;
 
-      if (newQueue.getParent() == null
-          || !(AbstractManagedParentQueue.class.
+      if (newQueue.getParent() == null || !(AbstractManagedParentQueue.class.
           isAssignableFrom(newQueue.getParent().getClass()))) {
         throw new SchedulerDynamicEditException(
-            "ParentQueue for " + newQueue.getQueueName()
-                + " is not properly set"
+            "ParentQueue for " + newQueue + " is not properly set"
                 + " (should be set and be a PlanQueue or ManagedParentQueue)");
       }
 
@@ -2027,6 +2059,7 @@ public class CapacityScheduler extends
       String queuename = newQueue.getQueueName();
       parentPlan.addChildQueue(newQueue);
       this.queueManager.addQueue(queuename, newQueue);
+
       LOG.info("Creation of AutoCreatedLeafQueue " + newQueue + " succeeded");
     } finally {
       writeLock.unlock();
@@ -2039,48 +2072,32 @@ public class CapacityScheduler extends
     try {
       writeLock.lock();
       LeafQueue queue = this.queueManager.getAndCheckLeafQueue(inQueue);
-      AbstractManagedParentQueue parent = (AbstractManagedParentQueue) queue
-          .getParent();
+      AbstractManagedParentQueue parent =
+          (AbstractManagedParentQueue) queue.getParent();
 
-      if (!(queue instanceof AutoCreatedLeafQueue)) {
+      if (!(AbstractAutoCreatedLeafQueue.class.isAssignableFrom(
+          queue.getClass()))) {
         throw new SchedulerDynamicEditException(
             "Entitlement can not be" + " modified dynamically since queue "
                 + inQueue + " is not a AutoCreatedLeafQueue");
       }
 
-      if (parent == null
-          || !(AbstractManagedParentQueue.class.isAssignableFrom(
-              parent.getClass()))) {
+      if (parent == null || !(AbstractManagedParentQueue.class.isAssignableFrom(
+          parent.getClass()))) {
         throw new SchedulerDynamicEditException(
             "The parent of AutoCreatedLeafQueue " + inQueue
                 + " must be a PlanQueue/ManagedParentQueue");
       }
 
-      AutoCreatedLeafQueue newQueue = (AutoCreatedLeafQueue) queue;
+      AbstractAutoCreatedLeafQueue newQueue =
+          (AbstractAutoCreatedLeafQueue) queue;
+      parent.validateQueueEntitlementChange(newQueue, entitlement);
 
-      float sumChilds = parent.sumOfChildCapacities();
-      float newChildCap =
-          sumChilds - queue.getCapacity() + entitlement.getCapacity();
+      newQueue.setEntitlement(entitlement);
 
-      if (newChildCap >= 0 && newChildCap < 1.0f + CSQueueUtils.EPSILON) {
-        // note: epsilon checks here are not ok, as the epsilons might
-        // accumulate and become a problem in aggregate
-        if (Math.abs(entitlement.getCapacity() - queue.getCapacity()) == 0
-            && Math.abs(
-            entitlement.getMaxCapacity() - queue.getMaximumCapacity()) == 0) {
-          return;
-        }
-        newQueue.setEntitlement(entitlement);
-      } else{
-        throw new SchedulerDynamicEditException(
-            "Sum of child queues should exceed 100% for auto creating parent "
-                + "queue : " + parent.getQueueName());
-      }
-      LOG.info(
-          "Set entitlement for AutoCreatedLeafQueue " + inQueue
-              + "  to " + queue.getCapacity() +
-              " request was (" + entitlement.getCapacity()
-              + ")");
+      LOG.info("Set entitlement for AutoCreatedLeafQueue " + inQueue + "  to "
+          + queue.getCapacity() + " request was (" + entitlement.getCapacity()
+          + ")");
     } finally {
       writeLock.unlock();
     }
@@ -2718,7 +2735,6 @@ public class CapacityScheduler extends
 
         addQueue(autoCreatedLeafQueue);
 
-        //TODO - Set entitlement through capacity management policy
       } else{
         throw new SchedulerDynamicEditException(
             "Could not auto-create leaf queue for " + leafQueueName

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index a33d81a..8aa41ee 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -923,6 +923,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     return getInt(NODE_LOCALITY_DELAY, DEFAULT_NODE_LOCALITY_DELAY);
   }
 
+  @VisibleForTesting
+  public void setNodeLocalityDelay(int nodeLocalityDelay) {
+    setInt(NODE_LOCALITY_DELAY, nodeLocalityDelay);
+  }
+
   public int getRackLocalityAdditionalDelay() {
     return getInt(RACK_LOCALITY_ADDITIONAL_DELAY,
         DEFAULT_RACK_LOCALITY_ADDITIONAL_DELAY);
@@ -1401,6 +1406,10 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     return maxApplicationsPerQueue;
   }
 
+  public void setGlobalMaximumApplicationsPerQueue(int val) {
+    setInt(QUEUE_GLOBAL_MAX_APPLICATION, val);
+  }
+
   /**
    * Ordering policy inside a parent queue to sort queues
    */
@@ -1621,8 +1630,12 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public static final boolean DEFAULT_AUTO_CREATE_CHILD_QUEUE_ENABLED = false;
 
   @Private
+  private static final String AUTO_CREATE_CHILD_QUEUE_PREFIX =
+      "auto-create-child-queue.";
+
+  @Private
   public static final String AUTO_CREATE_CHILD_QUEUE_ENABLED =
-      "auto-create-child-queue.enabled";
+      AUTO_CREATE_CHILD_QUEUE_PREFIX + "enabled";
 
   @Private
   public static final String AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX =
@@ -1722,8 +1735,83 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   }
 
   @Private
+  public static final String AUTO_CREATED_QUEUE_MANAGEMENT_POLICY =
+      AUTO_CREATE_CHILD_QUEUE_PREFIX + "management-policy";
+
+  @Private
+  public static final String DEFAULT_AUTO_CREATED_QUEUE_MANAGEMENT_POLICY =
+      "org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity"
+          + ".queuemanagement."
+          + "GuaranteedOrZeroCapacityOverTimePolicy";
+
+  @Private
+  private static final String QUEUE_MANAGEMENT_CONFIG_PREFIX =
+      "yarn.resourcemanager.monitor.capacity.queue-management.";
+
+  /**
+   * Time in milliseconds between invocations of this policy
+   */
+  @Private
+  public static final String QUEUE_MANAGEMENT_MONITORING_INTERVAL =
+      QUEUE_MANAGEMENT_CONFIG_PREFIX + "monitoring-interval";
+
+  @Private
+  public static final long DEFAULT_QUEUE_MANAGEMENT_MONITORING_INTERVAL =
+      1500L;
+
+  /**
+   * Queue Management computation policy for Auto Created queues
+   * @param queue The queue's path
+   * @return Configured policy class name
+   */
+  @Private
+  public String getAutoCreatedQueueManagementPolicy(String queue) {
+    String autoCreatedQueueManagementPolicy =
+        get(getQueuePrefix(queue) + AUTO_CREATED_QUEUE_MANAGEMENT_POLICY,
+            DEFAULT_AUTO_CREATED_QUEUE_MANAGEMENT_POLICY);
+    return autoCreatedQueueManagementPolicy;
+  }
+
+  /**
+   * Get The policy class configured to manage capacities for auto created leaf
+   * queues under the specified parent
+   *
+   * @param queueName The parent queue's name
+   * @return The policy class configured to manage capacities for auto created
+   * leaf queues under the specified parent queue
+   */
+  @Private
+  protected AutoCreatedQueueManagementPolicy
+  getAutoCreatedQueueManagementPolicyClass(
+      String queueName) {
+
+    String queueManagementPolicyClassName =
+        getAutoCreatedQueueManagementPolicy(queueName);
+    LOG.info("Using Auto Created Queue Management Policy: "
+        + queueManagementPolicyClassName + " for queue: " + queueName);
+    try {
+      Class<?> queueManagementPolicyClazz = getClassByName(
+          queueManagementPolicyClassName);
+      if (AutoCreatedQueueManagementPolicy.class.isAssignableFrom(
+          queueManagementPolicyClazz)) {
+        return (AutoCreatedQueueManagementPolicy) ReflectionUtils.newInstance(
+            queueManagementPolicyClazz, this);
+      } else{
+        throw new YarnRuntimeException(
+            "Class: " + queueManagementPolicyClassName + " not instance of "
+                + AutoCreatedQueueManagementPolicy.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException(
+          "Could not instantiate " + "AutoCreatedQueueManagementPolicy: "
+              + queueManagementPolicyClassName + " for queue: " + queueName,
+          e);
+    }
+  }
+
   @VisibleForTesting
-  public void setAutoCreatedLeafQueueTemplateCapacity(String queuePath,
+  @Private
+  public void setAutoCreatedLeafQueueConfigCapacity(String queuePath,
       float val) {
     String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
         queuePath);
@@ -1732,13 +1820,31 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
 
   @Private
   @VisibleForTesting
-  public void setAutoCreatedLeafQueueTemplateMaxCapacity(String queuePath,
+  public void setAutoCreatedLeafQueueConfigMaxCapacity(String queuePath,
       float val) {
     String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
         queuePath);
     setMaximumCapacity(leafQueueConfPrefix, val);
   }
 
+  @VisibleForTesting
+  @Private
+  public void setAutoCreatedLeafQueueConfigUserLimit(String queuePath,
+      int val) {
+    String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
+        queuePath);
+    setUserLimit(leafQueueConfPrefix, val);
+  }
+
+  @VisibleForTesting
+  @Private
+  public void setAutoCreatedLeafQueueConfigUserLimitFactor(String queuePath,
+      float val) {
+    String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
+        queuePath);
+    setUserLimitFactor(leafQueueConfPrefix, val);
+  }
+
   public static String getUnits(String resourceValue) {
     String units;
     for (int i = 0; i < resourceValue.length(); i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
index 7c918a5..ae74989 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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preempti
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 
 /**
@@ -94,4 +95,11 @@ public interface CapacitySchedulerContext {
    * @return if configuration is mutable
    */
   boolean isConfigurationMutable();
+
+  /**
+   * Get clock from scheduler
+   * @return Clock
+   */
+  Clock getClock();
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java
index eb50123..30ecd40 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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java
@@ -239,7 +239,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
             queueName + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
 
         List<CSQueue> childQueues = new ArrayList<>();
-        AutoCreatedLeafQueue resQueue = new AutoCreatedLeafQueue(csContext,
+        ReservationQueue resQueue = new ReservationQueue(csContext,
             defReservationId, (PlanQueue) queue);
         try {
           resQueue.setEntitlement(new QueueEntitlement(1.0f, 1.0f));
@@ -312,7 +312,8 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
       Map<String, CSQueue> newQueues) throws IOException {
     // check that all static queues are included in the newQueues list
     for (Map.Entry<String, CSQueue> e : queues.entrySet()) {
-      if (!(e.getValue() instanceof AutoCreatedLeafQueue)) {
+      if (!(AbstractAutoCreatedLeafQueue.class.isAssignableFrom(e.getValue()
+          .getClass()))) {
         String queueName = e.getKey();
         CSQueue oldQueue = e.getValue();
         CSQueue newQueue = newQueues.get(queueName);
@@ -394,7 +395,6 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
       String queueName = e.getKey();
       CSQueue existingQueue = e.getValue();
 
-      //TODO - Handle case when auto create is disabled on parent queues
       if (!newQueues.containsKey(queueName) && !(
           existingQueue instanceof AutoCreatedLeafQueue && conf
               .isAutoCreateChildQueueEnabled(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/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 41ec4ba..86fcbc9 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
@@ -138,7 +138,14 @@ public class LeafQueue extends AbstractCSQueue {
   @SuppressWarnings({ "unchecked", "rawtypes" })
   public LeafQueue(CapacitySchedulerContext cs,
       String queueName, CSQueue parent, CSQueue old) throws IOException {
-    super(cs, queueName, parent, old);
+      this(cs, cs.getConfiguration(), queueName, parent, old);
+  }
+
+  public LeafQueue(CapacitySchedulerContext cs,
+      CapacitySchedulerConfiguration configuration,
+      String queueName, CSQueue parent, CSQueue old) throws
+      IOException {
+    super(cs, configuration, queueName, parent, old);
     this.scheduler = cs;
 
     this.usersManager = new UsersManager(metrics, this, labelManager, scheduler,
@@ -149,17 +156,25 @@ public class LeafQueue extends AbstractCSQueue {
 
     if(LOG.isDebugEnabled()) {
       LOG.debug("LeafQueue:" + " name=" + queueName
-        + ", fullname=" + getQueuePath());
+          + ", fullname=" + getQueuePath());
     }
 
-    setupQueueConfigs(cs.getClusterResource());
+    setupQueueConfigs(cs.getClusterResource(), configuration);
+
   }
 
   protected void setupQueueConfigs(Resource clusterResource)
       throws IOException {
+    setupQueueConfigs(clusterResource, csContext.getConfiguration());
+  }
+
+  protected void setupQueueConfigs(Resource clusterResource,
+      CapacitySchedulerConfiguration conf) throws
+      IOException {
     try {
       writeLock.lock();
-      super.setupQueueConfigs(clusterResource);
+      CapacitySchedulerConfiguration schedConf = csContext.getConfiguration();
+      super.setupQueueConfigs(clusterResource, conf);
 
       this.lastClusterResource = clusterResource;
 
@@ -173,8 +188,6 @@ public class LeafQueue extends AbstractCSQueue {
       // absoluteMaxAvailCapacity during headroom/userlimit/allocation events)
       setQueueResourceLimitsInfo(clusterResource);
 
-      CapacitySchedulerConfiguration conf = csContext.getConfiguration();
-
       setOrderingPolicy(
           conf.<FiCaSchedulerApp>getAppOrderingPolicy(getQueuePath()));
 
@@ -183,11 +196,13 @@ public class LeafQueue extends AbstractCSQueue {
 
       maxApplications = conf.getMaximumApplicationsPerQueue(getQueuePath());
       if (maxApplications < 0) {
-        int maxGlobalPerQueueApps = conf.getGlobalMaximumApplicationsPerQueue();
+        int maxGlobalPerQueueApps = schedConf
+            .getGlobalMaximumApplicationsPerQueue();
         if (maxGlobalPerQueueApps > 0) {
           maxApplications = maxGlobalPerQueueApps;
         } else {
-          int maxSystemApps = conf.getMaximumSystemApplications();
+          int maxSystemApps = schedConf.
+              getMaximumSystemApplications();
           maxApplications =
               (int) (maxSystemApps * queueCapacities.getAbsoluteCapacity());
         }
@@ -218,9 +233,11 @@ public class LeafQueue extends AbstractCSQueue {
                         .join(getAccessibleNodeLabels().iterator(), ',')));
       }
 
-      nodeLocalityDelay = conf.getNodeLocalityDelay();
-      rackLocalityAdditionalDelay = conf.getRackLocalityAdditionalDelay();
-      rackLocalityFullReset = conf.getRackLocalityFullReset();
+      nodeLocalityDelay = schedConf.getNodeLocalityDelay();
+      rackLocalityAdditionalDelay = schedConf
+          .getRackLocalityAdditionalDelay();
+      rackLocalityFullReset = schedConf
+          .getRackLocalityFullReset();
 
       // re-init this since max allocation could have changed
       this.minimumAllocationFactor = Resources.ratio(resourceCalculator,
@@ -507,10 +524,11 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
-  @Override
-  public void reinitialize(
-      CSQueue newlyParsedQueue, Resource clusterResource) 
-  throws IOException {
+  protected void reinitialize(
+      CSQueue newlyParsedQueue, Resource clusterResource,
+      CapacitySchedulerConfiguration configuration) throws
+      IOException {
+
     try {
       writeLock.lock();
       // Sanity check
@@ -535,7 +553,7 @@ public class LeafQueue extends AbstractCSQueue {
             + newMax);
       }
 
-      setupQueueConfigs(clusterResource);
+      setupQueueConfigs(clusterResource, configuration);
 
       // queue metrics are updated, more resource may be available
       // activate the pending applications if possible
@@ -547,6 +565,14 @@ public class LeafQueue extends AbstractCSQueue {
   }
 
   @Override
+  public void reinitialize(
+      CSQueue newlyParsedQueue, Resource clusterResource)
+  throws IOException {
+    reinitialize(newlyParsedQueue, clusterResource,
+        csContext.getConfiguration());
+  }
+
+  @Override
   public void submitApplicationAttempt(FiCaSchedulerApp application,
       String userName) {
     // Careful! Locking order is important!
@@ -731,7 +757,7 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
-  private void activateApplications() {
+  protected void activateApplications() {
     try {
       writeLock.lock();
       // limit of allowed resource usage for application masters
@@ -1991,10 +2017,18 @@ public class LeafQueue extends AbstractCSQueue {
     queueCapacities.setCapacity(capacity);
   }
 
+  public void setCapacity(String nodeLabel, float capacity) {
+    queueCapacities.setCapacity(nodeLabel, capacity);
+  }
+
   public void setAbsoluteCapacity(float absoluteCapacity) {
     queueCapacities.setAbsoluteCapacity(absoluteCapacity);
   }
 
+  public void setAbsoluteCapacity(String nodeLabel, float absoluteCapacity) {
+    queueCapacities.setAbsoluteCapacity(nodeLabel, absoluteCapacity);
+  }
+
   public void setMaxApplicationsPerUser(int maxApplicationsPerUser) {
     this.maxApplicationsPerUser = maxApplicationsPerUser;
   }
@@ -2002,7 +2036,12 @@ public class LeafQueue extends AbstractCSQueue {
   public void setMaxApplications(int maxApplications) {
     this.maxApplications = maxApplications;
   }
-  
+
+  public void setMaxAMResourcePerQueuePercent(
+      float maxAMResourcePerQueuePercent) {
+    this.maxAMResourcePerQueuePercent = maxAMResourcePerQueuePercent;
+  }
+
   public OrderingPolicy<FiCaSchedulerApp>
       getOrderingPolicy() {
     return orderingPolicy;


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


[44/50] [abbrv] hadoop git commit: YARN-7443. Add native FPGA module support to do isolation with cgroups. (Zhankun Tang via wangda)

Posted by wa...@apache.org.
YARN-7443. Add native FPGA module support to do isolation with cgroups. (Zhankun Tang via wangda)

Change-Id: Ic4b7f9f3e032986b8f955139c9fe4d3a6c818a53


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

Branch: refs/heads/YARN-6592
Commit: 04b84da2456fb8c716e728b16db4851e2e87ec25
Parents: adca1a7
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Dec 8 15:18:22 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Dec 8 15:18:22 2017 -0800

----------------------------------------------------------------------
 .../hadoop-yarn/conf/container-executor.cfg     |   6 +
 .../src/CMakeLists.txt                          |   2 +
 .../main/native/container-executor/impl/main.c  |   6 +
 .../impl/modules/fpga/fpga-module.c             | 229 +++++++++++++++++++
 .../impl/modules/fpga/fpga-module.h             |  45 ++++
 .../test/modules/fpga/test-fpga-module.cc       | 216 +++++++++++++++++
 6 files changed, 504 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/04b84da2/hadoop-yarn-project/hadoop-yarn/conf/container-executor.cfg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/conf/container-executor.cfg b/hadoop-yarn-project/hadoop-yarn/conf/container-executor.cfg
index 7a84d76..4c698b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/conf/container-executor.cfg
+++ b/hadoop-yarn-project/hadoop-yarn/conf/container-executor.cfg
@@ -15,3 +15,9 @@ feature.tc.enabled=0
 #  docker.allowed.rw-mounts=## comma seperate volumes that can be mounted as read-write, add the yarn local and log dirs to this list to run Hadoop jobs
 #  docker.privileged-containers.enabled=0
 #  docker.allowed.volume-drivers=## comma seperated list of allowed volume-drivers
+
+# The configs below deal with settings for FPGA resource
+#[fpga]
+#  module.enabled=## Enable/Disable the FPGA resource handler module. set to "true" to enable, disabled by default
+#  fpga.major-device-number=## Major device number of FPGA, by default is 246. Strongly recommend setting this
+#  fpga.allowed-device-minor-numbers=## Comma separated allowed minor device numbers, empty means all FPGA devices managed by YARN.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04b84da2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
index 9d83bf3..79faeec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
@@ -132,6 +132,7 @@ add_library(container
     main/native/container-executor/impl/modules/cgroups/cgroups-operations.c
     main/native/container-executor/impl/modules/common/module-configs.c
     main/native/container-executor/impl/modules/gpu/gpu-module.c
+    main/native/container-executor/impl/modules/fpga/fpga-module.c
     main/native/container-executor/impl/utils/docker-util.c
 )
 
@@ -165,6 +166,7 @@ add_executable(cetest
         main/native/container-executor/test/utils/test-path-utils.cc
         main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc
         main/native/container-executor/test/modules/gpu/test-gpu-module.cc
+        main/native/container-executor/test/modules/fpga/test-fpga-module.cc
         main/native/container-executor/test/test_util.cc
         main/native/container-executor/test/utils/test_docker_util.cc)
 target_link_libraries(cetest gtest container)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04b84da2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
index 7b8f63f..ca84d40 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
@@ -22,6 +22,7 @@
 #include "util.h"
 #include "get_executable.h"
 #include "modules/gpu/gpu-module.h"
+#include "modules/fpga/fpga-module.h"
 #include "modules/cgroups/cgroups-operations.h"
 
 #include <errno.h>
@@ -244,6 +245,11 @@ static int validate_arguments(int argc, char **argv , int *operation) {
            &argv[1]);
   }
 
+  if (strcmp("--module-fpga", argv[1]) == 0) {
+    return handle_fpga_request(&update_cgroups_parameters, "fpga", argc - 1,
+           &argv[1]);
+  }
+
   if (strcmp("--checksetup", argv[1]) == 0) {
     *operation = CHECK_SETUP;
     return 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04b84da2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/fpga/fpga-module.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/fpga/fpga-module.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/fpga/fpga-module.c
new file mode 100644
index 0000000..c1a2f83
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/fpga/fpga-module.c
@@ -0,0 +1,229 @@
+/**
+ * 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.
+ */
+
+#include "configuration.h"
+#include "container-executor.h"
+#include "utils/string-utils.h"
+#include "modules/fpga/fpga-module.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "modules/common/module-configs.h"
+#include "modules/common/constants.h"
+#include "util.h"
+
+#include <stdio.h>
+#include <string.h>
+#include <stdlib.h>
+#include <getopt.h>
+#include <unistd.h>
+
+#define EXCLUDED_FPGAS_OPTION "excluded_fpgas"
+#define CONTAINER_ID_OPTION "container_id"
+#define DEFAULT_INTEL_MAJOR_NUMBER 246
+#define MAX_CONTAINER_ID_LEN 128
+
+static const struct section* cfg_section;
+
+static int internal_handle_fpga_request(
+    update_cgroups_parameters_function update_cgroups_parameters_func_p,
+    size_t n_minor_devices_to_block, int minor_devices[],
+    const char* container_id) {
+  char* allowed_minor_numbers_str = NULL;
+  int* allowed_minor_numbers = NULL;
+  size_t n_allowed_minor_numbers = 0;
+  int return_code = 0;
+
+  if (n_minor_devices_to_block == 0) {
+    // no device to block, just return;
+    return 0;
+  }
+
+  // Get major device number from cfg, if not set, major number of (Intel)
+  // will be the default value.
+  int major_device_number;
+  char* major_number_str = get_section_value(FPGA_MAJOR_NUMBER_CONFIG_KEY,
+     cfg_section);
+  if (!major_number_str || 0 == major_number_str[0]) {
+    // Default major number of Intel devices
+    major_device_number = DEFAULT_INTEL_MAJOR_NUMBER;
+  } else {
+    major_device_number = strtol(major_number_str, NULL, 0);
+  }
+
+  // Get allowed minor device numbers from cfg, if not set, means all minor
+  // devices can be used by YARN
+  allowed_minor_numbers_str = get_section_value(
+      FPGA_ALLOWED_DEVICES_MINOR_NUMBERS,
+      cfg_section);
+  if (!allowed_minor_numbers_str || 0 == allowed_minor_numbers_str[0]) {
+    allowed_minor_numbers = NULL;
+  } else {
+    int rc = get_numbers_split_by_comma(allowed_minor_numbers_str,
+                                        &allowed_minor_numbers,
+                                        &n_allowed_minor_numbers);
+    if (0 != rc) {
+      fprintf(ERRORFILE,
+          "Failed to get allowed minor device numbers from cfg, value=%s\n",
+          allowed_minor_numbers_str);
+      return_code = -1;
+      goto cleanup;
+    }
+
+    // Make sure we're trying to black devices allowed in config
+    for (int i = 0; i < n_minor_devices_to_block; i++) {
+      int found = 0;
+      for (int j = 0; j < n_allowed_minor_numbers; j++) {
+        if (minor_devices[i] == allowed_minor_numbers[j]) {
+          found = 1;
+          break;
+        }
+      }
+
+      if (!found) {
+        fprintf(ERRORFILE,
+          "Trying to blacklist device with minor-number=%d which is not on allowed list\n",
+          minor_devices[i]);
+        return_code = -1;
+        goto cleanup;
+      }
+    }
+  }
+
+  // Use cgroup helpers to blacklist devices
+  for (int i = 0; i < n_minor_devices_to_block; i++) {
+    char param_value[128];
+    memset(param_value, 0, sizeof(param_value));
+    snprintf(param_value, sizeof(param_value), "c %d:%d rwm",
+             major_device_number, minor_devices[i]);
+
+    int rc = update_cgroups_parameters_func_p("devices", "deny",
+      container_id, param_value);
+
+    if (0 != rc) {
+      fprintf(ERRORFILE, "CGroups: Failed to update cgroups\n");
+      return_code = -1;
+      goto cleanup;
+    }
+  }
+
+cleanup:
+  if (major_number_str) {
+    free(major_number_str);
+  }
+  if (allowed_minor_numbers) {
+    free(allowed_minor_numbers);
+  }
+  if (allowed_minor_numbers_str) {
+    free(allowed_minor_numbers_str);
+  }
+
+  return return_code;
+}
+
+void reload_fpga_configuration() {
+  cfg_section = get_configuration_section(FPGA_MODULE_SECTION_NAME, get_cfg());
+}
+
+/*
+ * Format of FPGA request commandline:
+ *
+ * c-e fpga --excluded_fpgas 0,1,3 --container_id container_x_y
+ */
+int handle_fpga_request(update_cgroups_parameters_function func,
+    const char* module_name, int module_argc, char** module_argv) {
+  if (!cfg_section) {
+    reload_fpga_configuration();
+  }
+
+  if (!module_enabled(cfg_section, FPGA_MODULE_SECTION_NAME)) {
+    fprintf(ERRORFILE,
+      "Please make sure fpga module is enabled before using it.\n");
+    return -1;
+  }
+
+  static struct option long_options[] = {
+    {EXCLUDED_FPGAS_OPTION, required_argument, 0, 'e' },
+    {CONTAINER_ID_OPTION, required_argument, 0, 'c' },
+    {0, 0, 0, 0}
+  };
+
+  int rc = 0;
+  int c = 0;
+  int option_index = 0;
+
+  int* minor_devices = NULL;
+  char container_id[MAX_CONTAINER_ID_LEN];
+  memset(container_id, 0, sizeof(container_id));
+  size_t n_minor_devices_to_block = 0;
+  int failed = 0;
+
+  optind = 1;
+  while((c = getopt_long(module_argc, module_argv, "e:c:",
+                         long_options, &option_index)) != -1) {
+    switch(c) {
+      case 'e':
+        rc = get_numbers_split_by_comma(optarg, &minor_devices,
+          &n_minor_devices_to_block);
+        if (0 != rc) {
+          fprintf(ERRORFILE,
+            "Failed to get minor devices number from command line, value=%s\n",
+            optarg);
+          failed = 1;
+          goto cleanup;
+        }
+        break;
+      case 'c':
+        if (!validate_container_id(optarg)) {
+          fprintf(ERRORFILE,
+            "Specified container_id=%s is invalid\n", optarg);
+          failed = 1;
+          goto cleanup;
+        }
+        strncpy(container_id, optarg, MAX_CONTAINER_ID_LEN);
+        break;
+      default:
+        fprintf(ERRORFILE,
+          "Unknown option in fpga command character %d %c, optionindex = %d\n",
+          c, c, optind);
+        failed = 1;
+        goto cleanup;
+    }
+  }
+
+  if (0 == container_id[0]) {
+    fprintf(ERRORFILE,
+      "[%s] --container_id must be specified.\n", __func__);
+    failed = 1;
+    goto cleanup;
+  }
+
+  if (!minor_devices) {
+     // Minor devices is null, skip following call.
+     fprintf(ERRORFILE, "is not specified, skip cgroups call.\n");
+     goto cleanup;
+  }
+
+  failed = internal_handle_fpga_request(func, n_minor_devices_to_block,
+         minor_devices,
+         container_id);
+
+cleanup:
+  if (minor_devices) {
+    free(minor_devices);
+  }
+  return failed;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04b84da2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/fpga/fpga-module.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/fpga/fpga-module.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/fpga/fpga-module.h
new file mode 100644
index 0000000..12d6ab3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/fpga/fpga-module.h
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _MODULES_FPGA_FPGA_MUDULE_H_
+#define _MODULES_FPGA_FPGA_MUDULE_H_
+
+#define FPGA_MAJOR_NUMBER_CONFIG_KEY "fpga.major-device-number"
+#define FPGA_ALLOWED_DEVICES_MINOR_NUMBERS "fpga.allowed-device-minor-numbers"
+#define FPGA_MODULE_SECTION_NAME "fpga"
+
+// For unit test stubbing
+typedef int (*update_cgroups_parameters_function)(const char*, const char*,
+   const char*, const char*);
+
+/**
+ * Handle fpga requests
+ */
+int handle_fpga_request(update_cgroups_parameters_function func,
+   const char* module_name, int module_argc, char** module_argv);
+
+/**
+ * Reload config from filesystem, visible for testing.
+ */
+void reload_fpga_configuration();
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04b84da2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/fpga/test-fpga-module.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/fpga/test-fpga-module.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/fpga/test-fpga-module.cc
new file mode 100644
index 0000000..1e5c5ea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/fpga/test-fpga-module.cc
@@ -0,0 +1,216 @@
+/**
+ * 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.
+ */
+
+#include <vector>
+
+#include <errno.h>
+#include <fcntl.h>
+#include <inttypes.h>
+#include <signal.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <sys/wait.h>
+#include <unistd.h>
+
+#include <gtest/gtest.h>
+#include <sstream>
+
+extern "C" {
+#include "configuration.h"
+#include "container-executor.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "modules/fpga/fpga-module.h"
+#include "test/test-container-executor-common.h"
+#include "util.h"
+}
+
+namespace ContainerExecutor {
+
+class TestFpgaModule : public ::testing::Test {
+protected:
+  virtual void SetUp() {
+    if (mkdirs(TEST_ROOT, 0755) != 0) {
+      fprintf(ERRORFILE, "Failed to mkdir TEST_ROOT: %s\n", TEST_ROOT);
+      exit(1);
+    }
+    LOGFILE = stdout;
+    ERRORFILE = stderr;
+  }
+
+  virtual void TearDown() {
+
+  }
+};
+
+static std::vector<const char*> cgroups_parameters_invoked;
+
+static int mock_update_cgroups_parameters(
+   const char* controller_name,
+   const char* param_name,
+   const char* group_id,
+   const char* value) {
+  char* buf = (char*) malloc(128);
+  strcpy(buf, controller_name);
+  cgroups_parameters_invoked.push_back(buf);
+
+  buf = (char*) malloc(128);
+  strcpy(buf, param_name);
+  cgroups_parameters_invoked.push_back(buf);
+
+  buf = (char*) malloc(128);
+  strcpy(buf, group_id);
+  cgroups_parameters_invoked.push_back(buf);
+
+  buf = (char*) malloc(128);
+  strcpy(buf, value);
+  cgroups_parameters_invoked.push_back(buf);
+  return 0;
+}
+
+static void verify_param_updated_to_cgroups(
+    int argc, const char** argv) {
+  ASSERT_EQ(argc, cgroups_parameters_invoked.size());
+
+  int offset = 0;
+  while (offset < argc) {
+    ASSERT_STREQ(argv[offset], cgroups_parameters_invoked[offset]);
+    offset++;
+  }
+}
+
+static void write_and_load_fpga_module_to_cfg(const char* cfg_filepath, int enabled) {
+  FILE *file = fopen(cfg_filepath, "w");
+  if (file == NULL) {
+    printf("FAIL: Could not open configuration file: %s\n", cfg_filepath);
+    exit(1);
+  }
+  fprintf(file, "[fpga]\n");
+  if (enabled) {
+    fprintf(file, "module.enabled=true\n");
+  } else {
+    fprintf(file, "module.enabled=false\n");
+  }
+  fclose(file);
+
+  // Read config file
+  read_executor_config(cfg_filepath);
+  reload_fpga_configuration();
+}
+
+static void test_fpga_module_enabled_disabled(int enabled) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_cgroups_module_enabled_disabled.cfg";
+  write_and_load_fpga_module_to_cfg(filename, enabled);
+
+  char* argv[] = { (char*) "--module-fpga", (char*) "--excluded_fpgas", (char*) "0,1",
+                   (char*) "--container_id",
+                   (char*) "container_1498064906505_0001_01_000001" };
+
+  int rc = handle_fpga_request(&mock_update_cgroups_parameters,
+              "fpga", 5, argv);
+
+  int EXPECTED_RC;
+  if (enabled) {
+    EXPECTED_RC = 0;
+  } else {
+    EXPECTED_RC = -1;
+  }
+  ASSERT_EQ(EXPECTED_RC, rc);
+}
+
+TEST_F(TestFpgaModule, test_verify_fpga_module_calls_cgroup_parameter) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_verify_fpga_module_calls_cgroup_parameter.cfg";
+  write_and_load_fpga_module_to_cfg(filename, 1);
+
+  char* container_id = (char*) "container_1498064906505_0001_01_000001";
+  char* argv[] = { (char*) "--module-fpga", (char*) "--excluded_fpgas", (char*) "0,1",
+                   (char*) "--container_id",
+                   container_id };
+
+  /* Test case 1: block 2 devices */
+  cgroups_parameters_invoked.clear();
+  int rc = handle_fpga_request(&mock_update_cgroups_parameters,
+     "fpga", 5, argv);
+  ASSERT_EQ(0, rc) << "Should success.\n";
+
+  // Verify cgroups parameters
+  const char* expected_cgroups_argv[] = { "devices", "deny", container_id, "c 246:0 rwm",
+    "devices", "deny", container_id, "c 246:1 rwm"};
+  verify_param_updated_to_cgroups(8, expected_cgroups_argv);
+
+  /* Test case 2: block 0 devices */
+  cgroups_parameters_invoked.clear();
+  char* argv_1[] = { (char*) "--module-fpga", (char*) "--container_id", container_id };
+  rc = handle_fpga_request(&mock_update_cgroups_parameters,
+     "fpga", 3, argv_1);
+  ASSERT_EQ(0, rc) << "Should success.\n";
+
+  // Verify cgroups parameters
+  verify_param_updated_to_cgroups(0, NULL);
+
+  /* Test case 3: block 2 non-sequential devices */
+  cgroups_parameters_invoked.clear();
+  char* argv_2[] = { (char*) "--module-fpga", (char*) "--excluded_fpgas", (char*) "1,3",
+                   (char*) "--container_id", container_id };
+  rc = handle_fpga_request(&mock_update_cgroups_parameters,
+     "fpga", 5, argv_2);
+  ASSERT_EQ(0, rc) << "Should success.\n";
+
+  // Verify cgroups parameters
+  const char* expected_cgroups_argv_2[] = { "devices", "deny", container_id, "c 246:1 rwm",
+    "devices", "deny", container_id, "c 246:3 rwm"};
+  verify_param_updated_to_cgroups(8, expected_cgroups_argv_2);
+}
+
+TEST_F(TestFpgaModule, test_illegal_cli_parameters) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_illegal_cli_parameters.cfg";
+  write_and_load_fpga_module_to_cfg(filename, 1);
+
+  // Illegal container id - 1
+  char* argv[] = { (char*) "--module-fpga", (char*) "--excluded_fpgas", (char*) "0,1",
+                   (char*) "--container_id", (char*) "xxxx" };
+  int rc = handle_fpga_request(&mock_update_cgroups_parameters,
+     "fpga", 5, argv);
+  ASSERT_NE(0, rc) << "Should fail.\n";
+
+  // Illegal container id - 2
+  char* argv_1[] = { (char*) "--module-fpga", (char*) "--excluded_fpgas", (char*) "0,1",
+                   (char*) "--container_id", (char*) "container_1" };
+  rc = handle_fpga_request(&mock_update_cgroups_parameters,
+     "fpga", 5, argv_1);
+  ASSERT_NE(0, rc) << "Should fail.\n";
+
+  // Illegal container id - 3
+  char* argv_2[] = { (char*) "--module-fpga", (char*) "--excluded_fpgas", (char*) "0,1" };
+  rc = handle_fpga_request(&mock_update_cgroups_parameters,
+     "fpga", 3, argv_2);
+  ASSERT_NE(0, rc) << "Should fail.\n";
+}
+
+TEST_F(TestFpgaModule, test_fpga_module_disabled) {
+  test_fpga_module_enabled_disabled(0);
+}
+
+TEST_F(TestFpgaModule, test_fpga_module_enabled) {
+  test_fpga_module_enabled_disabled(1);
+}
+} // namespace ContainerExecutor
\ No newline at end of file


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


[35/50] [abbrv] hadoop git commit: HDFS-12825. Fsck report shows config key name for min replication issues (Contributed by Gabor Bota).

Posted by wa...@apache.org.
HDFS-12825. Fsck report shows config key name for min replication issues (Contributed by Gabor Bota).


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

Branch: refs/heads/YARN-6592
Commit: ef7d334d364378070880e647eaf8bac2f12561ee
Parents: f196383
Author: Manoj Govindassamy <ma...@apache.org>
Authored: Fri Dec 8 11:46:30 2017 -0800
Committer: Manoj Govindassamy <ma...@apache.org>
Committed: Fri Dec 8 11:46:30 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java  | 2 +-
 .../test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef7d334d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index b6d6971..a82c9fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -1236,7 +1236,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
                 ((float) (numUnderMinReplicatedBlocks * 100) / (float) totalBlocks))
                 .append(" %)");
           }
-          res.append("\n  ").append(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY + ":\t")
+          res.append("\n  ").append("MINIMAL BLOCK REPLICATION:\t")
              .append(minReplication);
         }
         if(corruptFiles>0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef7d334d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 0c04738..f80fd70 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -902,7 +902,7 @@ public class TestFsck {
     System.out.println(outStr);
     assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
     assertTrue(outStr.contains("UNDER MIN REPL'D BLOCKS:\t1 (100.0 %)"));
-    assertTrue(outStr.contains("dfs.namenode.replication.min:\t2"));
+    assertTrue(outStr.contains("MINIMAL BLOCK REPLICATION:\t2"));
   }
 
   @Test(timeout = 90000)


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


[06/50] [abbrv] hadoop git commit: YARN-7611. Node manager web UI should display container type in containers page. Contributed by Weiwei Yang.

Posted by wa...@apache.org.
YARN-7611. Node manager web UI should display container type in containers page. Contributed by Weiwei Yang.


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

Branch: refs/heads/YARN-6592
Commit: 05c347fe51c01494ed8110f8f116a01c90205f13
Parents: 73b8697
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Dec 6 12:21:52 2017 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Wed Dec 6 12:21:52 2017 +0800

----------------------------------------------------------------------
 .../yarn/server/nodemanager/webapp/AllContainersPage.java      | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05c347fe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
index 3fc6f3c..afc9692 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
@@ -51,9 +51,9 @@ public class AllContainersPage extends NMView {
 
   private String containersTableInit() {
     return tableInit().
-        // containerid, containerid, log-url
+        // containerid, executiontype, containerid, log-url
         append(", aoColumns:[").append(getContainersIdColumnDefs())
-        .append(", null, {bSearchable:false}]} ").toString();
+        .append(", null, null, {bSearchable:false}]} ").toString();
   }
 
   private String getContainersIdColumnDefs() {
@@ -83,6 +83,7 @@ public class AllContainersPage extends NMView {
           .thead()
             .tr()
               .td().__("ContainerId").__()
+              .td().__("ExecutionType").__()
               .td().__("ContainerState").__()
               .td().__("logs").__()
             .__()
@@ -94,6 +95,7 @@ public class AllContainersPage extends NMView {
           .tr()
             .td().a(url("container", info.getId()), info.getId())
             .__()
+            .td().__(info.getExecutionType()).__()
             .td().__(info.getState()).__()
             .td()
                 .a(url(info.getShortLogLink()), "logs").__()


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


[15/50] [abbrv] hadoop git commit: HADOOP-15080. Aliyun OSS: update oss sdk from 2.8.1 to 2.8.3 to remove its dependency on Cat-x json-lib.

Posted by wa...@apache.org.
HADOOP-15080.  Aliyun OSS: update oss sdk from 2.8.1 to 2.8.3 to remove its dependency on Cat-x json-lib.


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

Branch: refs/heads/YARN-6592
Commit: 67b2661e3d73a68ba7ca73b112bf6baea128631e
Parents: e411dd6
Author: Sammi Chen <sa...@intel.com>
Authored: Thu Dec 7 22:46:11 2017 +0800
Committer: Sammi Chen <sa...@intel.com>
Committed: Thu Dec 7 22:46:11 2017 +0800

----------------------------------------------------------------------
 hadoop-project/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/67b2661e/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 04b93c4..0866f3e 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1165,7 +1165,7 @@
       <dependency>
         <groupId>com.aliyun.oss</groupId>
         <artifactId>aliyun-sdk-oss</artifactId>
-        <version>2.8.1</version>
+        <version>2.8.3</version>
         <exclusions>
           <exclusion>
             <groupId>org.apache.httpcomponents</groupId>


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


[03/50] [abbrv] hadoop git commit: YARN-7381. Enable the configuration: yarn.nodemanager.log-container-debug-info.enabled by default in yarn-default.xml. (Xuan Gong via wangda)

Posted by wa...@apache.org.
YARN-7381. Enable the configuration: yarn.nodemanager.log-container-debug-info.enabled by default in yarn-default.xml. (Xuan Gong via wangda)

Change-Id: I1ed58dafad5cc276eea5c0b0813cf04f57d73a87


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

Branch: refs/heads/YARN-6592
Commit: 0311cf05358cd75388f48f048c44fba52ec90f00
Parents: 6555af8
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Dec 5 13:09:49 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Dec 5 13:09:49 2017 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java   | 2 +-
 .../hadoop-yarn-common/src/main/resources/yarn-default.xml         | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0311cf05/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 831abf5..e57f988 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
@@ -142,7 +142,7 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_LOG_CONTAINER_DEBUG_INFO =
       YarnConfiguration.NM_PREFIX + "log-container-debug-info.enabled";
 
-  public static final boolean DEFAULT_NM_LOG_CONTAINER_DEBUG_INFO = false;
+  public static final boolean DEFAULT_NM_LOG_CONTAINER_DEBUG_INFO = true;
 
   ////////////////////////////////
   // IPC Configs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0311cf05/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 2550c42..192f62e 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
@@ -1282,7 +1282,7 @@
     slowness in launching containers.
     </description>
     <name>yarn.nodemanager.log-container-debug-info.enabled</name>
-    <value>false</value>
+    <value>true</value>
   </property>
 
   <property>


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


[48/50] [abbrv] hadoop git commit: YARN-6593. [API] Introduce Placement Constraint object. (Konstantinos Karanasos via wangda)

Posted by wa...@apache.org.
YARN-6593. [API] Introduce Placement Constraint object. (Konstantinos Karanasos via wangda)

Change-Id: Id00edb7185fdf01cce6e40f920cac3585f8cbe9c


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

Branch: refs/heads/YARN-6592
Commit: ccd57dc9653e8771628489e1d7a91e79f7853e22
Parents: 670e8d4
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Aug 3 14:03:55 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Sun Dec 10 10:27:01 2017 -0800

----------------------------------------------------------------------
 .../yarn/api/resource/PlacementConstraint.java  | 567 +++++++++++++++++++
 .../yarn/api/resource/PlacementConstraints.java | 286 ++++++++++
 .../hadoop/yarn/api/resource/package-info.java  |  23 +
 .../src/main/proto/yarn_protos.proto            |  55 ++
 .../api/resource/TestPlacementConstraints.java  | 106 ++++
 .../PlacementConstraintFromProtoConverter.java  | 116 ++++
 .../pb/PlacementConstraintToProtoConverter.java | 174 ++++++
 .../apache/hadoop/yarn/api/pb/package-info.java |  23 +
 .../yarn/api/records/impl/pb/ProtoUtils.java    |  27 +
 .../PlacementConstraintTransformations.java     | 209 +++++++
 .../hadoop/yarn/api/resource/package-info.java  |  23 +
 .../TestPlacementConstraintPBConversion.java    | 195 +++++++
 .../TestPlacementConstraintTransformations.java | 183 ++++++
 13 files changed, 1987 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
new file mode 100644
index 0000000..f0e3982
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
@@ -0,0 +1,567 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.resource;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * {@code PlacementConstraint} represents a placement constraint for a resource
+ * allocation.
+ */
+@Public
+@Unstable
+public class PlacementConstraint {
+
+  /**
+   * The constraint expression tree.
+   */
+  private AbstractConstraint constraintExpr;
+
+  public PlacementConstraint(AbstractConstraint constraintExpr) {
+    this.constraintExpr = constraintExpr;
+  }
+
+  /**
+   * Get the constraint expression of the placement constraint.
+   *
+   * @return the constraint expression
+   */
+  public AbstractConstraint getConstraintExpr() {
+    return constraintExpr;
+  }
+
+  /**
+   * Interface used to enable the elements of the constraint tree to be visited.
+   */
+  @Private
+  public interface Visitable {
+    /**
+     * Visitor pattern.
+     *
+     * @param visitor visitor to be used
+     * @param <T> defines the type that the visitor will use and the return type
+     *          of the accept.
+     * @return the result of visiting a given object.
+     */
+    <T> T accept(Visitor<T> visitor);
+
+  }
+
+  /**
+   * Visitor API for a constraint tree.
+   *
+   * @param <T> determines the return type of the visit methods.
+   */
+  @Private
+  public interface Visitor<T> {
+    T visit(SingleConstraint constraint);
+
+    T visit(TargetExpression target);
+
+    T visit(TargetConstraint constraint);
+
+    T visit(CardinalityConstraint constraint);
+
+    T visit(And constraint);
+
+    T visit(Or constraint);
+
+    T visit(DelayedOr constraint);
+
+    T visit(TimedPlacementConstraint constraint);
+  }
+
+  /**
+   * Abstract class that acts as the superclass of all placement constraint
+   * classes.
+   */
+  public abstract static class AbstractConstraint implements Visitable {
+    public PlacementConstraint build() {
+      return new PlacementConstraint(this);
+    }
+  }
+
+  static final String NODE_SCOPE = "node";
+  static final String RACK_SCOPE = "rack";
+
+  /**
+   * Consider a set of nodes N that belongs to the scope specified in the
+   * constraint. If the target expressions are satisfied at least minCardinality
+   * times and at most max-cardinality times in the node set N, then the
+   * constraint is satisfied.
+   *
+   * For example, a constraint of the form {@code {RACK, 2, 10,
+   * allocationTag("zk")}}, requires an allocation to be placed within a rack
+   * that has at least 2 and at most 10 other allocations with tag "zk".
+   */
+  public static class SingleConstraint extends AbstractConstraint {
+    private String scope;
+    private int minCardinality;
+    private int maxCardinality;
+    private Set<TargetExpression> targetExpressions;
+
+    public SingleConstraint(String scope, int minCardinality,
+        int maxCardinality, Set<TargetExpression> targetExpressions) {
+      this.scope = scope;
+      this.minCardinality = minCardinality;
+      this.maxCardinality = maxCardinality;
+      this.targetExpressions = targetExpressions;
+    }
+
+    public SingleConstraint(String scope, int minC, int maxC,
+        TargetExpression... targetExpressions) {
+      this(scope, minC, maxC, new HashSet<>(Arrays.asList(targetExpressions)));
+    }
+
+    /**
+     * Get the scope of the constraint.
+     *
+     * @return the scope of the constraint
+     */
+    public String getScope() {
+      return scope;
+    }
+
+    /**
+     * Get the minimum cardinality of the constraint.
+     *
+     * @return the minimum cardinality of the constraint
+     */
+    public int getMinCardinality() {
+      return minCardinality;
+    }
+
+    /**
+     * Get the maximum cardinality of the constraint.
+     *
+     * @return the maximum cardinality of the constraint
+     */
+    public int getMaxCardinality() {
+      return maxCardinality;
+    }
+
+    /**
+     * Get the target expressions of the constraint.
+     *
+     * @return the set of target expressions
+     */
+    public Set<TargetExpression> getTargetExpressions() {
+      return targetExpressions;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class representing the target expressions that are used in placement
+   * constraints. They might refer to expressions on node attributes, allocation
+   * tags, or be self-targets (referring to the allocation to which the
+   * constraint is attached).
+   */
+  public static class TargetExpression implements Visitable {
+    /**
+     * Enum specifying the type of the target expression.
+     */
+    public enum TargetType {
+      NODE_ATTRIBUTE, ALLOCATION_TAG, SELF
+    }
+
+    private TargetType targetType;
+    private String targetKey;
+    private Set<String> targetValues;
+
+    public TargetExpression(TargetType targetType, String targetKey,
+        Set<String> targetValues) {
+      this.targetType = targetType;
+      this.targetKey = targetKey;
+      this.targetValues = targetValues;
+    }
+
+    public TargetExpression(TargetType targetType) {
+      this(targetType, null, new HashSet<>());
+    }
+
+    public TargetExpression(TargetType targetType, String targetKey,
+        String... targetValues) {
+      this(targetType, targetKey, new HashSet<>(Arrays.asList(targetValues)));
+    }
+
+    /**
+     * Get the type of the target expression.
+     *
+     * @return the type of the target expression
+     */
+    public TargetType getTargetType() {
+      return targetType;
+    }
+
+    /**
+     * Get the key of the target expression.
+     *
+     * @return the key of the target expression
+     */
+    public String getTargetKey() {
+      return targetKey;
+    }
+
+    /**
+     * Get the set of values of the target expression.
+     *
+     * @return the set of values of the target expression
+     */
+    public Set<String> getTargetValues() {
+      return targetValues;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = targetType != null ? targetType.hashCode() : 0;
+      result = 31 * result + (targetKey != null ? targetKey.hashCode() : 0);
+      result =
+          31 * result + (targetValues != null ? targetValues.hashCode() : 0);
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null) {
+        return false;
+      }
+      if (!(o instanceof TargetExpression)) {
+        return false;
+      }
+
+      TargetExpression that = (TargetExpression) o;
+      if (targetType != that.targetType) {
+        return false;
+      }
+      if (targetKey != null ? !targetKey.equals(that.targetKey)
+          : that.targetKey != null) {
+        return false;
+      }
+      return targetValues != null ? targetValues.equals(that.targetValues)
+          : that.targetValues == null;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a target constraint. Such a constraint requires an
+   * allocation to be placed within a scope that satisfies some specified
+   * expressions on node attributes and allocation tags.
+   *
+   * It is a specialized version of the {@link SingleConstraint}, where the
+   * minimum and the maximum cardinalities take specific values based on the
+   * {@link TargetOperator} used.
+   */
+  public static class TargetConstraint extends AbstractConstraint {
+    enum TargetOperator {
+      IN, NOT_IN
+    }
+
+    private TargetOperator op;
+    private String scope;
+    private Set<TargetExpression> targetExpressions;
+
+    public TargetConstraint(TargetOperator op, String scope,
+        Set<TargetExpression> targetExpressions) {
+      this.op = op;
+      this.scope = scope;
+      this.targetExpressions = targetExpressions;
+    }
+
+    /**
+     * Get the target operator of the constraint.
+     *
+     * @return the target operator
+     */
+    public TargetOperator getOp() {
+      return op;
+    }
+
+    /**
+     * Get the scope of the constraint.
+     *
+     * @return the scope of the constraint
+     */
+    public String getScope() {
+      return scope;
+    }
+
+    /**
+     * Get the set of target expressions.
+     *
+     * @return the set of target expressions
+     */
+    public Set<TargetExpression> getTargetExpressions() {
+      return targetExpressions;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a cardinality constraint. Such a constraint the
+   * number of allocations within a given scope to some minimum and maximum
+   * values.
+   *
+   * It is a specialized version of the {@link SingleConstraint}, where the
+   * target is self (i.e., the allocation to which the constraint is attached).
+   */
+  public static class CardinalityConstraint extends AbstractConstraint {
+    private String scope;
+    private int minCardinality;
+    private int maxCardinality;
+
+    public CardinalityConstraint(String scope, int minCardinality,
+        int maxCardinality) {
+      this.scope = scope;
+      this.minCardinality = minCardinality;
+      this.maxCardinality = maxCardinality;
+    }
+
+    /**
+     * Get the scope of the constraint.
+     *
+     * @return the scope of the constraint
+     */
+    public String getScope() {
+      return scope;
+    }
+
+    /**
+     * Get the minimum cardinality of the constraint.
+     *
+     * @return the minimum cardinality of the constraint
+     */
+    public int getMinCardinality() {
+      return minCardinality;
+    }
+
+    /**
+     * Get the maximum cardinality of the constraint.
+     *
+     * @return the maximum cardinality of the constraint
+     */
+    public int getMaxCardinality() {
+      return maxCardinality;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents composite constraints, which comprise other
+   * constraints, forming a constraint tree.
+   *
+   * @param <R> the type of constraints that are used as children of the
+   *          specific composite constraint
+   */
+  public abstract static class CompositeConstraint<R extends Visitable>
+      extends AbstractConstraint {
+
+    /**
+     * Get the children of this composite constraint.
+     *
+     * @return the children of the composite constraint
+     */
+    public abstract List<R> getChildren();
+  }
+
+  /**
+   * Class that represents a composite constraint that is a conjunction of other
+   * constraints.
+   */
+  public static class And extends CompositeConstraint<AbstractConstraint> {
+    private List<AbstractConstraint> children;
+
+    public And(List<AbstractConstraint> children) {
+      this.children = children;
+    }
+
+    public And(AbstractConstraint... children) {
+      this(Arrays.asList(children));
+    }
+
+    @Override
+    public List<AbstractConstraint> getChildren() {
+      return children;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a composite constraint that is a disjunction of other
+   * constraints.
+   */
+  public static class Or extends CompositeConstraint<AbstractConstraint> {
+    private List<AbstractConstraint> children;
+
+    public Or(List<AbstractConstraint> children) {
+      this.children = children;
+    }
+
+    public Or(AbstractConstraint... children) {
+      this(Arrays.asList(children));
+    }
+
+    @Override
+    public List<AbstractConstraint> getChildren() {
+      return children;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a composite constraint that comprises a list of timed
+   * placement constraints (see {@link TimedPlacementConstraint}). The scheduler
+   * should try to satisfy first the first timed child constraint within the
+   * specified time window. If this is not possible, it should attempt to
+   * satisfy the second, and so on.
+   */
+  public static class DelayedOr
+      extends CompositeConstraint<TimedPlacementConstraint> {
+    private List<TimedPlacementConstraint> children = new ArrayList<>();
+
+    public DelayedOr(List<TimedPlacementConstraint> children) {
+      this.children = children;
+    }
+
+    public DelayedOr(TimedPlacementConstraint... children) {
+      this(Arrays.asList(children));
+    }
+
+    @Override
+    public List<TimedPlacementConstraint> getChildren() {
+      return children;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Represents a timed placement constraint that has to be satisfied within a
+   * time window.
+   */
+  public static class TimedPlacementConstraint implements Visitable {
+    /**
+     * The unit of scheduling delay.
+     */
+    public enum DelayUnit {
+      MILLISECONDS, OPPORTUNITIES
+    }
+
+    private AbstractConstraint constraint;
+    private long schedulingDelay;
+    private DelayUnit delayUnit;
+
+    public TimedPlacementConstraint(AbstractConstraint constraint,
+        long schedulingDelay, DelayUnit delayUnit) {
+      this.constraint = constraint;
+      this.schedulingDelay = schedulingDelay;
+      this.delayUnit = delayUnit;
+    }
+
+    public TimedPlacementConstraint(AbstractConstraint constraint,
+        long schedulingDelay) {
+      this(constraint, schedulingDelay, DelayUnit.MILLISECONDS);
+    }
+
+    public TimedPlacementConstraint(AbstractConstraint constraint) {
+      this(constraint, Long.MAX_VALUE, DelayUnit.MILLISECONDS);
+    }
+
+    /**
+     * Get the constraint that has to be satisfied within the time window.
+     *
+     * @return the constraint to be satisfied
+     */
+    public AbstractConstraint getConstraint() {
+      return constraint;
+    }
+
+    /**
+     * Sets the constraint that has to be satisfied within the time window.
+     *
+     * @param constraint the constraint to be satisfied
+     */
+    public void setConstraint(AbstractConstraint constraint) {
+      this.constraint = constraint;
+    }
+
+    /**
+     * Get the scheduling delay value that determines the time window within
+     * which the constraint has to be satisfied.
+     *
+     * @return the value of the scheduling delay
+     */
+    public long getSchedulingDelay() {
+      return schedulingDelay;
+    }
+
+    /**
+     * The unit of the scheduling delay.
+     *
+     * @return the unit of the delay
+     */
+    public DelayUnit getDelayUnit() {
+      return delayUnit;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
new file mode 100644
index 0000000..8e84280
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
@@ -0,0 +1,286 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.resource;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+
+/**
+ * This class contains various static methods for the applications to create
+ * placement constraints (see also {@link PlacementConstraint}).
+ */
+@Public
+@Unstable
+public final class PlacementConstraints {
+
+  // Suppresses default constructor, ensuring non-instantiability.
+  private PlacementConstraints() {
+  }
+
+  // Creation of simple constraints.
+
+  public static final String NODE = PlacementConstraint.NODE_SCOPE;
+  public static final String RACK = PlacementConstraint.RACK_SCOPE;
+
+  /**
+   * Creates a constraint that requires allocations to be placed on nodes that
+   * satisfy all target expressions within the given scope (e.g., node or rack).
+   *
+   * For example, {@code targetIn(RACK, allocationTag("hbase-m"))}, allows
+   * allocations on nodes that belong to a rack that has at least one tag with
+   * value "hbase-m".
+   *
+   * @param scope the scope within which the target expressions should be
+   *          satisfied
+   * @param targetExpressions the expressions that need to be satisfied within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetIn(String scope,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, 1, Integer.MAX_VALUE, targetExpressions);
+  }
+
+  /**
+   * Creates a constraint that requires allocations to be placed on nodes that
+   * belong to a scope (e.g., node or rack) that does not satisfy any of the
+   * target expressions.
+   *
+   * @param scope the scope within which the target expressions should not be
+   *          true
+   * @param targetExpressions the expressions that need to not be true within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetNotIn(String scope,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, 0, 0, targetExpressions);
+  }
+
+  /**
+   * Creates a constraint that restricts the number of allocations within a
+   * given scope (e.g., node or rack).
+   *
+   * For example, {@code cardinality(NODE, 3, 10)}, restricts the number of
+   * allocations per node to be no less than 3 and no more than 10.
+   *
+   * @param scope the scope of the constraint
+   * @param minCardinality determines the minimum number of allocations within
+   *          the scope
+   * @param maxCardinality determines the maximum number of allocations within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint cardinality(String scope, int minCardinality,
+      int maxCardinality) {
+    return new SingleConstraint(scope, minCardinality, maxCardinality,
+        PlacementTargets.self());
+  }
+
+  /**
+   * Similar to {@link #cardinality(String, int, int)}, but determines only the
+   * minimum cardinality (the maximum cardinality is unbound).
+   *
+   * @param scope the scope of the constraint
+   * @param minCardinality determines the minimum number of allocations within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint minCardinality(String scope,
+      int minCardinality) {
+    return cardinality(scope, minCardinality, Integer.MAX_VALUE);
+  }
+
+  /**
+   * Similar to {@link #cardinality(String, int, int)}, but determines only the
+   * maximum cardinality (the minimum can be as low as 0).
+   *
+   * @param scope the scope of the constraint
+   * @param maxCardinality determines the maximum number of allocations within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint maxCardinality(String scope,
+      int maxCardinality) {
+    return cardinality(scope, 0, maxCardinality);
+  }
+
+  /**
+   * This constraint generalizes the cardinality and target constraints.
+   *
+   * Consider a set of nodes N that belongs to the scope specified in the
+   * constraint. If the target expressions are satisfied at least minCardinality
+   * times and at most max-cardinality times in the node set N, then the
+   * constraint is satisfied.
+   *
+   * For example, {@code targetCardinality(RACK, 2, 10, allocationTag("zk"))},
+   * requires an allocation to be placed within a rack that has at least 2 and
+   * at most 10 other allocations with tag "zk".
+   *
+   * @param scope the scope of the constraint
+   * @param minCardinality the minimum number of times the target expressions
+   *          have to be satisfied with the given scope
+   * @param maxCardinality the maximum number of times the target expressions
+   *          have to be satisfied with the given scope
+   * @param targetExpressions the target expressions
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetCardinality(String scope,
+      int minCardinality, int maxCardinality,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, minCardinality, maxCardinality,
+        targetExpressions);
+  }
+
+  // Creation of target expressions to be used in simple constraints.
+
+  /**
+   * Class with static methods for constructing target expressions to be used in
+   * placement constraints.
+   */
+  public static class PlacementTargets {
+
+    /**
+     * Constructs a target expression on a node attribute. It is satisfied if
+     * the specified node attribute has one of the specified values.
+     *
+     * @param attributeKey the name of the node attribute
+     * @param attributeValues the set of values that the attribute should take
+     *          values from
+     * @return the resulting expression on the node attribute
+     */
+    public static TargetExpression nodeAttribute(String attributeKey,
+        String... attributeValues) {
+      return new TargetExpression(TargetType.NODE_ATTRIBUTE, attributeKey,
+          attributeValues);
+    }
+
+    /**
+     * Constructs a target expression on an allocation tag. It is satisfied if
+     * the there are allocations with one of the given tags.
+     *
+     * @param allocationTags the set of tags that the attribute should take
+     *          values from
+     * @return the resulting expression on the allocation tags
+     */
+    public static TargetExpression allocationTag(String... allocationTags) {
+      return new TargetExpression(TargetType.ALLOCATION_TAG, null,
+          allocationTags);
+    }
+
+    /**
+     * The default target expression that uses as target the allocation that
+     * specifies the constraint.
+     *
+     * @return the self-target
+     */
+    public static TargetExpression self() {
+      return new TargetExpression(TargetType.SELF);
+    }
+  }
+
+  // Creation of compound constraints.
+
+  /**
+   * A conjunction of constraints.
+   *
+   * @param children the children constraints that should all be satisfied
+   * @return the resulting placement constraint
+   */
+  public static And and(AbstractConstraint... children) {
+    return new And(children);
+  }
+
+  /**
+   * A disjunction of constraints.
+   *
+   * @param children the children constraints, one of which should be satisfied
+   * @return the resulting placement constraint
+   */
+  public static Or or(AbstractConstraint... children) {
+    return new Or(children);
+  }
+
+  /**
+   * Creates a composite constraint that includes a list of timed placement
+   * constraints. The scheduler should try to satisfy first the first timed
+   * child constraint within the specified time window. If this is not possible,
+   * it should attempt to satisfy the second, and so on.
+   *
+   * @param children the timed children constraints
+   * @return the resulting composite constraint
+   */
+  public static DelayedOr delayedOr(TimedPlacementConstraint... children) {
+    return new DelayedOr(children);
+  }
+
+  // Creation of timed constraints to be used in a DELAYED_OR constraint.
+
+  /**
+   * Creates a placement constraint that has to be satisfied within a time
+   * window.
+   *
+   * @param constraint the placement constraint
+   * @param delay the length of the time window within which the constraint has
+   *          to be satisfied
+   * @param timeUnit the unit of time of the time window
+   * @return the resulting timed placement constraint
+   */
+  public static TimedPlacementConstraint timedClockConstraint(
+      AbstractConstraint constraint, long delay, TimeUnit timeUnit) {
+    return new TimedPlacementConstraint(constraint, timeUnit.toMillis(delay),
+        TimedPlacementConstraint.DelayUnit.MILLISECONDS);
+  }
+
+  /**
+   * Creates a placement constraint that has to be satisfied within a number of
+   * placement opportunities (invocations of the scheduler).
+   *
+   * @param constraint the placement constraint
+   * @param delay the number of scheduling opportunities within which the
+   *          constraint has to be satisfied
+   * @return the resulting timed placement constraint
+   */
+  public static TimedPlacementConstraint timedOpportunitiesConstraint(
+      AbstractConstraint constraint, long delay) {
+    return new TimedPlacementConstraint(constraint, delay,
+        TimedPlacementConstraint.DelayUnit.OPPORTUNITIES);
+  }
+
+  /**
+   * Creates a {@link PlacementConstraint} given a constraint expression.
+   *
+   * @param constraintExpr the constraint expression
+   * @return the placement constraint
+   */
+  public static PlacementConstraint build(AbstractConstraint constraintExpr) {
+    return constraintExpr.build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
new file mode 100644
index 0000000..660dc02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * API related to resources.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.api.resource;
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 3a9662b..968b75e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -578,6 +578,61 @@ enum SignalContainerCommandProto {
   FORCEFUL_SHUTDOWN = 3;
 }
 
+////////////////////////////////////////////////////////////////////////
+////// Placement constraints ///////////////////////////////////////////
+////////////////////////////////////////////////////////////////////////
+
+message PlacementConstraintProto {
+  optional SimplePlacementConstraintProto simpleConstraint = 1;
+  optional CompositePlacementConstraintProto compositeConstraint = 2;
+}
+
+message SimplePlacementConstraintProto {
+  required string scope = 1;
+  repeated PlacementConstraintTargetProto targetExpressions = 2;
+  optional int32 minCardinality = 3;
+  optional int32 maxCardinality = 4;
+}
+
+message PlacementConstraintTargetProto {
+  enum TargetType {
+    NODE_ATTRIBUTE = 1;
+    ALLOCATION_TAG = 2;
+    SELF = 3;
+  }
+
+  required TargetType targetType = 1;
+  optional string targetKey = 2;
+  repeated string targetValues = 3;
+}
+
+message TimedPlacementConstraintProto {
+  enum DelayUnit {
+    MILLISECONDS = 1;
+    OPPORTUNITIES = 2;
+  }
+
+  required PlacementConstraintProto placementConstraint = 1;
+  required int64 schedulingDelay = 2;
+  optional DelayUnit delayUnit = 3 [ default = MILLISECONDS ];
+}
+
+message CompositePlacementConstraintProto {
+  enum CompositeType {
+    // All children constraints have to be satisfied.
+    AND = 1;
+    // One of the children constraints has to be satisfied.
+    OR = 2;
+    // Attempt to satisfy the first child constraint for delays[0] units (e.g.,
+    // millisec or heartbeats). If this fails, try to satisfy the second child
+    // constraint for delays[1] units and so on.
+    DELAYED_OR = 3;
+  }
+
+  required CompositeType compositeType = 1;
+  repeated PlacementConstraintProto childConstraints = 2;
+  repeated TimedPlacementConstraintProto timedChildConstraints = 3;
+}
 
 ////////////////////////////////////////////////////////////////////////
 ////// From reservation_protocol /////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java
new file mode 100644
index 0000000..e25d477
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.resource;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.and;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNotIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.nodeAttribute;
+
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for the various static methods in
+ * {@link org.apache.hadoop.yarn.api.resource.PlacementConstraints}.
+ */
+public class TestPlacementConstraints {
+
+  @Test
+  public void testNodeAffinityToTag() {
+    AbstractConstraint constraintExpr =
+        targetIn(NODE, allocationTag("hbase-m"));
+
+    SingleConstraint sConstraint = (SingleConstraint) constraintExpr;
+    Assert.assertEquals(NODE, sConstraint.getScope());
+    Assert.assertEquals(1, sConstraint.getMinCardinality());
+    Assert.assertEquals(Integer.MAX_VALUE, sConstraint.getMaxCardinality());
+
+    Assert.assertEquals(1, sConstraint.getTargetExpressions().size());
+    TargetExpression tExpr =
+        sConstraint.getTargetExpressions().iterator().next();
+    Assert.assertNull(tExpr.getTargetKey());
+    Assert.assertEquals(TargetType.ALLOCATION_TAG, tExpr.getTargetType());
+    Assert.assertEquals(1, tExpr.getTargetValues().size());
+    Assert.assertEquals("hbase-m", tExpr.getTargetValues().iterator().next());
+
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+    Assert.assertNotNull(constraint.getConstraintExpr());
+  }
+
+  @Test
+  public void testNodeAntiAffinityToAttribute() {
+    AbstractConstraint constraintExpr =
+        targetNotIn(NODE, nodeAttribute("java", "1.8"));
+
+    SingleConstraint sConstraint = (SingleConstraint) constraintExpr;
+    Assert.assertEquals(NODE, sConstraint.getScope());
+    Assert.assertEquals(0, sConstraint.getMinCardinality());
+    Assert.assertEquals(0, sConstraint.getMaxCardinality());
+
+    Assert.assertEquals(1, sConstraint.getTargetExpressions().size());
+    TargetExpression tExpr =
+        sConstraint.getTargetExpressions().iterator().next();
+    Assert.assertEquals("java", tExpr.getTargetKey());
+    Assert.assertEquals(TargetType.NODE_ATTRIBUTE, tExpr.getTargetType());
+    Assert.assertEquals(1, tExpr.getTargetValues().size());
+    Assert.assertEquals("1.8", tExpr.getTargetValues().iterator().next());
+  }
+
+  @Test
+  public void testAndConstraint() {
+    AbstractConstraint constraintExpr =
+        and(targetIn(RACK, allocationTag("spark")), maxCardinality(NODE, 3),
+            targetCardinality(RACK, 2, 10, allocationTag("zk")));
+
+    And andExpr = (And) constraintExpr;
+    Assert.assertEquals(3, andExpr.getChildren().size());
+    SingleConstraint sConstr = (SingleConstraint) andExpr.getChildren().get(0);
+    TargetExpression tExpr = sConstr.getTargetExpressions().iterator().next();
+    Assert.assertEquals("spark", tExpr.getTargetValues().iterator().next());
+
+    sConstr = (SingleConstraint) andExpr.getChildren().get(1);
+    Assert.assertEquals(0, sConstr.getMinCardinality());
+    Assert.assertEquals(3, sConstr.getMaxCardinality());
+
+    sConstr = (SingleConstraint) andExpr.getChildren().get(2);
+    Assert.assertEquals(2, sConstr.getMinCardinality());
+    Assert.assertEquals(10, sConstr.getMaxCardinality());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
new file mode 100644
index 0000000..926b6fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.pb;
+
+import static org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType.AND;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.TimedPlacementConstraintProto;
+
+/**
+ * {@code PlacementConstraintFromProtoConverter} generates an
+ * {@link PlacementConstraint.AbstractConstraint} given a
+ * {@link PlacementConstraintProto}.
+ */
+@Private
+public class PlacementConstraintFromProtoConverter {
+
+  private PlacementConstraintProto constraintProto;
+
+  public PlacementConstraintFromProtoConverter(
+      PlacementConstraintProto constraintProto) {
+    this.constraintProto = constraintProto;
+  }
+
+  public PlacementConstraint convert() {
+    return new PlacementConstraint(convert(constraintProto));
+  }
+
+  private AbstractConstraint convert(PlacementConstraintProto proto) {
+    return proto.hasSimpleConstraint() ? convert(proto.getSimpleConstraint())
+        : convert(proto.getCompositeConstraint());
+  }
+
+  private SingleConstraint convert(SimplePlacementConstraintProto proto) {
+    Set<TargetExpression> targets = new HashSet<>();
+    for (PlacementConstraintTargetProto tp : proto.getTargetExpressionsList()) {
+      targets.add(convert(tp));
+    }
+
+    return new SingleConstraint(proto.getScope(), proto.getMinCardinality(),
+        proto.getMaxCardinality(), targets);
+  }
+
+  private TargetExpression convert(PlacementConstraintTargetProto proto) {
+    return new TargetExpression(
+        ProtoUtils.convertFromProtoFormat(proto.getTargetType()),
+        proto.hasTargetKey() ? proto.getTargetKey() : null,
+        new HashSet<>(proto.getTargetValuesList()));
+  }
+
+  private AbstractConstraint convert(CompositePlacementConstraintProto proto) {
+    switch (proto.getCompositeType()) {
+    case AND:
+    case OR:
+      List<AbstractConstraint> children = new ArrayList<>();
+      for (PlacementConstraintProto cp : proto.getChildConstraintsList()) {
+        children.add(convert(cp));
+      }
+      return (proto.getCompositeType() == AND) ? new And(children)
+          : new Or(children);
+    case DELAYED_OR:
+      List<TimedPlacementConstraint> tChildren = new ArrayList<>();
+      for (TimedPlacementConstraintProto cp : proto
+          .getTimedChildConstraintsList()) {
+        tChildren.add(convert(cp));
+      }
+      return new DelayedOr(tChildren);
+    default:
+      throw new YarnRuntimeException(
+          "Encountered unexpected type of composite constraint.");
+    }
+  }
+
+  private TimedPlacementConstraint convert(
+      TimedPlacementConstraintProto proto) {
+    AbstractConstraint pConstraint = convert(proto.getPlacementConstraint());
+
+    return new TimedPlacementConstraint(pConstraint, proto.getSchedulingDelay(),
+        ProtoUtils.convertFromProtoFormat(proto.getDelayUnit()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
new file mode 100644
index 0000000..7816e18
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CardinalityConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CompositeConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.TimedPlacementConstraintProto;
+
+import com.google.protobuf.GeneratedMessage;
+
+/**
+ * {@code PlacementConstraintToProtoConverter} generates a
+ * {@link PlacementConstraintProto} given a
+ * {@link PlacementConstraint.AbstractConstraint}.
+ */
+@Private
+public class PlacementConstraintToProtoConverter
+    implements PlacementConstraint.Visitor<GeneratedMessage> {
+
+  private PlacementConstraint placementConstraint;
+
+  public PlacementConstraintToProtoConverter(
+      PlacementConstraint placementConstraint) {
+    this.placementConstraint = placementConstraint;
+  }
+
+  public PlacementConstraintProto convert() {
+    return (PlacementConstraintProto) placementConstraint.getConstraintExpr()
+        .accept(this);
+  }
+
+  @Override
+  public GeneratedMessage visit(SingleConstraint constraint) {
+    SimplePlacementConstraintProto.Builder sb =
+        SimplePlacementConstraintProto.newBuilder();
+
+    if (constraint.getScope() != null) {
+      sb.setScope(constraint.getScope());
+    }
+    sb.setMinCardinality(constraint.getMinCardinality());
+    sb.setMaxCardinality(constraint.getMaxCardinality());
+    if (constraint.getTargetExpressions() != null) {
+      for (TargetExpression target : constraint.getTargetExpressions()) {
+        sb.addTargetExpressions(
+            (PlacementConstraintTargetProto) target.accept(this));
+      }
+
+    }
+    SimplePlacementConstraintProto sProto = sb.build();
+
+    // Wrap around PlacementConstraintProto object.
+    PlacementConstraintProto.Builder pb = PlacementConstraintProto.newBuilder();
+    pb.setSimpleConstraint(sProto);
+    return pb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(TargetExpression target) {
+    PlacementConstraintTargetProto.Builder tb =
+        PlacementConstraintTargetProto.newBuilder();
+
+    tb.setTargetType(ProtoUtils.convertToProtoFormat(target.getTargetType()));
+    if (target.getTargetKey() != null) {
+      tb.setTargetKey(target.getTargetKey());
+    }
+    if (target.getTargetValues() != null) {
+      tb.addAllTargetValues(target.getTargetValues());
+    }
+    return tb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(TargetConstraint constraint) {
+    throw new YarnRuntimeException("Unexpected TargetConstraint found.");
+  }
+
+  @Override
+  public GeneratedMessage visit(CardinalityConstraint constraint) {
+    throw new YarnRuntimeException("Unexpected CardinalityConstraint found.");
+  }
+
+  private GeneratedMessage visitAndOr(
+      CompositeConstraint<AbstractConstraint> composite, CompositeType type) {
+    CompositePlacementConstraintProto.Builder cb =
+        CompositePlacementConstraintProto.newBuilder();
+
+    cb.setCompositeType(type);
+
+    for (AbstractConstraint c : composite.getChildren()) {
+      cb.addChildConstraints((PlacementConstraintProto) c.accept(this));
+    }
+    CompositePlacementConstraintProto cProto = cb.build();
+
+    // Wrap around PlacementConstraintProto object.
+    PlacementConstraintProto.Builder pb = PlacementConstraintProto.newBuilder();
+    pb.setCompositeConstraint(cProto);
+    return pb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(And constraint) {
+    return visitAndOr(constraint, CompositeType.AND);
+  }
+
+  @Override
+  public GeneratedMessage visit(Or constraint) {
+    return visitAndOr(constraint, CompositeType.OR);
+  }
+
+  @Override
+  public GeneratedMessage visit(DelayedOr constraint) {
+    CompositePlacementConstraintProto.Builder cb =
+        CompositePlacementConstraintProto.newBuilder();
+
+    cb.setCompositeType(CompositeType.DELAYED_OR);
+
+    for (TimedPlacementConstraint c : constraint.getChildren()) {
+      cb.addTimedChildConstraints(
+          (TimedPlacementConstraintProto) c.accept(this));
+    }
+    CompositePlacementConstraintProto cProto = cb.build();
+
+    // Wrap around PlacementConstraintProto object.
+    PlacementConstraintProto.Builder pb = PlacementConstraintProto.newBuilder();
+    pb.setCompositeConstraint(cProto);
+    return pb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(TimedPlacementConstraint constraint) {
+    TimedPlacementConstraintProto.Builder tb =
+        TimedPlacementConstraintProto.newBuilder();
+
+    tb.setDelayUnit(ProtoUtils.convertToProtoFormat(constraint.getDelayUnit()));
+    tb.setSchedulingDelay(constraint.getSchedulingDelay());
+    tb.setPlacementConstraint(
+        (PlacementConstraintProto) constraint.getConstraint().accept(this));
+
+    return tb.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java
new file mode 100644
index 0000000..18da80f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * API related to protobuf objects that are not backed by PBImpl classes.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.api.pb;
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index f3e665b..168d864 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.api.records.UpdateContainerError;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
 import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAccessTypeProto;
@@ -70,10 +72,12 @@ import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceVisibilityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.TimedPlacementConstraintProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryPolicyProto;
@@ -507,6 +511,29 @@ public class ProtoUtils {
     }
     return ret;
   }
+
+  public static PlacementConstraintTargetProto.TargetType convertToProtoFormat(
+          TargetExpression.TargetType t) {
+    return PlacementConstraintTargetProto.TargetType.valueOf(t.name());
+  }
+
+  public static TargetExpression.TargetType convertFromProtoFormat(
+          PlacementConstraintTargetProto.TargetType t) {
+    return TargetExpression.TargetType.valueOf(t.name());
+  }
+
+  /*
+   * TimedPlacementConstraint.DelayUnit
+   */
+  public static TimedPlacementConstraintProto.DelayUnit convertToProtoFormat(
+          TimedPlacementConstraint.DelayUnit u) {
+    return TimedPlacementConstraintProto.DelayUnit.valueOf(u.name());
+  }
+
+  public static TimedPlacementConstraint.DelayUnit convertFromProtoFormat(
+          TimedPlacementConstraintProto.DelayUnit u) {
+    return TimedPlacementConstraint.DelayUnit.valueOf(u.name());
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java
new file mode 100644
index 0000000..e9eda6f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.resource;
+
+import java.util.ListIterator;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CardinalityConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CompositeConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint.TargetOperator;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * This class contains inner classes that define transformation on a
+ * {@link PlacementConstraint} expression.
+ */
+@Private
+public class PlacementConstraintTransformations {
+
+  /**
+   * The default implementation of the {@link PlacementConstraint.Visitor} that
+   * does a traversal of the constraint tree, performing no action for the lead
+   * constraints.
+   */
+  public static class AbstractTransformer
+      implements PlacementConstraint.Visitor<AbstractConstraint> {
+
+    private PlacementConstraint placementConstraint;
+
+    public AbstractTransformer(PlacementConstraint placementConstraint) {
+      this.placementConstraint = placementConstraint;
+    }
+
+    /**
+     * This method performs the transformation of the
+     * {@link #placementConstraint}.
+     *
+     * @return the transformed placement constraint.
+     */
+    public PlacementConstraint transform() {
+      AbstractConstraint constraintExpr =
+          placementConstraint.getConstraintExpr();
+
+      // Visit the constraint tree to perform the transformation.
+      constraintExpr = constraintExpr.accept(this);
+
+      return new PlacementConstraint(constraintExpr);
+    }
+
+    @Override
+    public AbstractConstraint visit(SingleConstraint constraint) {
+      // Do nothing.
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(TargetExpression expression) {
+      // Do nothing.
+      return null;
+    }
+
+    @Override
+    public AbstractConstraint visit(TargetConstraint constraint) {
+      // Do nothing.
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(CardinalityConstraint constraint) {
+      // Do nothing.
+      return constraint;
+    }
+
+    private AbstractConstraint visitAndOr(
+        CompositeConstraint<AbstractConstraint> constraint) {
+      for (ListIterator<AbstractConstraint> iter =
+          constraint.getChildren().listIterator(); iter.hasNext();) {
+        AbstractConstraint child = iter.next();
+        child = child.accept(this);
+        iter.set(child);
+      }
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(And constraint) {
+      return visitAndOr(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(Or constraint) {
+      return visitAndOr(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(DelayedOr constraint) {
+      constraint.getChildren().forEach(
+          child -> child.setConstraint(child.getConstraint().accept(this)));
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(TimedPlacementConstraint constraint) {
+      // Do nothing.
+      return null;
+    }
+  }
+
+  /**
+   * Visits a {@link PlacementConstraint} tree and substitutes each
+   * {@link TargetConstraint} and {@link CardinalityConstraint} with an
+   * equivalent {@link SingleConstraint}.
+   */
+  public static class SingleConstraintTransformer extends AbstractTransformer {
+
+    public SingleConstraintTransformer(PlacementConstraint constraint) {
+      super(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(TargetConstraint constraint) {
+      AbstractConstraint newConstraint;
+      if (constraint.getOp() == TargetOperator.IN) {
+        newConstraint = new SingleConstraint(constraint.getScope(), 1,
+            Integer.MAX_VALUE, constraint.getTargetExpressions());
+      } else if (constraint.getOp() == TargetOperator.NOT_IN) {
+        newConstraint = new SingleConstraint(constraint.getScope(), 0, 0,
+            constraint.getTargetExpressions());
+      } else {
+        throw new YarnRuntimeException(
+            "Encountered unexpected type of constraint target operator: "
+                + constraint.getOp());
+      }
+      return newConstraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(CardinalityConstraint constraint) {
+      return new SingleConstraint(constraint.getScope(),
+          constraint.getMinCardinality(), constraint.getMaxCardinality(),
+          new TargetExpression(TargetExpression.TargetType.SELF));
+    }
+  }
+
+  /**
+   * Visits a {@link PlacementConstraint} tree and, whenever possible,
+   * substitutes each {@link SingleConstraint} with a {@link TargetConstraint}
+   * or a {@link CardinalityConstraint}. When such a substitution is not
+   * possible, we keep the original {@link SingleConstraint}.
+   */
+  public static class SpecializedConstraintTransformer
+      extends AbstractTransformer {
+
+    public SpecializedConstraintTransformer(PlacementConstraint constraint) {
+      super(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(SingleConstraint constraint) {
+      AbstractConstraint transformedConstraint = constraint;
+      // Check if it is a cardinality constraint.
+      if (constraint.getTargetExpressions().size() == 1) {
+        TargetExpression targetExpr =
+            constraint.getTargetExpressions().iterator().next();
+        if (targetExpr.getTargetType() == TargetExpression.TargetType.SELF) {
+          transformedConstraint = new CardinalityConstraint(
+              constraint.getScope(), constraint.getMinCardinality(),
+              constraint.getMaxCardinality());
+        }
+      }
+      // Check if it is a target constraint.
+      if (constraint.getMinCardinality() == 1
+          && constraint.getMaxCardinality() == Integer.MAX_VALUE) {
+        transformedConstraint = new TargetConstraint(TargetOperator.IN,
+            constraint.getScope(), constraint.getTargetExpressions());
+      } else if (constraint.getMinCardinality() == 0
+          && constraint.getMaxCardinality() == 0) {
+        transformedConstraint = new TargetConstraint(TargetOperator.NOT_IN,
+            constraint.getScope(), constraint.getTargetExpressions());
+      }
+
+      return transformedConstraint;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
new file mode 100644
index 0000000..660dc02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * API related to resources.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.api.resource;
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java
new file mode 100644
index 0000000..bd245e2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.cardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.or;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+
+import java.util.Iterator;
+
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintFromProtoConverter;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintToProtoConverter;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for {@link PlacementConstraintToProtoConverter} and
+ * {@link PlacementConstraintFromProtoConverter}.
+ */
+public class TestPlacementConstraintPBConversion {
+
+  @Test
+  public void testTargetConstraintProtoConverter() {
+    AbstractConstraint sConstraintExpr =
+        targetIn(NODE, allocationTag("hbase-m"));
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Convert to proto.
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(sConstraint);
+    PlacementConstraintProto protoConstraint = toProtoConverter.convert();
+
+    Assert.assertTrue(protoConstraint.hasSimpleConstraint());
+    Assert.assertFalse(protoConstraint.hasCompositeConstraint());
+    SimplePlacementConstraintProto sProto =
+        protoConstraint.getSimpleConstraint();
+    Assert.assertEquals(single.getScope(), sProto.getScope());
+    Assert.assertEquals(single.getMinCardinality(), sProto.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(), sProto.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions().size(),
+        sProto.getTargetExpressionsList().size());
+
+    // Convert from proto.
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(protoConstraint);
+    PlacementConstraint newConstraint = fromProtoConverter.convert();
+
+    AbstractConstraint newConstraintExpr = newConstraint.getConstraintExpr();
+    Assert.assertTrue(newConstraintExpr instanceof SingleConstraint);
+    SingleConstraint newSingle = (SingleConstraint) newConstraintExpr;
+    Assert.assertEquals(single.getScope(), newSingle.getScope());
+    Assert.assertEquals(single.getMinCardinality(),
+        newSingle.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(),
+        newSingle.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions(),
+        newSingle.getTargetExpressions());
+  }
+
+  @Test
+  public void testCardinalityConstraintProtoConverter() {
+    AbstractConstraint sConstraintExpr = cardinality(RACK, 3, 10);
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Convert to proto.
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(sConstraint);
+    PlacementConstraintProto protoConstraint = toProtoConverter.convert();
+
+    compareSimpleConstraintToProto(single, protoConstraint);
+
+    // Convert from proto.
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(protoConstraint);
+    PlacementConstraint newConstraint = fromProtoConverter.convert();
+
+    AbstractConstraint newConstraintExpr = newConstraint.getConstraintExpr();
+    Assert.assertTrue(newConstraintExpr instanceof SingleConstraint);
+    SingleConstraint newSingle = (SingleConstraint) newConstraintExpr;
+    compareSimpleConstraints(single, newSingle);
+  }
+
+  @Test
+  public void testCompositeConstraintProtoConverter() {
+    AbstractConstraint constraintExpr =
+        or(targetIn(RACK, allocationTag("spark")), maxCardinality(NODE, 3),
+            targetCardinality(RACK, 2, 10, allocationTag("zk")));
+    Assert.assertTrue(constraintExpr instanceof Or);
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+    Or orExpr = (Or) constraintExpr;
+
+    // Convert to proto.
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(constraint);
+    PlacementConstraintProto protoConstraint = toProtoConverter.convert();
+
+    Assert.assertFalse(protoConstraint.hasSimpleConstraint());
+    Assert.assertTrue(protoConstraint.hasCompositeConstraint());
+    CompositePlacementConstraintProto cProto =
+        protoConstraint.getCompositeConstraint();
+
+    Assert.assertEquals(CompositeType.OR, cProto.getCompositeType());
+    Assert.assertEquals(3, cProto.getChildConstraintsCount());
+    Assert.assertEquals(0, cProto.getTimedChildConstraintsCount());
+    Iterator<AbstractConstraint> orChildren = orExpr.getChildren().iterator();
+    Iterator<PlacementConstraintProto> orProtoChildren =
+        cProto.getChildConstraintsList().iterator();
+    while (orChildren.hasNext() && orProtoChildren.hasNext()) {
+      AbstractConstraint orChild = orChildren.next();
+      PlacementConstraintProto orProtoChild = orProtoChildren.next();
+      compareSimpleConstraintToProto((SingleConstraint) orChild, orProtoChild);
+    }
+
+    // Convert from proto.
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(protoConstraint);
+    PlacementConstraint newConstraint = fromProtoConverter.convert();
+
+    AbstractConstraint newConstraintExpr = newConstraint.getConstraintExpr();
+    Assert.assertTrue(newConstraintExpr instanceof Or);
+    Or newOrExpr = (Or) newConstraintExpr;
+    Assert.assertEquals(3, newOrExpr.getChildren().size());
+    orChildren = orExpr.getChildren().iterator();
+    Iterator<AbstractConstraint> newOrChildren =
+        newOrExpr.getChildren().iterator();
+    while (orChildren.hasNext() && newOrChildren.hasNext()) {
+      AbstractConstraint orChild = orChildren.next();
+      AbstractConstraint newOrChild = newOrChildren.next();
+      compareSimpleConstraints((SingleConstraint) orChild,
+          (SingleConstraint) newOrChild);
+    }
+  }
+
+  private void compareSimpleConstraintToProto(SingleConstraint constraint,
+      PlacementConstraintProto proto) {
+    Assert.assertTrue(proto.hasSimpleConstraint());
+    Assert.assertFalse(proto.hasCompositeConstraint());
+    SimplePlacementConstraintProto sProto = proto.getSimpleConstraint();
+    Assert.assertEquals(constraint.getScope(), sProto.getScope());
+    Assert.assertEquals(constraint.getMinCardinality(),
+        sProto.getMinCardinality());
+    Assert.assertEquals(constraint.getMaxCardinality(),
+        sProto.getMaxCardinality());
+    Assert.assertEquals(constraint.getTargetExpressions().size(),
+        sProto.getTargetExpressionsList().size());
+  }
+
+  private void compareSimpleConstraints(SingleConstraint single,
+      SingleConstraint newSingle) {
+    Assert.assertEquals(single.getScope(), newSingle.getScope());
+    Assert.assertEquals(single.getMinCardinality(),
+        newSingle.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(),
+        newSingle.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions(),
+        newSingle.getTargetExpressions());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd57dc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
new file mode 100644
index 0000000..1763735
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.resource;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.cardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.or;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CardinalityConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint.TargetOperator;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraintTransformations.SingleConstraintTransformer;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraintTransformations.SpecializedConstraintTransformer;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for {@link PlacementConstraintTransformations}.
+ */
+public class TestPlacementConstraintTransformations {
+
+  @Test
+  public void testTargetConstraint() {
+    AbstractConstraint sConstraintExpr =
+        targetIn(NODE, allocationTag("hbase-m"));
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Transform from SimpleConstraint to specialized TargetConstraint
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(sConstraint);
+    PlacementConstraint tConstraint = specTransformer.transform();
+
+    AbstractConstraint tConstraintExpr = tConstraint.getConstraintExpr();
+    Assert.assertTrue(tConstraintExpr instanceof TargetConstraint);
+
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    TargetConstraint target = (TargetConstraint) tConstraintExpr;
+    Assert.assertEquals(single.getScope(), target.getScope());
+    Assert.assertEquals(TargetOperator.IN, target.getOp());
+    Assert.assertEquals(single.getTargetExpressions(),
+        target.getTargetExpressions());
+
+    // Transform from specialized TargetConstraint to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(tConstraint);
+    sConstraint = singleTransformer.transform();
+
+    sConstraintExpr = sConstraint.getConstraintExpr();
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+
+    single = (SingleConstraint) sConstraintExpr;
+    Assert.assertEquals(target.getScope(), single.getScope());
+    Assert.assertEquals(1, single.getMinCardinality());
+    Assert.assertEquals(Integer.MAX_VALUE, single.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions(),
+        target.getTargetExpressions());
+  }
+
+  @Test
+  public void testCardinalityConstraint() {
+    AbstractConstraint sConstraintExpr = cardinality(RACK, 3, 10);
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Transform from SimpleConstraint to specialized CardinalityConstraint
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(sConstraint);
+    PlacementConstraint cConstraint = specTransformer.transform();
+
+    AbstractConstraint cConstraintExpr = cConstraint.getConstraintExpr();
+    Assert.assertTrue(cConstraintExpr instanceof CardinalityConstraint);
+
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    CardinalityConstraint cardinality = (CardinalityConstraint) cConstraintExpr;
+    Assert.assertEquals(single.getScope(), cardinality.getScope());
+    Assert.assertEquals(single.getMinCardinality(),
+        cardinality.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(),
+        cardinality.getMaxCardinality());
+
+    // Transform from specialized CardinalityConstraint to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(cConstraint);
+    sConstraint = singleTransformer.transform();
+
+    sConstraintExpr = sConstraint.getConstraintExpr();
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+
+    single = (SingleConstraint) sConstraintExpr;
+    Assert.assertEquals(cardinality.getScope(), single.getScope());
+    Assert.assertEquals(cardinality.getMinCardinality(),
+        single.getMinCardinality());
+    Assert.assertEquals(cardinality.getMaxCardinality(),
+        single.getMaxCardinality());
+    Assert.assertEquals(new HashSet<>(Arrays.asList(PlacementTargets.self())),
+        single.getTargetExpressions());
+  }
+
+  @Test
+  public void testTargetCardinalityConstraint() {
+    AbstractConstraint constraintExpr =
+        targetCardinality(RACK, 3, 10, allocationTag("zk"));
+    Assert.assertTrue(constraintExpr instanceof SingleConstraint);
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+
+    // Apply transformation. Should be a no-op.
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(constraint);
+    PlacementConstraint newConstraint = specTransformer.transform();
+
+    // The constraint expression should be the same.
+    Assert.assertEquals(constraintExpr, newConstraint.getConstraintExpr());
+  }
+
+  @Test
+  public void testCompositeConstraint() {
+    AbstractConstraint constraintExpr =
+        or(targetIn(RACK, allocationTag("spark")), maxCardinality(NODE, 3),
+            targetCardinality(RACK, 2, 10, allocationTag("zk")));
+    Assert.assertTrue(constraintExpr instanceof Or);
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+    Or orExpr = (Or) constraintExpr;
+    for (AbstractConstraint child : orExpr.getChildren()) {
+      Assert.assertTrue(child instanceof SingleConstraint);
+    }
+
+    // Apply transformation. Should transform target and cardinality constraints
+    // included in the composite constraint to specialized ones.
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(constraint);
+    PlacementConstraint specConstraint = specTransformer.transform();
+
+    Or specOrExpr = (Or) specConstraint.getConstraintExpr();
+    List<AbstractConstraint> specChildren = specOrExpr.getChildren();
+    Assert.assertEquals(3, specChildren.size());
+    Assert.assertTrue(specChildren.get(0) instanceof TargetConstraint);
+    Assert.assertTrue(specChildren.get(1) instanceof CardinalityConstraint);
+    Assert.assertTrue(specChildren.get(2) instanceof SingleConstraint);
+
+    // Transform from specialized TargetConstraint to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(specConstraint);
+    PlacementConstraint simConstraint = singleTransformer.transform();
+    Assert.assertTrue(constraintExpr instanceof Or);
+    Or simOrExpr = (Or) specConstraint.getConstraintExpr();
+    for (AbstractConstraint child : simOrExpr.getChildren()) {
+      Assert.assertTrue(child instanceof SingleConstraint);
+    }
+  }
+
+}


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


[14/50] [abbrv] hadoop git commit: YARN-7607. Remove the trailing duplicated timestamp in container diagnostics message. Contributed by Weiwei Yang.

Posted by wa...@apache.org.
YARN-7607. Remove the trailing duplicated timestamp in container diagnostics message. Contributed by Weiwei Yang.


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

Branch: refs/heads/YARN-6592
Commit: e411dd6666041a4ea68ab34e734802271497ae6c
Parents: c2e8a5c
Author: Weiwei Yang <ww...@apache.org>
Authored: Thu Dec 7 17:29:40 2017 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Thu Dec 7 17:29:40 2017 +0800

----------------------------------------------------------------------
 .../container/ContainerImpl.java                | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e411dd66/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 901b013..f95c2a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -1541,7 +1541,7 @@ public class ContainerImpl implements Container {
       ContainerExitEvent exitEvent = (ContainerExitEvent) event;
       container.exitCode = exitEvent.getExitCode();
       if (exitEvent.getDiagnosticInfo() != null) {
-        container.addDiagnostics(exitEvent.getDiagnosticInfo(), "\n");
+        container.addDiagnostics(exitEvent.getDiagnosticInfo() + "\n");
       }
 
       // TODO: Add containerWorkDir to the deletion service.
@@ -1578,7 +1578,7 @@ public class ContainerImpl implements Container {
           container.addDiagnostics("Diagnostic message from attempt "
               + n + " : ", "\n");
         }
-        container.addDiagnostics(exitEvent.getDiagnosticInfo(), "\n");
+        container.addDiagnostics(exitEvent.getDiagnosticInfo() + "\n");
       }
 
       if (container.shouldRetry(container.exitCode)) {
@@ -1737,7 +1737,7 @@ public class ContainerImpl implements Container {
 
       ContainerResourceFailedEvent rsrcFailedEvent =
           (ContainerResourceFailedEvent) event;
-      container.addDiagnostics(rsrcFailedEvent.getDiagnosticMessage(), "\n");
+      container.addDiagnostics(rsrcFailedEvent.getDiagnosticMessage() + "\n");
 
       // Inform the localizer to decrement reference counts and cleanup
       // resources.
@@ -1760,7 +1760,7 @@ public class ContainerImpl implements Container {
       container.metrics.endInitingContainer();
       ContainerKillEvent killEvent = (ContainerKillEvent) event;
       container.exitCode = killEvent.getContainerExitStatus();
-      container.addDiagnostics(killEvent.getDiagnostic(), "\n");
+      container.addDiagnostics(killEvent.getDiagnostic() + "\n");
       container.addDiagnostics("Container is killed before being launched.\n");
     }
   }
@@ -1799,7 +1799,7 @@ public class ContainerImpl implements Container {
           new ContainersLauncherEvent(container,
               ContainersLauncherEventType.CLEANUP_CONTAINER));
       ContainerKillEvent killEvent = (ContainerKillEvent) event;
-      container.addDiagnostics(killEvent.getDiagnostic(), "\n");
+      container.addDiagnostics(killEvent.getDiagnostic() + "\n");
       container.exitCode = killEvent.getContainerExitStatus();
     }
   }
@@ -1838,7 +1838,7 @@ public class ContainerImpl implements Container {
       }
 
       if (exitEvent.getDiagnosticInfo() != null) {
-        container.addDiagnostics(exitEvent.getDiagnosticInfo(), "\n");
+        container.addDiagnostics(exitEvent.getDiagnosticInfo() + "\n");
       }
 
       // The process/process-grp is killed. Decrement reference counts and
@@ -1890,7 +1890,7 @@ public class ContainerImpl implements Container {
       } else {
         ContainerKillEvent killEvent = (ContainerKillEvent) event;
         container.exitCode = killEvent.getContainerExitStatus();
-        container.addDiagnostics(killEvent.getDiagnostic(), "\n");
+        container.addDiagnostics(killEvent.getDiagnostic() + "\n");
         container.addDiagnostics("Container is killed before being launched.\n");
         container.metrics.killedContainer();
         NMAuditLogger.logSuccess(container.user,
@@ -2010,7 +2010,7 @@ public class ContainerImpl implements Container {
     public void transition(ContainerImpl container, ContainerEvent event) {
       ContainerDiagnosticsUpdateEvent updateEvent =
           (ContainerDiagnosticsUpdateEvent) event;
-      container.addDiagnostics(updateEvent.getDiagnosticsUpdate(), "\n");
+      container.addDiagnostics(updateEvent.getDiagnosticsUpdate() + "\n");
     }
   }
 
@@ -2028,7 +2028,7 @@ public class ContainerImpl implements Container {
           new ContainersLauncherEvent(container,
               ContainersLauncherEventType.PAUSE_CONTAINER));
       ContainerPauseEvent pauseEvent = (ContainerPauseEvent) event;
-      container.addDiagnostics(pauseEvent.getDiagnostic(), "\n");
+      container.addDiagnostics(pauseEvent.getDiagnostic() + "\n");
     }
   }
 
@@ -2061,7 +2061,7 @@ public class ContainerImpl implements Container {
           new ContainersLauncherEvent(container,
               ContainersLauncherEventType.RESUME_CONTAINER));
       ContainerResumeEvent resumeEvent = (ContainerResumeEvent) event;
-      container.addDiagnostics(resumeEvent.getDiagnostic(), "\n");
+      container.addDiagnostics(resumeEvent.getDiagnostic() + "\n");
     }
   }
 


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


[33/50] [abbrv] hadoop git commit: HADOOP-15104. AliyunOSS: change the default value of max error retry. Contributed by Jinhu Wu

Posted by wa...@apache.org.
HADOOP-15104. AliyunOSS: change the default value of max error retry. Contributed by Jinhu Wu


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

Branch: refs/heads/YARN-6592
Commit: ce04340ec73617daff74378056a95c5d0cc0a790
Parents: 19e0894
Author: Kai Zheng <zh...@alibaba-inc.com>
Authored: Fri Dec 8 23:03:54 2017 +0800
Committer: Kai Zheng <zh...@alibaba-inc.com>
Committed: Fri Dec 8 23:03:54 2017 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce04340e/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
index baa171f..dd71842 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
@@ -66,7 +66,7 @@ public final class Constants {
 
   // Number of times we should retry errors
   public static final String MAX_ERROR_RETRIES_KEY = "fs.oss.attempts.maximum";
-  public static final int MAX_ERROR_RETRIES_DEFAULT = 20;
+  public static final int MAX_ERROR_RETRIES_DEFAULT = 10;
 
   // Time until we give up trying to establish a connection to oss
   public static final String ESTABLISH_TIMEOUT_KEY =


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


[12/50] [abbrv] hadoop git commit: MAPREDUCE-6998. Moving logging APIs over to slf4j in hadoop-mapreduce-client-jobclient. Contributed by Gergely Novák.

Posted by wa...@apache.org.
MAPREDUCE-6998. Moving logging APIs over to slf4j in hadoop-mapreduce-client-jobclient. Contributed by Gergely Novák.


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

Branch: refs/heads/YARN-6592
Commit: d4cae977a2471ad7b8f803617e41b6f94df19c11
Parents: 6cca5b3
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 7 16:21:25 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 7 16:21:25 2017 +0900

----------------------------------------------------------------------
 .../java/org/apache/hadoop/mapred/ClientCache.java   |  6 +++---
 .../apache/hadoop/mapred/ClientServiceDelegate.java  |  7 ++++---
 .../apache/hadoop/mapred/ResourceMgrDelegate.java    |  7 ++++---
 .../java/org/apache/hadoop/mapred/YARNRunner.java    |  6 +++---
 .../java/org/apache/hadoop/fi/ProbabilityModel.java  |  7 ++++---
 .../org/apache/hadoop/fs/AccumulatingReducer.java    |  8 +++++---
 .../test/java/org/apache/hadoop/fs/DFSCIOTest.java   |  6 +++---
 .../org/apache/hadoop/fs/DistributedFSCheck.java     |  7 ++++---
 .../java/org/apache/hadoop/fs/JHLogAnalyzer.java     |  7 ++++---
 .../test/java/org/apache/hadoop/fs/TestDFSIO.java    |  6 +++---
 .../src/test/java/org/apache/hadoop/fs/TestJHLA.java |  7 ++++---
 .../hadoop/fs/loadGenerator/LoadGeneratorMR.java     |  6 +++---
 .../java/org/apache/hadoop/fs/slive/AppendOp.java    |  6 +++---
 .../org/apache/hadoop/fs/slive/ConfigExtractor.java  |  7 ++++---
 .../java/org/apache/hadoop/fs/slive/CreateOp.java    |  6 +++---
 .../java/org/apache/hadoop/fs/slive/DeleteOp.java    |  6 +++---
 .../test/java/org/apache/hadoop/fs/slive/ListOp.java |  6 +++---
 .../java/org/apache/hadoop/fs/slive/MkdirOp.java     |  6 +++---
 .../test/java/org/apache/hadoop/fs/slive/ReadOp.java |  6 +++---
 .../java/org/apache/hadoop/fs/slive/RenameOp.java    |  6 +++---
 .../org/apache/hadoop/fs/slive/ReportWriter.java     |  6 +++---
 .../java/org/apache/hadoop/fs/slive/SleepOp.java     |  6 +++---
 .../java/org/apache/hadoop/fs/slive/SliveMapper.java |  6 +++---
 .../org/apache/hadoop/fs/slive/SliveReducer.java     |  6 +++---
 .../java/org/apache/hadoop/fs/slive/SliveTest.java   |  6 +++---
 .../java/org/apache/hadoop/fs/slive/TestSlive.java   | 10 +++++-----
 .../java/org/apache/hadoop/fs/slive/TruncateOp.java  |  6 +++---
 .../org/apache/hadoop/fs/slive/WeightSelector.java   |  7 ++++---
 .../test/java/org/apache/hadoop/hdfs/NNBench.java    |  7 +++----
 .../org/apache/hadoop/hdfs/NNBenchWithoutMR.java     |  8 ++++----
 .../java/org/apache/hadoop/mapred/BigMapOutput.java  |  8 ++++----
 .../test/java/org/apache/hadoop/mapred/MRBench.java  |  6 +++---
 .../java/org/apache/hadoop/mapred/MiniMRCluster.java | 15 ++++++++-------
 .../hadoop/mapred/MiniMRYarnClusterAdapter.java      |  7 ++++---
 .../org/apache/hadoop/mapred/ReliabilityTest.java    | 13 +++++++------
 .../org/apache/hadoop/mapred/TestBadRecords.java     |  8 ++++----
 .../org/apache/hadoop/mapred/TestClientRedirect.java |  7 ++++---
 .../hadoop/mapred/TestCombineFileInputFormat.java    | 11 +++++------
 .../mapred/TestCombineSequenceFileInputFormat.java   |  8 ++++----
 .../hadoop/mapred/TestCombineTextInputFormat.java    |  8 ++++----
 .../mapred/TestConcatenatedCompressedInput.java      |  9 +++++----
 .../hadoop/mapred/TestFixedLengthInputFormat.java    | 11 ++++++-----
 .../org/apache/hadoop/mapred/TestJobCleanup.java     |  9 +++++----
 .../apache/hadoop/mapred/TestJobSysDirWithDFS.java   |  8 ++++----
 .../hadoop/mapred/TestKeyValueTextInputFormat.java   |  8 +++++---
 .../hadoop/mapred/TestMRTimelineEventHandling.java   |  8 ++++----
 .../org/apache/hadoop/mapred/TestMapProgress.java    |  8 +++++---
 .../apache/hadoop/mapred/TestMiniMRChildTask.java    |  8 ++++----
 .../hadoop/mapred/TestMultiFileInputFormat.java      |  8 +++++---
 .../mapred/TestSequenceFileAsBinaryOutputFormat.java | 13 ++++++-------
 .../org/apache/hadoop/mapred/TestSortedRanges.java   | 12 ++++++------
 .../mapred/TestSpecialCharactersInOutputPath.java    | 15 +++++++--------
 .../org/apache/hadoop/mapred/TestTaskStatus.java     |  4 +---
 .../apache/hadoop/mapred/TestTextInputFormat.java    | 11 +++++------
 .../org/apache/hadoop/mapred/TestYARNRunner.java     | 14 ++++++++------
 .../apache/hadoop/mapred/ThreadedMapBenchmark.java   |  7 ++++---
 .../java/org/apache/hadoop/mapred/UtilsForTests.java |  6 +++---
 .../mapred/jobcontrol/TestLocalJobControl.java       |  9 +++++----
 .../org/apache/hadoop/mapred/pipes/TestPipes.java    |  8 ++++----
 .../org/apache/hadoop/mapreduce/GrowingSleepJob.java | 12 ++++++------
 .../hadoop/mapreduce/JobHistoryFileParser.java       |  7 ++++---
 .../hadoop/mapreduce/JobHistoryFileReplayHelper.java |  9 ++++-----
 .../mapreduce/JobHistoryFileReplayMapperV1.java      |  9 ++++-----
 .../mapreduce/JobHistoryFileReplayMapperV2.java      |  8 ++++----
 .../apache/hadoop/mapreduce/MapReduceTestUtil.java   | 13 ++++---------
 .../hadoop/mapreduce/MiniHadoopClusterManager.java   |  8 ++++----
 .../hadoop/mapreduce/SimpleEntityWriterV1.java       |  7 ++++---
 .../hadoop/mapreduce/SimpleEntityWriterV2.java       |  9 +++++----
 .../org/apache/hadoop/mapreduce/TestCounters.java    | 11 +++++------
 .../org/apache/hadoop/mapreduce/TestLocalRunner.java | 11 ++++++-----
 .../org/apache/hadoop/mapreduce/TestMRJobClient.java |  7 ++++---
 .../apache/hadoop/mapreduce/TestMapCollection.java   | 13 ++++++-------
 .../apache/hadoop/mapreduce/TestValueIterReset.java  |  8 ++++----
 .../hadoop/mapreduce/TimelineEntityConverterV1.java  |  8 ++++----
 .../hadoop/mapreduce/TimelineEntityConverterV2.java  |  8 ++++----
 .../lib/db/TestDataDrivenDBInputFormat.java          |  6 +++---
 .../input/TestCombineSequenceFileInputFormat.java    |  8 ++++----
 .../lib/input/TestCombineTextInputFormat.java        |  8 ++++----
 .../lib/input/TestFixedLengthInputFormat.java        | 12 ++++++------
 .../lib/input/TestMRKeyValueTextInputFormat.java     | 12 +++++-------
 .../lib/input/TestMRSequenceFileInputFilter.java     |  8 ++++----
 .../lib/jobcontrol/TestMapReduceJobControl.java      | 12 ++++++------
 .../TestMRSequenceFileAsBinaryOutputFormat.java      |  8 ++++----
 .../mapreduce/lib/partition/TestKeyFieldHelper.java  | 10 ++++++----
 .../hadoop/mapreduce/security/TestJHSSecurity.java   | 10 +++++-----
 .../hadoop/mapreduce/util/MRAsyncDiskService.java    |  7 ++++---
 .../mapreduce/util/TestMRAsyncDiskService.java       |  9 ++++-----
 .../hadoop/mapreduce/v2/MiniMRYarnCluster.java       |  7 ++++---
 .../v2/TestMRAMWithNonNormalizedCapabilities.java    |  7 ++++---
 .../hadoop/mapreduce/v2/TestMRAppWithCombiner.java   |  7 ++++---
 .../org/apache/hadoop/mapreduce/v2/TestMRJobs.java   |  6 +++---
 .../mapreduce/v2/TestMRJobsWithHistoryService.java   |  8 ++++----
 .../hadoop/mapreduce/v2/TestMRJobsWithProfiler.java  |  8 ++++----
 .../apache/hadoop/mapreduce/v2/TestMROldApiJobs.java |  7 ++++---
 .../org/apache/hadoop/mapreduce/v2/TestRMNMInfo.java |  7 ++++---
 .../mapreduce/v2/TestSpeculativeExecution.java       |  7 ++++---
 .../org/apache/hadoop/mapreduce/v2/TestUberAM.java   |  6 +++---
 .../src/test/java/testjar/UserNamePermission.java    | 13 ++++---------
 98 files changed, 411 insertions(+), 394 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java
index 93ea5c4..8268d1e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java
@@ -23,8 +23,6 @@ import java.security.PrivilegedAction;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.mapreduce.JobID;
@@ -35,13 +33,15 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ClientCache {
 
   private final Configuration conf;
   private final ResourceMgrDelegate rm;
 
-  private static final Log LOG = LogFactory.getLog(ClientCache.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ClientCache.class);
 
   private Map<JobID, ClientServiceDelegate> cache = 
       new HashMap<JobID, ClientServiceDelegate>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
index 72339e5..792e496 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
@@ -29,8 +29,6 @@ import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.RPC;
@@ -79,11 +77,14 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
 public class ClientServiceDelegate {
-  private static final Log LOG = LogFactory.getLog(ClientServiceDelegate.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ClientServiceDelegate.class);
   private static final String UNAVAILABLE = "N/A";
 
   // Caches for per-user NotRunningJobs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
index 94f741a..ac4b73b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
@@ -25,8 +25,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -78,11 +76,14 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
 public class ResourceMgrDelegate extends YarnClient {
-  private static final Log LOG = LogFactory.getLog(ResourceMgrDelegate.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ResourceMgrDelegate.class);
       
   private YarnConfiguration conf;
   private ApplicationSubmissionContext application;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
index 12a3079..127e1dc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
@@ -36,8 +36,6 @@ import java.util.Vector;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
@@ -99,6 +97,8 @@ import org.apache.hadoop.yarn.security.client.RMDelegationTokenSelector;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -108,7 +108,7 @@ import com.google.common.annotations.VisibleForTesting;
 @SuppressWarnings("unchecked")
 public class YARNRunner implements ClientProtocol {
 
-  private static final Log LOG = LogFactory.getLog(YARNRunner.class);
+  private static final Logger LOG = LoggerFactory.getLogger(YARNRunner.class);
 
   private static final String RACK_GROUP = "rack";
   private static final String NODE_IF_RACK_GROUP = "node1";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fi/ProbabilityModel.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fi/ProbabilityModel.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fi/ProbabilityModel.java
index 48e57f5..ce3f1fa 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fi/ProbabilityModel.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fi/ProbabilityModel.java
@@ -19,9 +19,9 @@ package org.apache.hadoop.fi;
 
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class is responsible for the decision of when a fault 
@@ -42,7 +42,8 @@ import org.apache.hadoop.conf.Configuration;
  */
 public class ProbabilityModel {
   private static Random generator = new Random();
-  private static final Log LOG = LogFactory.getLog(ProbabilityModel.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ProbabilityModel.class);
 
   static final String FPROB_NAME = "fi.";
   private static final String ALL_PROBABILITIES = FPROB_NAME + "*";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/AccumulatingReducer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/AccumulatingReducer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/AccumulatingReducer.java
index 3991d94..f6c2a06 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/AccumulatingReducer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/AccumulatingReducer.java
@@ -20,10 +20,10 @@ package org.apache.hadoop.fs;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Reducer that accumulates values based on their type.
@@ -47,7 +47,9 @@ public class AccumulatingReducer extends MapReduceBase
   static final String VALUE_TYPE_LONG = "l:";
   static final String VALUE_TYPE_FLOAT = "f:";
   static final String VALUE_TYPE_STRING = "s:";
-  private static final Log LOG = LogFactory.getLog(AccumulatingReducer.class);
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AccumulatingReducer.class);
   
   protected String hostName;
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java
index b01954e..d718556 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java
@@ -28,8 +28,6 @@ import java.io.PrintStream;
 import java.util.Date;
 import java.util.StringTokenizer;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
@@ -38,6 +36,8 @@ import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.mapred.*;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
  /**
  * Distributed i/o benchmark.
@@ -69,7 +69,7 @@ import org.junit.Test;
 @Ignore
 public class DFSCIOTest {
   // Constants
-  private static final Log LOG = LogFactory.getLog(DFSCIOTest.class);
+  private static final Logger LOG = LoggerFactory.getLogger(DFSCIOTest.class);
   private static final int TEST_TYPE_READ = 0;
   private static final int TEST_TYPE_WRITE = 1;
   private static final int TEST_TYPE_CLEANUP = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DistributedFSCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DistributedFSCheck.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DistributedFSCheck.java
index 67ef5d9..8ed2c19 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DistributedFSCheck.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DistributedFSCheck.java
@@ -33,8 +33,6 @@ import java.util.Vector;
 
 import junit.framework.TestCase;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
@@ -42,6 +40,8 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.mapred.*;
 import org.junit.Ignore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Distributed checkup of the file system consistency.
@@ -56,7 +56,8 @@ import org.junit.Ignore;
 @Ignore
 public class DistributedFSCheck extends TestCase {
   // Constants
-  private static final Log LOG = LogFactory.getLog(DistributedFSCheck.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DistributedFSCheck.class);
   private static final int TEST_TYPE_READ = 0;
   private static final int TEST_TYPE_CLEANUP = 2;
   private static final int DEFAULT_BUFFER_SIZE = 1000000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/JHLogAnalyzer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/JHLogAnalyzer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/JHLogAnalyzer.java
index 91c3c26..5e3e745 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/JHLogAnalyzer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/JHLogAnalyzer.java
@@ -34,8 +34,6 @@ import java.util.Map;
 import java.util.StringTokenizer;
 import java.util.HashMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
@@ -46,6 +44,8 @@ import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Job History Log Analyzer.
@@ -144,7 +144,8 @@ import org.apache.hadoop.util.StringUtils;
  */
 @SuppressWarnings("deprecation")
 public class JHLogAnalyzer {
-  private static final Log LOG = LogFactory.getLog(JHLogAnalyzer.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JHLogAnalyzer.class);
   // Constants
   private static final String JHLA_ROOT_DIR = 
                             System.getProperty("test.build.data", "stats/JHLA");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
index 61485be..68befea 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
@@ -33,8 +33,6 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.Random;
 import java.util.StringTokenizer;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -62,6 +60,8 @@ import org.apache.hadoop.util.ToolRunner;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Distributed i/o benchmark.
@@ -92,7 +92,7 @@ import org.junit.Test;
  */
 public class TestDFSIO implements Tool {
   // Constants
-  private static final Log LOG = LogFactory.getLog(TestDFSIO.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestDFSIO.class);
   private static final int DEFAULT_BUFFER_SIZE = 1000000;
   private static final String BASE_FILE_NAME = "test_io_";
   private static final String DEFAULT_RES_FILE_NAME = "TestDFSIO_results.log";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java
index 31950fd..9334a8a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java
@@ -23,11 +23,11 @@ import java.io.FileOutputStream;
 import java.io.OutputStreamWriter;
 import java.io.File;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Test Job History Log Analyzer.
@@ -35,7 +35,8 @@ import org.junit.Test;
  * @see JHLogAnalyzer
  */
 public class TestJHLA {
-  private static final Log LOG = LogFactory.getLog(JHLogAnalyzer.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JHLogAnalyzer.class);
   private String historyLog = System.getProperty("test.build.data", 
                                   "build/test/data") + "/history/test.log";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGeneratorMR.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGeneratorMR.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGeneratorMR.java
index c47d971..044c77c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGeneratorMR.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGeneratorMR.java
@@ -26,8 +26,6 @@ import java.net.UnknownHostException;
 import java.util.EnumSet;
 import java.util.Iterator;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.CreateFlag;
@@ -50,6 +48,8 @@ import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** The load generator is a tool for testing NameNode behavior under
  * different client loads.
@@ -63,7 +63,7 @@ import org.apache.hadoop.util.ToolRunner;
  *
  */
 public class LoadGeneratorMR extends LoadGenerator {
-  public static final Log LOG = LogFactory.getLog(LoadGenerator.class);
+  public static final Logger LOG = LoggerFactory.getLogger(LoadGenerator.class);
   private static int numMapTasks = 1;
   private String mrOutDir;
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/AppendOp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/AppendOp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/AppendOp.java
index 82d221f..b118e6e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/AppendOp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/AppendOp.java
@@ -24,12 +24,12 @@ import java.io.OutputStream;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.slive.DataWriter.GenerateOutput;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Operation which selects a random file and appends a random amount of bytes
@@ -41,7 +41,7 @@ import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
  */
 class AppendOp extends Operation {
 
-  private static final Log LOG = LogFactory.getLog(AppendOp.class);
+  private static final Logger LOG = LoggerFactory.getLogger(AppendOp.class);
 
   AppendOp(ConfigExtractor cfg, Random rnd) {
     super(AppendOp.class.getSimpleName(), cfg, rnd);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ConfigExtractor.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ConfigExtractor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ConfigExtractor.java
index ef4e436..2668770 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ConfigExtractor.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ConfigExtractor.java
@@ -22,12 +22,12 @@ import java.text.NumberFormat;
 import java.util.HashMap;
 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.fs.Path;
 import org.apache.hadoop.fs.slive.Constants.OperationType;
 import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Simple access layer onto of a configuration object that extracts the slive
@@ -35,7 +35,8 @@ import org.apache.hadoop.util.StringUtils;
  */
 class ConfigExtractor {
 
-  private static final Log LOG = LogFactory.getLog(ConfigExtractor.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConfigExtractor.class);
 
   private Configuration config;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/CreateOp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/CreateOp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/CreateOp.java
index 7da32c7..d7d2989 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/CreateOp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/CreateOp.java
@@ -22,13 +22,13 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.slive.DataWriter.GenerateOutput;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Operation which selects a random file and a random number of bytes to create
@@ -42,7 +42,7 @@ import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
  */
 class CreateOp extends Operation {
 
-  private static final Log LOG = LogFactory.getLog(CreateOp.class);
+  private static final Logger LOG = LoggerFactory.getLogger(CreateOp.class);
 
   private static int DEF_IO_BUFFER_SIZE = 4096;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/DeleteOp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/DeleteOp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/DeleteOp.java
index 31afd16..0ed425a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/DeleteOp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/DeleteOp.java
@@ -23,11 +23,11 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Operation which selects a random file and attempts to delete that file (if it
@@ -39,7 +39,7 @@ import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
  */
 class DeleteOp extends Operation {
 
-  private static final Log LOG = LogFactory.getLog(DeleteOp.class);
+  private static final Logger LOG = LoggerFactory.getLogger(DeleteOp.class);
 
   DeleteOp(ConfigExtractor cfg, Random rnd) {
     super(DeleteOp.class.getSimpleName(), cfg, rnd);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ListOp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ListOp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ListOp.java
index f83a6bd..46377d5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ListOp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ListOp.java
@@ -23,12 +23,12 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Operation which selects a random directory and attempts to list that
@@ -41,7 +41,7 @@ import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
  */
 class ListOp extends Operation {
 
-  private static final Log LOG = LogFactory.getLog(ListOp.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ListOp.class);
 
   ListOp(ConfigExtractor cfg, Random rnd) {
     super(ListOp.class.getSimpleName(), cfg, rnd);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/MkdirOp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/MkdirOp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/MkdirOp.java
index 585f418..0f24d14 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/MkdirOp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/MkdirOp.java
@@ -23,11 +23,11 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Operation which selects a random directory and attempts to create that
@@ -40,7 +40,7 @@ import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
  */
 class MkdirOp extends Operation {
 
-  private static final Log LOG = LogFactory.getLog(MkdirOp.class);
+  private static final Logger LOG = LoggerFactory.getLogger(MkdirOp.class);
 
   MkdirOp(ConfigExtractor cfg, Random rnd) {
     super(MkdirOp.class.getSimpleName(), cfg, rnd);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReadOp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReadOp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReadOp.java
index 9683323..d6f29a6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReadOp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReadOp.java
@@ -24,12 +24,12 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.slive.DataVerifier.VerifyOutput;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Operation which selects a random file and selects a random read size (from
@@ -43,7 +43,7 @@ import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
  * number of failures and the amount of time taken to fail
  */
 class ReadOp extends Operation {
-  private static final Log LOG = LogFactory.getLog(ReadOp.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ReadOp.class);
 
   ReadOp(ConfigExtractor cfg, Random rnd) {
     super(ReadOp.class.getSimpleName(), cfg, rnd);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/RenameOp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/RenameOp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/RenameOp.java
index 94d6db4..a608a87 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/RenameOp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/RenameOp.java
@@ -23,11 +23,11 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Operation which selects a random file and a second random file and attempts
@@ -60,7 +60,7 @@ class RenameOp extends Operation {
     }
   }
 
-  private static final Log LOG = LogFactory.getLog(RenameOp.class);
+  private static final Logger LOG = LoggerFactory.getLogger(RenameOp.class);
 
   RenameOp(ConfigExtractor cfg, Random rnd) {
     super(RenameOp.class.getSimpleName(), cfg, rnd);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReportWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReportWriter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReportWriter.java
index 1f9abde..873a2ffd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReportWriter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/ReportWriter.java
@@ -24,8 +24,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class which provides a report for the given operation output
@@ -48,7 +48,7 @@ class ReportWriter {
   static final String NOT_FOUND = "files_not_found";
   static final String BAD_FILES = "bad_files";
 
-  private static final Log LOG = LogFactory.getLog(ReportWriter.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ReportWriter.class);
 
   private static final String SECTION_DELIM = "-------------";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SleepOp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SleepOp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SleepOp.java
index 9fc9b30..2b9d54e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SleepOp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SleepOp.java
@@ -21,10 +21,10 @@ package org.apache.hadoop.fs.slive;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Operation which sleeps for a given number of milliseconds according to the
@@ -32,7 +32,7 @@ import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
  */
 class SleepOp extends Operation {
 
-  private static final Log LOG = LogFactory.getLog(SleepOp.class);
+  private static final Logger LOG = LoggerFactory.getLogger(SleepOp.class);
 
   SleepOp(ConfigExtractor cfg, Random rnd) {
     super(SleepOp.class.getSimpleName(), cfg, rnd);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveMapper.java
index e02c5d9..93ee9f9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveMapper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveMapper.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
 import org.apache.hadoop.io.Text;
@@ -35,6 +33,8 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The slive class which sets up the mapper to be used which itself will receive
@@ -45,7 +45,7 @@ import org.apache.hadoop.util.StringUtils;
 public class SliveMapper extends MapReduceBase implements
     Mapper<Object, Object, Text, Text> {
 
-  private static final Log LOG = LogFactory.getLog(SliveMapper.class);
+  private static final Logger LOG = LoggerFactory.getLogger(SliveMapper.class);
 
   private static final String OP_TYPE = SliveMapper.class.getSimpleName();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveReducer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveReducer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveReducer.java
index 323559c..d4f2473 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveReducer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveReducer.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.fs.slive;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
@@ -30,6 +28,8 @@ import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The slive reducer which iterates over the given input values and merges them
@@ -38,7 +38,7 @@ import org.apache.hadoop.util.StringUtils;
 public class SliveReducer extends MapReduceBase implements
     Reducer<Text, Text, Text, Text> {
 
-  private static final Log LOG = LogFactory.getLog(SliveReducer.class);
+  private static final Logger LOG = LoggerFactory.getLogger(SliveReducer.class);
 
   private ConfigExtractor config;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveTest.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveTest.java
index 97360d6..c2918ab 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveTest.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/SliveTest.java
@@ -30,8 +30,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,6 +43,8 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Slive test entry point + main program
@@ -61,7 +61,7 @@ import org.apache.hadoop.util.ToolRunner;
  */
 public class SliveTest implements Tool {
 
-  private static final Log LOG = LogFactory.getLog(SliveTest.class);
+  private static final Logger LOG = LoggerFactory.getLogger(SliveTest.class);
 
   // ensures the hdfs configurations are loaded if they exist
   static {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TestSlive.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TestSlive.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TestSlive.java
index 25e3340..575cd0b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TestSlive.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TestSlive.java
@@ -31,8 +31,6 @@ import java.util.List;
 import java.util.Random;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -43,13 +41,15 @@ import org.apache.hadoop.fs.slive.DataWriter.GenerateOutput;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Junit 4 test for slive
  */
 public class TestSlive {
 
-  private static final Log LOG = LogFactory.getLog(TestSlive.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestSlive.class);
 
   private static final Random rnd = new Random(1L);
 
@@ -258,13 +258,13 @@ public class TestSlive {
     DataWriter writer = new DataWriter(rnd);
     FileOutputStream fs = new FileOutputStream(fn);
     GenerateOutput ostat = writer.writeSegment(byteAm, fs);
-    LOG.info(ostat);
+    LOG.info(ostat.toString());
     fs.close();
     assertTrue(ostat.getBytesWritten() == byteAm);
     DataVerifier vf = new DataVerifier();
     FileInputStream fin = new FileInputStream(fn);
     VerifyOutput vfout = vf.verifyFile(byteAm, new DataInputStream(fin));
-    LOG.info(vfout);
+    LOG.info(vfout.toString());
     fin.close();
     assertEquals(vfout.getBytesRead(), byteAm);
     assertTrue(vfout.getChunksDifferent() == 0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TruncateOp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TruncateOp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TruncateOp.java
index 202d807..295b797 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TruncateOp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/TruncateOp.java
@@ -22,12 +22,12 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.List;
 import java.util.Random;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Operation which selects a random file and truncates a random amount of bytes
@@ -40,7 +40,7 @@ import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
  */
 class TruncateOp extends Operation {
 
-  private static final Log LOG = LogFactory.getLog(TruncateOp.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TruncateOp.class);
 
   TruncateOp(ConfigExtractor cfg, Random rnd) {
     super(TruncateOp.class.getSimpleName(), cfg, rnd);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/WeightSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/WeightSelector.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/WeightSelector.java
index d8acc39..3d80357 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/WeightSelector.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/slive/WeightSelector.java
@@ -26,12 +26,12 @@ import java.util.Map;
 import java.util.Random;
 import java.util.TreeMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.slive.Constants.Distribution;
 import org.apache.hadoop.fs.slive.Constants.OperationType;
 import org.apache.hadoop.fs.slive.Weights.UniformWeight;
 import org.apache.hadoop.fs.slive.ObserveableOp.Observer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class is the main handler that selects operations to run using the
@@ -47,7 +47,8 @@ class WeightSelector {
     Double weight(int elapsed, int duration);
   }
 
-  private static final Log LOG = LogFactory.getLog(WeightSelector.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(WeightSelector.class);
 
   private static class OperationInfo {
     Integer amountLeft;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
index 29eac43..2346c3c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
@@ -30,8 +30,6 @@ import java.util.Date;
 import java.util.Iterator;
 import java.util.StringTokenizer;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -57,6 +55,8 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This program executes a specified operation that applies load to 
@@ -78,8 +78,7 @@ import org.apache.hadoop.util.ToolRunner;
  */
 
 public class NNBench extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(
-          "org.apache.hadoop.hdfs.NNBench");
+  private static final Logger LOG = LoggerFactory.getLogger(NNBench.class);
   
   private static String CONTROL_DIR_NAME = "control";
   private static String OUTPUT_DIR_NAME = "output";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBenchWithoutMR.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBenchWithoutMR.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBenchWithoutMR.java
index 9b63010..af16177 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBenchWithoutMR.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBenchWithoutMR.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hdfs;
 import java.io.IOException;
 import java.util.Date;
 
-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.FSDataOutputStream;
@@ -30,6 +28,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.mapred.JobConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This program executes a specified operation that applies load to 
@@ -45,8 +45,8 @@ import org.apache.hadoop.mapred.JobConf;
  */
 public class NNBenchWithoutMR {
   
-  private static final Log LOG = LogFactory.getLog(
-                                            "org.apache.hadoop.hdfs.NNBench");
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NNBenchWithoutMR.class);
   
   // variable initialzed from command line arguments
   private static long startTime = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/BigMapOutput.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/BigMapOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/BigMapOutput.java
index 14e32fd..964673b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/BigMapOutput.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/BigMapOutput.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.util.Date;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,10 +35,12 @@ import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class BigMapOutput extends Configured implements Tool {
-  public static final Log LOG =
-    LogFactory.getLog(BigMapOutput.class.getName());
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BigMapOutput.class);
   private static Random random = new Random();
   public static String MIN_KEY = "mapreduce.bmo.minkey";
   public static String MIN_VALUE = "mapreduce.bmo.minvalue";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MRBench.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MRBench.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MRBench.java
index 5286e86..5328756 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MRBench.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MRBench.java
@@ -24,8 +24,6 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -33,13 +31,15 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Runs a job multiple times and takes average of all runs.
  */
 public class MRBench extends Configured implements Tool{
   
-  private static final Log LOG = LogFactory.getLog(MRBench.class);
+  private static final Logger LOG = LoggerFactory.getLogger(MRBench.class);
   private static final String DEFAULT_INPUT_SUB = "mr_input";
   private static final String DEFAULT_OUTPUT_SUB = "mr_output";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRCluster.java
index 2e144414..e7df5b3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRCluster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRCluster.java
@@ -20,13 +20,13 @@ package org.apache.hadoop.mapred;
 import java.io.IOException;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class is an MR2 replacement for older MR1 MiniMRCluster, that was used
@@ -45,7 +45,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class MiniMRCluster {
-  private static final Log LOG = LogFactory.getLog(MiniMRCluster.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MiniMRCluster.class);
 
   private MiniMRClientCluster mrClientCluster;
 
@@ -98,7 +99,7 @@ public class MiniMRCluster {
     try {
       jobConf = new JobConf(mrClientCluster.getConfig());
     } catch (IOException e) {
-      LOG.error(e);
+      LOG.error(e.getMessage());
     }
     return jobConf;
   }
@@ -108,7 +109,7 @@ public class MiniMRCluster {
     try {
       jobConf = new JobConf(mrClientCluster.getConfig());
     } catch (IOException e) {
-      LOG.error(e);
+      LOG.error(e.getMessage());
     }
     return jobConf;
   }
@@ -224,7 +225,7 @@ public class MiniMRCluster {
     try {
       jobConf = new JobConf(mrClientCluster.getConfig());
     } catch (IOException e) {
-      LOG.error(e);
+      LOG.error(e.getMessage());
     }
     return jobConf;
   }
@@ -266,7 +267,7 @@ public class MiniMRCluster {
     try {
       mrClientCluster.stop();
     } catch (IOException e) {
-      LOG.error(e);
+      LOG.error(e.getMessage());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRYarnClusterAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRYarnClusterAdapter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRYarnClusterAdapter.java
index 94d6ff3..4f89840 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRYarnClusterAdapter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRYarnClusterAdapter.java
@@ -18,13 +18,13 @@
 
 package org.apache.hadoop.mapred;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * An adapter for MiniMRYarnCluster providing a MiniMRClientCluster interface.
@@ -34,7 +34,8 @@ public class MiniMRYarnClusterAdapter implements MiniMRClientCluster {
 
   private MiniMRYarnCluster miniMRYarnCluster;
 
-  private static final Log LOG = LogFactory.getLog(MiniMRYarnClusterAdapter.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MiniMRYarnClusterAdapter.class);
 
   public MiniMRYarnClusterAdapter(MiniMRYarnCluster miniMRYarnCluster) {
     this.miniMRYarnCluster = miniMRYarnCluster;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ReliabilityTest.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ReliabilityTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ReliabilityTest.java
index 983a4a7..303857b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ReliabilityTest.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ReliabilityTest.java
@@ -29,8 +29,6 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.StringTokenizer;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -41,6 +39,8 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class tests reliability of the framework in the face of failures of
@@ -73,7 +73,8 @@ import org.apache.hadoop.util.ToolRunner;
 public class ReliabilityTest extends Configured implements Tool {
 
   private String dir;
-  private static final Log LOG = LogFactory.getLog(ReliabilityTest.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReliabilityTest.class);
 
   private void displayUsage() {
     LOG.info("This must be run in only the distributed mode " +
@@ -207,7 +208,7 @@ public class ReliabilityTest extends Configured implements Tool {
               args);
           checkJobExitStatus(status, jobClass);
         } catch (Exception e) {
-          LOG.fatal("JOB " + jobClass + " failed to run");
+          LOG.error("JOB " + jobClass + " failed to run");
           System.exit(-1);
         }
       }
@@ -325,7 +326,7 @@ public class ReliabilityTest extends Configured implements Tool {
           killed = true;
           return;
         } catch (Exception e) {
-          LOG.fatal(StringUtils.stringifyException(e));
+          LOG.error(StringUtils.stringifyException(e));
         }
       }
     }
@@ -495,7 +496,7 @@ public class ReliabilityTest extends Configured implements Tool {
         } catch (InterruptedException ie) {
           killed = true;
         } catch (Exception e) {
-          LOG.fatal(StringUtils.stringifyException(e));
+          LOG.error(StringUtils.stringifyException(e));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java
index c2d6257..b45a2a6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java
@@ -30,8 +30,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.StringTokenizer;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
@@ -41,6 +39,8 @@ import org.apache.hadoop.mapreduce.TaskCounter;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -48,8 +48,8 @@ import static org.junit.Assert.assertNotNull;
 @Ignore
 public class TestBadRecords extends ClusterMapReduceTestCase {
   
-  private static final Log LOG = 
-    LogFactory.getLog(TestBadRecords.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestBadRecords.class);
   
   private static final List<String> MAPPER_BAD_RECORDS = 
     Arrays.asList("hello01","hello04","hello05");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
index a9b4626..f97d0a4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
@@ -24,8 +24,6 @@ import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.Iterator;
 
-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.mapreduce.Cluster;
@@ -144,6 +142,8 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.junit.Assert;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestClientRedirect {
 
@@ -151,7 +151,8 @@ public class TestClientRedirect {
     DefaultMetricsSystem.setMiniClusterMode(true);
   }
 
-  private static final Log LOG = LogFactory.getLog(TestClientRedirect.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestClientRedirect.class);
   private static final String RMADDRESS = "0.0.0.0:8054";
   private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineFileInputFormat.java
index de7880d..4ed9eb2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineFileInputFormat.java
@@ -26,16 +26,15 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.mapred.lib.CombineFileInputFormat;
 import org.apache.hadoop.mapred.lib.CombineFileSplit;
 import org.apache.hadoop.mapred.lib.CombineFileRecordReader;
-
 import org.junit.Test;
-import static org.junit.Assert.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import static org.junit.Assert.*;
 
 public class TestCombineFileInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestCombineFileInputFormat.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestCombineFileInputFormat.class);
   
   private static JobConf defaultConf = new JobConf();
   private static FileSystem localFs = null; 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java
index 8cdaa80..4f1d6ba 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java
@@ -25,8 +25,6 @@ import java.io.IOException;
 import java.util.BitSet;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -36,10 +34,12 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.mapred.lib.CombineFileSplit;
 import org.apache.hadoop.mapred.lib.CombineSequenceFileInputFormat;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestCombineSequenceFileInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestCombineSequenceFileInputFormat.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestCombineSequenceFileInputFormat.class);
 
   private static Configuration conf = new Configuration();
   private static FileSystem localFs = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java
index 581e62b..394630c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java
@@ -31,8 +31,6 @@ import java.util.BitSet;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
@@ -43,10 +41,12 @@ import org.apache.hadoop.mapred.lib.CombineFileSplit;
 import org.apache.hadoop.mapred.lib.CombineTextInputFormat;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestCombineTextInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestCombineTextInputFormat.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestCombineTextInputFormat.class);
 
   private static JobConf defaultConf = new JobConf();
   private static FileSystem localFs = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java
index 15d651d..977d083 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java
@@ -30,8 +30,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.zip.Inflater;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
@@ -46,10 +44,13 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.junit.After;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 @Ignore
 public class TestConcatenatedCompressedInput {
-  private static final Log LOG =
-    LogFactory.getLog(TestConcatenatedCompressedInput.class.getName());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestConcatenatedCompressedInput.class);
   private static int MAX_LENGTH = 10000;
   private static JobConf defaultConf = new JobConf();
   private static FileSystem localFs = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFixedLengthInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFixedLengthInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFixedLengthInputFormat.java
index 8013feb..4864dd0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFixedLengthInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFixedLengthInputFormat.java
@@ -26,8 +26,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -35,14 +33,18 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.compress.*;
 import org.apache.hadoop.util.ReflectionUtils;
-
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static org.junit.Assert.*;
 
 public class TestFixedLengthInputFormat {
 
-  private static Log LOG;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFixedLengthInputFormat.class);
+
   private static Configuration defaultConf;
   private static FileSystem localFs; 
   private static Path workDir;
@@ -55,7 +57,6 @@ public class TestFixedLengthInputFormat {
   @BeforeClass
   public static void onlyOnce() {
     try {
-      LOG = LogFactory.getLog(TestFixedLengthInputFormat.class.getName());
       defaultConf = new Configuration();
       defaultConf.set("fs.defaultFS", "file:///");
       localFs = FileSystem.getLocal(defaultConf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cae977/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobCleanup.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobCleanup.java
index bf762d9..13f2301 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobCleanup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobCleanup.java
@@ -22,9 +22,6 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.Log;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
@@ -36,6 +33,9 @@ import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.LoggerFactory;
+import org.slf4j.Logger;
+
 import static org.junit.Assert.*;
 
 /**
@@ -54,7 +54,8 @@ public class TestJobCleanup {
   private static Path emptyInDir = null;
   private static int outDirs = 0;
 
-  private static Log LOG = LogFactory.getLog(TestJobCleanup.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestJobCleanup.class);
 
   @BeforeClass
   public static void setUp() throws IOException {


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


[07/50] [abbrv] hadoop git commit: HDFS-12872. EC Checksum broken when BlockAccessToken is enabled.

Posted by wa...@apache.org.
HDFS-12872. EC Checksum broken when BlockAccessToken is enabled.


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

Branch: refs/heads/YARN-6592
Commit: 56b1ff80dd9fbcde8d21a604eff0babb3a16418f
Parents: 05c347f
Author: Xiao Chen <xi...@apache.org>
Authored: Tue Dec 5 20:48:02 2017 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Dec 5 20:48:16 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java | 3 ++-
 .../hadoop/hdfs/server/blockmanagement/BlockManager.java    | 9 ++++-----
 .../test/java/org/apache/hadoop/hdfs/TestFileChecksum.java  | 9 ++++++++-
 3 files changed, 14 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/56b1ff80/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
index 4dbaa01..e7d5b3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
@@ -192,6 +192,7 @@ public class LocatedBlocks {
         + "\n  underConstruction=" + underConstruction
         + "\n  blocks=" + blocks
         + "\n  lastLocatedBlock=" + lastLocatedBlock
-        + "\n  isLastBlockComplete=" + isLastBlockComplete + "}";
+        + "\n  isLastBlockComplete=" + isLastBlockComplete
+        + "\n  ecPolicy=" + ecPolicy + "}";
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56b1ff80/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 1cdb159..ae04aac 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
@@ -1378,12 +1378,11 @@ public class BlockManager implements BlockStatsMXBean {
               b.getStorageIDs());
         }
         sb.setBlockTokens(blockTokens);
-      } else {
-        b.setBlockToken(blockTokenSecretManager.generateToken(
-            NameNode.getRemoteUser().getShortUserName(),
-            b.getBlock(), EnumSet.of(mode), b.getStorageTypes(),
-            b.getStorageIDs()));
       }
+      b.setBlockToken(blockTokenSecretManager.generateToken(
+          NameNode.getRemoteUser().getShortUserName(),
+          b.getBlock(), EnumSet.of(mode), b.getStorageTypes(),
+          b.getStorageIDs()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56b1ff80/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
index 9d6687c..d201ce1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -33,8 +34,12 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.slf4j.event.Level;
 
 import java.io.IOException;
+import java.util.Random;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
 
 /**
  * This test serves a prototype to demo the idea proposed so far. It creates two
@@ -77,6 +82,7 @@ public class TestFileChecksum {
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
         false);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     Path ecPath = new Path(ecDir);
     cluster.getFileSystem().mkdir(ecPath, FsPermission.getDirDefault());
@@ -89,6 +95,7 @@ public class TestFileChecksum {
     bytesPerCRC = conf.getInt(
         HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
         HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
+    GenericTestUtils.setLogLevel(FileChecksumHelper.LOG, Level.DEBUG);
   }
 
   @After
@@ -518,7 +525,7 @@ public class TestFileChecksum {
 
     LocatedBlock locatedBlock = locatedBlocks.get(0);
     DatanodeInfo[] datanodes = locatedBlock.getLocations();
-    DatanodeInfo chosenDn = datanodes[0];
+    DatanodeInfo chosenDn = datanodes[new Random().nextInt(datanodes.length)];
 
     int idx = 0;
     for (DataNode dn : cluster.getDataNodes()) {


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


[17/50] [abbrv] hadoop git commit: HADOOP-15098. TestClusterTopology#testChooseRandom fails intermittently. Contributed by Zsolt Venczel.

Posted by wa...@apache.org.
HADOOP-15098. TestClusterTopology#testChooseRandom fails intermittently. Contributed by Zsolt Venczel.


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

Branch: refs/heads/YARN-6592
Commit: acb92904d04053551d3010937e133f74450043df
Parents: 67662e2
Author: Sean Mackrory <ma...@apache.org>
Authored: Thu Dec 7 10:52:02 2017 -0700
Committer: Sean Mackrory <ma...@apache.org>
Committed: Thu Dec 7 12:30:58 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/net/TestClusterTopology.java  | 58 ++++++++++++--------
 1 file changed, 35 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/acb92904/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java
index 09f0914..6aad6c5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java
@@ -139,38 +139,50 @@ public class TestClusterTopology extends Assert {
     NodeElement node4 = getNewNode("node4", "/d1/r3");
     cluster.add(node4);
 
+    // Number of test runs
+    int numTestRuns = 3;
+    int chiSquareTestRejectedCounter = 0;
+
     // Number of iterations to do the test
     int numIterations = 100;
 
-    // Pick random nodes
-    HashMap<String,Integer> histogram = new HashMap<String,Integer>();
-    for (int i=0; i<numIterations; i++) {
-      String randomNode = cluster.chooseRandom(NodeBase.ROOT).getName();
-      if (!histogram.containsKey(randomNode)) {
-        histogram.put(randomNode, 0);
+    for (int testRun = 0; testRun < numTestRuns; ++testRun) {
+
+      // Pick random nodes
+      HashMap<String, Integer> histogram = new HashMap<String, Integer>();
+      for (int i = 0; i < numIterations; i++) {
+        String randomNode = cluster.chooseRandom(NodeBase.ROOT).getName();
+        if (!histogram.containsKey(randomNode)) {
+          histogram.put(randomNode, 0);
+        }
+        histogram.put(randomNode, histogram.get(randomNode) + 1);
+      }
+      assertEquals("Random is not selecting all nodes", 4, histogram.size());
+
+      // Check with 99% confidence alpha=0.01 as confidence = 100 * (1 - alpha)
+      ChiSquareTest chiSquareTest = new ChiSquareTest();
+      double[] expected = new double[histogram.size()];
+      long[] observed = new long[histogram.size()];
+      int j = 0;
+      for (Integer occurrence : histogram.values()) {
+        expected[j] = 1.0 * numIterations / histogram.size();
+        observed[j] = occurrence;
+        j++;
+      }
+      boolean chiSquareTestRejected =
+            chiSquareTest.chiSquareTest(expected, observed, 0.01);
+
+      if (chiSquareTestRejected) {
+        ++chiSquareTestRejectedCounter;
       }
-      histogram.put(randomNode, histogram.get(randomNode) + 1);
-    }
-    assertEquals("Random is not selecting all nodes", 4, histogram.size());
-
-    // Check with 99% confidence (alpha=0.01 as confidence = (100 * (1 - alpha)
-    ChiSquareTest chiSquareTest = new ChiSquareTest();
-    double[] expected = new double[histogram.size()];
-    long[] observed = new long[histogram.size()];
-    int j=0;
-    for (Integer occurrence : histogram.values()) {
-      expected[j] = 1.0 * numIterations / histogram.size();
-      observed[j] = occurrence;
-      j++;
     }
-    boolean chiSquareTestRejected =
-        chiSquareTest.chiSquareTest(expected, observed, 0.01);
 
     // Check that they have the proper distribution
-    assertFalse("Not choosing nodes randomly", chiSquareTestRejected);
+    assertFalse("Random not choosing nodes with proper distribution",
+            chiSquareTestRejectedCounter==3);
 
     // Pick random nodes excluding the 2 nodes in /d1/r3
-    histogram = new HashMap<String,Integer>();
+    HashMap<String, Integer> histogram = new HashMap<String, Integer>();
     for (int i=0; i<numIterations; i++) {
       String randomNode = cluster.chooseRandom("~/d1/r3").getName();
       if (!histogram.containsKey(randomNode)) {


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


[16/50] [abbrv] hadoop git commit: HDFS-12840. Creating a file with non-default EC policy in a EC zone is not correctly serialized in the editlog. Contributed by Lei (Eddy) Xu.

Posted by wa...@apache.org.
HDFS-12840. Creating a file with non-default EC policy in a EC zone is not correctly serialized in the editlog. Contributed by Lei (Eddy) Xu.


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

Branch: refs/heads/YARN-6592
Commit: 67662e2ac9e68f32b725c8118cf2be79a662fca5
Parents: 67b2661
Author: Lei Xu <le...@apache.org>
Authored: Thu Dec 7 11:15:40 2017 -0800
Committer: Lei Xu <le...@apache.org>
Committed: Thu Dec 7 11:15:40 2017 -0800

----------------------------------------------------------------------
 .../io/erasurecode/ErasureCodeConstants.java    |   2 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  21 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   3 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   2 +-
 .../hdfs/server/namenode/FSEditLogOp.java       |  26 +
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  14 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  23 +
 .../TestDistributedFileSystemWithECFile.java    |  55 ++
 .../namenode/OfflineEditsViewerHelper.java      |   4 +-
 .../server/namenode/TestNamenodeRetryCache.java |  14 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |  13 +-
 .../hadoop-hdfs/src/test/resources/editsStored  | Bin 6753 -> 7909 bytes
 .../src/test/resources/editsStored.xml          | 536 +++++++++++++------
 13 files changed, 531 insertions(+), 182 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
index 73b8f56..2eac016 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
@@ -52,6 +52,6 @@ public final class ErasureCodeConstants {
 
   public static final byte MAX_POLICY_ID = Byte.MAX_VALUE;
   public static final byte USER_DEFINED_POLICY_START_ID = (byte) 64;
-  public static final byte REPLICATION_POLICY_ID = (byte) 63;
+  public static final byte REPLICATION_POLICY_ID = (byte) 0;
   public static final String REPLICATION_POLICY_NAME = REPLICATION_CODEC_NAME;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index b202212..c4041a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.util.ChunkedArrayList;
@@ -415,22 +416,28 @@ class FSDirWriteFileOp {
       PermissionStatus permissions, List<AclEntry> aclEntries,
       List<XAttr> xAttrs, short replication, long modificationTime, long atime,
       long preferredBlockSize, boolean underConstruction, String clientName,
-      String clientMachine, byte storagePolicyId) {
+      String clientMachine, byte storagePolicyId, byte ecPolicyID) {
     final INodeFile newNode;
     Preconditions.checkNotNull(existing);
     assert fsd.hasWriteLock();
     try {
       // check if the file has an EC policy
-      boolean isStriped = false;
-      ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.
-          unprotectedGetErasureCodingPolicy(fsd.getFSNamesystem(), existing);
-      if (ecPolicy != null) {
-        isStriped = true;
+      boolean isStriped =
+          ecPolicyID != ErasureCodeConstants.REPLICATION_POLICY_ID;
+      ErasureCodingPolicy ecPolicy = null;
+      if (isStriped) {
+        ecPolicy = fsd.getFSNamesystem().getErasureCodingPolicyManager()
+          .getByID(ecPolicyID);
+        if (ecPolicy == null) {
+          throw new IOException(String.format(
+              "Cannot find erasure coding policy for new file %s/%s, " +
+                  "ecPolicyID=%d",
+              existing.getPath(), Arrays.toString(localName), ecPolicyID));
+        }
       }
       final BlockType blockType = isStriped ?
           BlockType.STRIPED : BlockType.CONTIGUOUS;
       final Short replicationFactor = (!isStriped ? replication : null);
-      final Byte ecPolicyID = (isStriped ? ecPolicy.getId() : null);
       if (underConstruction) {
         newNode = newINodeFile(id, permissions, modificationTime,
             modificationTime, replicationFactor, ecPolicyID, preferredBlockSize,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 72e00ee..d6fb212 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -815,7 +815,8 @@ public class FSEditLog implements LogsPurgeable {
       .setClientMachine(
           newNode.getFileUnderConstructionFeature().getClientMachine())
       .setOverwrite(overwrite)
-      .setStoragePolicyId(newNode.getLocalStoragePolicyID());
+      .setStoragePolicyId(newNode.getLocalStoragePolicyID())
+      .setErasureCodingPolicyId(newNode.getErasureCodingPolicyID());
 
     AclFeature f = newNode.getAclFeature();
     if (f != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index a21b8ea..b0fe60a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -385,7 +385,7 @@ public class FSEditLogLoader {
             addCloseOp.xAttrs, replication, addCloseOp.mtime,
             addCloseOp.atime, addCloseOp.blockSize, true,
             addCloseOp.clientName, addCloseOp.clientMachine,
-            addCloseOp.storagePolicyId);
+            addCloseOp.storagePolicyId, addCloseOp.erasureCodingPolicyId);
         assert newFile != null;
         iip = INodesInPath.replace(iip, iip.length() - 1, newFile);
         fsNamesys.leaseManager.addLease(addCloseOp.clientName, newFile.getId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
index 2dc9d33..c0daaf1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
@@ -127,6 +127,7 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
 import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.ipc.ClientId;
 import org.apache.hadoop.ipc.RpcConstants;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -425,10 +426,12 @@ public abstract class FSEditLogOp {
     String clientMachine;
     boolean overwrite;
     byte storagePolicyId;
+    byte erasureCodingPolicyId;
     
     private AddCloseOp(FSEditLogOpCodes opCode) {
       super(opCode);
       storagePolicyId = HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+      erasureCodingPolicyId = ErasureCodeConstants.REPLICATION_POLICY_ID;
       assert(opCode == OP_ADD || opCode == OP_CLOSE || opCode == OP_APPEND);
     }
 
@@ -449,6 +452,7 @@ public abstract class FSEditLogOp {
       clientMachine = null;
       overwrite = false;
       storagePolicyId = 0;
+      erasureCodingPolicyId = ErasureCodeConstants.REPLICATION_POLICY_ID;
     }
 
     <T extends AddCloseOp> T setInodeId(long inodeId) {
@@ -535,6 +539,11 @@ public abstract class FSEditLogOp {
       return (T)this;
     }
 
+    <T extends AddCloseOp> T setErasureCodingPolicyId(byte ecPolicyId) {
+      this.erasureCodingPolicyId = ecPolicyId;
+      return (T)this;
+    }
+
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
       FSImageSerialization.writeLong(inodeId, out);
@@ -555,6 +564,7 @@ public abstract class FSEditLogOp {
         FSImageSerialization.writeString(clientMachine,out);
         FSImageSerialization.writeBoolean(overwrite, out);
         FSImageSerialization.writeByte(storagePolicyId, out);
+        FSImageSerialization.writeByte(erasureCodingPolicyId, out);
         // write clientId and callId
         writeRpcIds(rpcClientId, rpcCallId, out);
       }
@@ -633,6 +643,14 @@ public abstract class FSEditLogOp {
           this.storagePolicyId =
               HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
         }
+
+        if (NameNodeLayoutVersion.supports(
+            NameNodeLayoutVersion.Feature.ERASURE_CODING, logVersion)) {
+          this.erasureCodingPolicyId = FSImageSerialization.readByte(in);
+        } else {
+          this.erasureCodingPolicyId =
+              ErasureCodeConstants.REPLICATION_POLICY_ID;
+        }
         // read clientId and callId
         readRpcIds(in, logVersion);
       } else {
@@ -695,6 +713,8 @@ public abstract class FSEditLogOp {
       }
       builder.append(", storagePolicyId=");
       builder.append(storagePolicyId);
+      builder.append(", erasureCodingPolicyId=");
+      builder.append(erasureCodingPolicyId);
       builder.append(", opCode=");
       builder.append(opCode);
       builder.append(", txid=");
@@ -730,6 +750,8 @@ public abstract class FSEditLogOp {
         if (aclEntries != null) {
           appendAclEntriesToXml(contentHandler, aclEntries);
         }
+        XMLUtils.addSaxString(contentHandler, "ERASURE_CODING_POLICY_ID",
+            Byte.toString(erasureCodingPolicyId));
         appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
       }
     }
@@ -758,6 +780,10 @@ public abstract class FSEditLogOp {
       }
       this.permissions = permissionStatusFromXml(st);
       aclEntries = readAclEntriesFromXml(st);
+      if (st.hasChildren("ERASURE_CODING_POLICY_ID")) {
+        this.erasureCodingPolicyId = Byte.parseByte(st.getValue(
+            "ERASURE_CODING_POLICY_ID"));
+      }
       readRpcIdsFromXml(st);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index fcdb064..3f2fb33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hdfs.util.LongBitFormat;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.util.StringUtils;
 
 /** I-node for closed file. */
@@ -200,9 +201,10 @@ public class INodeFile extends INodeWithAdditionalFields
         // as the PolicyID can never be in negative.
         layoutRedundancy |= erasureCodingPolicyID;
       } else {
-        Preconditions.checkArgument(replication != null &&
-            erasureCodingPolicyID == null);
-        Preconditions.checkArgument(replication >= 0 &&
+        Preconditions.checkArgument(erasureCodingPolicyID == null ||
+                erasureCodingPolicyID ==
+                    ErasureCodeConstants.REPLICATION_POLICY_ID);
+        Preconditions.checkArgument(replication != null && replication >= 0 &&
             replication <= MAX_REDUNDANCY,
             "Invalid replication value " + replication);
         layoutRedundancy |= replication;
@@ -588,10 +590,8 @@ public class INodeFile extends INodeWithAdditionalFields
     setStoragePolicyID(storagePolicyId);
   }
 
-
   /**
-   * @return The ID of the erasure coding policy on the file. -1 represents no
-   *          EC policy.
+   * @return The ID of the erasure coding policy on the file.
    */
   @VisibleForTesting
   @Override
@@ -599,7 +599,7 @@ public class INodeFile extends INodeWithAdditionalFields
     if (isStriped()) {
       return HeaderFormat.getECPolicyID(header);
     }
-    return -1;
+    return ErasureCodeConstants.REPLICATION_POLICY_ID;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 1048b0a..1411a7f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -1510,6 +1510,29 @@ public class DFSTestUtil {
     // OP_REMOVE_ERASURE_CODING_POLICY
     filesystem.removeErasureCodingPolicy(newPolicy1.getName());
     filesystem.removeErasureCodingPolicy(newPolicy2.getName());
+
+    // OP_ADD on erasure coding directory
+    Path ecDir = new Path("/ec");
+    filesystem.mkdirs(ecDir);
+    final ErasureCodingPolicy defaultEcPolicy =
+        SystemErasureCodingPolicies.getByID(
+            SystemErasureCodingPolicies.RS_6_3_POLICY_ID);
+    final ErasureCodingPolicy ecPolicyRS32 =
+        SystemErasureCodingPolicies.getByID(
+            SystemErasureCodingPolicies.RS_3_2_POLICY_ID);
+    filesystem.enableErasureCodingPolicy(ecPolicyRS32.getName());
+    filesystem.enableErasureCodingPolicy(defaultEcPolicy.getName());
+    filesystem.setErasureCodingPolicy(ecDir, defaultEcPolicy.getName());
+
+    try (FSDataOutputStream out = filesystem.createFile(
+        new Path(ecDir, "replicated")).replicate().build()) {
+      out.write("replicated".getBytes());
+    }
+
+    try (FSDataOutputStream out = filesystem.createFile(
+        new Path(ecDir, "RS-3-2")).ecPolicyName(ecPolicyRS32.getName()).build()) {
+      out.write("RS-3-2".getBytes());
+    }
   }
 
   public static void abortStream(DFSOutputStream out) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java
index 14a2ec4..0a3010f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java
@@ -19,11 +19,13 @@ package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -32,6 +34,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -194,4 +198,55 @@ public class TestDistributedFileSystemWithECFile {
     assertTrue(lastBlock.getOffset() == blockGroupSize);
     assertTrue(lastBlock.getLength() == lastBlockSize);
   }
+
+  @Test(timeout=60000)
+  public void testReplayEditLogsForReplicatedFile() throws Exception {
+    cluster.shutdown();
+
+    ErasureCodingPolicy rs63 = SystemErasureCodingPolicies.getByID(
+        SystemErasureCodingPolicies.RS_6_3_POLICY_ID
+    );
+    ErasureCodingPolicy rs32 = SystemErasureCodingPolicies.getByID(
+        SystemErasureCodingPolicies.RS_3_2_POLICY_ID
+    );
+    // Test RS(6,3) as default policy
+    int numDataNodes = rs63.getNumDataUnits() + rs63.getNumParityUnits();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(numDataNodes)
+        .build();
+
+    cluster.transitionToActive(0);
+    fs = cluster.getFileSystem(0);
+    fs.enableErasureCodingPolicy(rs63.getName());
+    fs.enableErasureCodingPolicy(rs32.getName());
+
+    Path dir = new Path("/ec");
+    fs.mkdirs(dir);
+    fs.setErasureCodingPolicy(dir, rs63.getName());
+
+    // Create an erasure coded file with the default policy.
+    Path ecFile = new Path(dir, "ecFile");
+    createFile(ecFile.toString(), 10);
+    // Create a replicated file.
+    Path replicatedFile = new Path(dir, "replicated");
+    try (FSDataOutputStream out = fs.createFile(replicatedFile)
+      .replicate().build()) {
+      out.write(123);
+    }
+    // Create an EC file with a different policy.
+    Path ecFile2 = new Path(dir, "RS-3-2");
+    try (FSDataOutputStream out = fs.createFile(ecFile2)
+         .ecPolicyName(rs32.getName()).build()) {
+      out.write(456);
+    }
+
+    cluster.transitionToStandby(0);
+    cluster.transitionToActive(1);
+
+    fs = cluster.getFileSystem(1);
+    assertNull(fs.getErasureCodingPolicy(replicatedFile));
+    assertEquals(rs63, fs.getErasureCodingPolicy(ecFile));
+    assertEquals(rs32, fs.getErasureCodingPolicy(ecFile2));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
index b5ba108..d637af5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
@@ -98,8 +98,10 @@ public class OfflineEditsViewerHelper {
     config.setBoolean(
         DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     config.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    final int numDataNodes = 9;
     cluster =
-      new MiniDFSCluster.Builder(config).manageNameDfsDirs(false).build();
+      new MiniDFSCluster.Builder(config).manageNameDfsDirs(false)
+          .numDataNodes(numDataNodes).build();
     cluster.waitClusterUp();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
index 42ff698..0995f13 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
@@ -43,9 +43,11 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.ipc.ClientId;
@@ -79,6 +81,11 @@ import org.junit.Test;
 public class TestNamenodeRetryCache {
   private static final byte[] CLIENT_ID = ClientId.getClientId();
   private static MiniDFSCluster cluster;
+  private static ErasureCodingPolicy defaultEcPolicy =
+      SystemErasureCodingPolicies.getByID(
+          SystemErasureCodingPolicies.RS_6_3_POLICY_ID);
+  private static int numDataNodes = defaultEcPolicy.getNumDataUnits() +
+      defaultEcPolicy.getNumParityUnits() + 1;
   private static NamenodeProtocols nnRpc;
   private static final FsPermission perm = FsPermission.getDefault();
   private static DistributedFileSystem filesystem;
@@ -93,7 +100,8 @@ public class TestNamenodeRetryCache {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BlockSize);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
-    cluster = new MiniDFSCluster.Builder(conf).build();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numDataNodes).build();
     cluster.waitActive();
     nnRpc = cluster.getNameNode().getRpcServer();
     filesystem = cluster.getFileSystem();
@@ -436,7 +444,7 @@ public class TestNamenodeRetryCache {
 
     LightWeightCache<CacheEntry, CacheEntry> cacheSet = 
         (LightWeightCache<CacheEntry, CacheEntry>) namesystem.getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 39, cacheSet.size());
     
     Map<CacheEntry, CacheEntry> oldEntries = 
         new HashMap<CacheEntry, CacheEntry>();
@@ -455,7 +463,7 @@ public class TestNamenodeRetryCache {
     assertTrue(namesystem.hasRetryCache());
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 39, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index 1d114d6..43aaa92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -67,11 +67,13 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -93,7 +95,12 @@ public class TestRetryCacheWithHA {
   private static final Log LOG = LogFactory.getLog(TestRetryCacheWithHA.class);
   
   private static final int BlockSize = 1024;
-  private static final short DataNodes = 3;
+  private static ErasureCodingPolicy defaultEcPolicy =
+      SystemErasureCodingPolicies.getByID(
+          SystemErasureCodingPolicies.RS_6_3_POLICY_ID);
+  private static final short DataNodes = (short)(
+      defaultEcPolicy.getNumDataUnits() +
+      defaultEcPolicy.getNumParityUnits() + 1);
   private static final int CHECKTIMES = 10;
   private static final int ResponseSize = 3;
   
@@ -166,7 +173,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn0 = cluster.getNamesystem(0);
     LightWeightCache<CacheEntry, CacheEntry> cacheSet = 
         (LightWeightCache<CacheEntry, CacheEntry>) fsn0.getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 39, cacheSet.size());
     
     Map<CacheEntry, CacheEntry> oldEntries = 
         new HashMap<CacheEntry, CacheEntry>();
@@ -187,7 +194,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn1 = cluster.getNamesystem(1);
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) fsn1
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 39, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
index 3f2817a..a0ae78e 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67662e2a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
index 2a57c73..7e1881c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
@@ -13,8 +13,8 @@
       <TXID>2</TXID>
       <DELEGATION_KEY>
         <KEY_ID>1</KEY_ID>
-        <EXPIRY_DATE>1512000829976</EXPIRY_DATE>
-        <KEY>e7457bcc6ab95a84</KEY>
+        <EXPIRY_DATE>1513298395825</EXPIRY_DATE>
+        <KEY>ddb3d2c37b57926a</KEY>
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -24,8 +24,8 @@
       <TXID>3</TXID>
       <DELEGATION_KEY>
         <KEY_ID>2</KEY_ID>
-        <EXPIRY_DATE>1512000829980</EXPIRY_DATE>
-        <KEY>07cc38caf6c47bb4</KEY>
+        <EXPIRY_DATE>1513298395827</EXPIRY_DATE>
+        <KEY>57acfb80c8b539fa</KEY>
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -36,11 +36,11 @@
       <LENGTH>0</LENGTH>
       <INODEID>16386</INODEID>
       <PATH>/file_create</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632199</MTIME>
-      <ATIME>1511309632199</ATIME>
+      <REPLICATION>3</REPLICATION>
+      <MTIME>1512607197452</MTIME>
+      <ATIME>1512607197452</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
@@ -48,8 +48,9 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>5</RPC_CALLID>
+      <ERASURE_CODING_POLICY_ID>0</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>35</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -59,9 +60,9 @@
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632248</MTIME>
-      <ATIME>1511309632199</ATIME>
+      <REPLICATION>3</REPLICATION>
+      <MTIME>1512607197500</MTIME>
+      <ATIME>1512607197452</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME/>
       <CLIENT_MACHINE/>
@@ -78,11 +79,11 @@
     <DATA>
       <TXID>6</TXID>
       <PATH>/file_create</PATH>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <NEWBLOCK>false</NEWBLOCK>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>7</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>37</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -92,9 +93,9 @@
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632263</MTIME>
-      <ATIME>1511309632199</ATIME>
+      <REPLICATION>3</REPLICATION>
+      <MTIME>1512607197516</MTIME>
+      <ATIME>1512607197452</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME/>
       <CLIENT_MACHINE/>
@@ -114,10 +115,10 @@
       <INODEID>16387</INODEID>
       <PATH>/update_blocks</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632266</MTIME>
-      <ATIME>1511309632266</ATIME>
+      <MTIME>1512607197519</MTIME>
+      <ATIME>1512607197519</ATIME>
       <BLOCKSIZE>4096</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
@@ -125,8 +126,9 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>9</RPC_CALLID>
+      <ERASURE_CODING_POLICY_ID>0</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>39</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -188,8 +190,8 @@
       <INODEID>0</INODEID>
       <PATH>/update_blocks</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632454</MTIME>
-      <ATIME>1511309632266</ATIME>
+      <MTIME>1512607197657</MTIME>
+      <ATIME>1512607197519</ATIME>
       <BLOCKSIZE>4096</BLOCKSIZE>
       <CLIENT_NAME/>
       <CLIENT_MACHINE/>
@@ -216,9 +218,9 @@
       <LENGTH>0</LENGTH>
       <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1511309632467</TIMESTAMP>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>17</RPC_CALLID>
+      <TIMESTAMP>1512607197671</TIMESTAMP>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>47</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -227,9 +229,9 @@
       <TXID>17</TXID>
       <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
-      <TIMESTAMP>1511309632480</TIMESTAMP>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>19</RPC_CALLID>
+      <TIMESTAMP>1512607197680</TIMESTAMP>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>49</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -239,7 +241,7 @@
       <LENGTH>0</LENGTH>
       <INODEID>16388</INODEID>
       <PATH>/directory_mkdir</PATH>
-      <TIMESTAMP>1511309632488</TIMESTAMP>
+      <TIMESTAMP>1512607197690</TIMESTAMP>
       <PERMISSION_STATUS>
         <USERNAME>lei</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -274,8 +276,8 @@
       <TXID>22</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>24</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>54</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -285,8 +287,8 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>25</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>55</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -295,8 +297,8 @@
       <TXID>24</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>26</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>56</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -306,11 +308,11 @@
       <LENGTH>0</LENGTH>
       <INODEID>16389</INODEID>
       <PATH>/file_create</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632528</MTIME>
-      <ATIME>1511309632528</ATIME>
+      <REPLICATION>3</REPLICATION>
+      <MTIME>1512607197723</MTIME>
+      <ATIME>1512607197723</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
@@ -318,8 +320,9 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>27</RPC_CALLID>
+      <ERASURE_CODING_POLICY_ID>0</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>57</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -329,9 +332,9 @@
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632530</MTIME>
-      <ATIME>1511309632528</ATIME>
+      <REPLICATION>3</REPLICATION>
+      <MTIME>1512607197726</MTIME>
+      <ATIME>1512607197723</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME/>
       <CLIENT_MACHINE/>
@@ -402,10 +405,10 @@
       <LENGTH>0</LENGTH>
       <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1511309632561</TIMESTAMP>
+      <TIMESTAMP>1512607197754</TIMESTAMP>
       <OPTIONS>TO_TRASH</OPTIONS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>35</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>65</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -416,10 +419,10 @@
       <INODEID>16390</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632566</MTIME>
-      <ATIME>1511309632566</ATIME>
+      <MTIME>1512607197759</MTIME>
+      <ATIME>1512607197759</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
@@ -427,8 +430,9 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>37</RPC_CALLID>
+      <ERASURE_CODING_POLICY_ID>0</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>67</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -533,8 +537,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632593</MTIME>
-      <ATIME>1511309632566</ATIME>
+      <MTIME>1512607197800</MTIME>
+      <ATIME>1512607197759</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME/>
       <CLIENT_MACHINE/>
@@ -569,10 +573,10 @@
       <INODEID>16391</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632596</MTIME>
-      <ATIME>1511309632596</ATIME>
+      <MTIME>1512607197803</MTIME>
+      <ATIME>1512607197803</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
@@ -580,8 +584,9 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>46</RPC_CALLID>
+      <ERASURE_CODING_POLICY_ID>0</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>76</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -686,8 +691,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632618</MTIME>
-      <ATIME>1511309632596</ATIME>
+      <MTIME>1512607197837</MTIME>
+      <ATIME>1512607197803</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME/>
       <CLIENT_MACHINE/>
@@ -722,10 +727,10 @@
       <INODEID>16392</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632620</MTIME>
-      <ATIME>1511309632620</ATIME>
+      <MTIME>1512607197839</MTIME>
+      <ATIME>1512607197839</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
@@ -733,8 +738,9 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>55</RPC_CALLID>
+      <ERASURE_CODING_POLICY_ID>0</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>85</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -839,8 +845,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632643</MTIME>
-      <ATIME>1511309632620</ATIME>
+      <MTIME>1512607197878</MTIME>
+      <ATIME>1512607197839</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME/>
       <CLIENT_MACHINE/>
@@ -873,13 +879,13 @@
       <TXID>67</TXID>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
-      <TIMESTAMP>1511309632648</TIMESTAMP>
+      <TIMESTAMP>1512607197882</TIMESTAMP>
       <SOURCES>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE2>/file_concat_1</SOURCE2>
       </SOURCES>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>63</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>93</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -890,10 +896,10 @@
       <INODEID>16393</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632651</MTIME>
-      <ATIME>1511309632651</ATIME>
+      <MTIME>1512607197885</MTIME>
+      <ATIME>1512607197885</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
@@ -901,8 +907,9 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>65</RPC_CALLID>
+      <ERASURE_CODING_POLICY_ID>0</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>95</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -974,8 +981,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632667</MTIME>
-      <ATIME>1511309632651</ATIME>
+      <MTIME>1512607197909</MTIME>
+      <ATIME>1512607197885</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME/>
       <CLIENT_MACHINE/>
@@ -1002,10 +1009,10 @@
     <DATA>
       <TXID>76</TXID>
       <SRC>/file_create</SRC>
-      <CLIENTNAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENTNAME>
+      <CLIENTNAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENTNAME>
       <CLIENTMACHINE>127.0.0.1</CLIENTMACHINE>
       <NEWLENGTH>512</NEWLENGTH>
-      <TIMESTAMP>1511309632671</TIMESTAMP>
+      <TIMESTAMP>1512607197912</TIMESTAMP>
     </DATA>
   </RECORD>
   <RECORD>
@@ -1016,15 +1023,15 @@
       <INODEID>16394</INODEID>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
-      <MTIME>1511309632686</MTIME>
-      <ATIME>1511309632686</ATIME>
+      <MTIME>1512607197921</MTIME>
+      <ATIME>1512607197921</ATIME>
       <PERMISSION_STATUS>
         <USERNAME>lei</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>511</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>72</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>102</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -1034,11 +1041,11 @@
       <LENGTH>0</LENGTH>
       <INODEID>16395</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1511309632689</MTIME>
-      <ATIME>1511309632689</ATIME>
+      <REPLICATION>3</REPLICATION>
+      <MTIME>1512607197925</MTIME>
+      <ATIME>1512607197925</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_2134933941_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <OVERWRITE>true</OVERWRITE>
       <PERMISSION_STATUS>
@@ -1046,8 +1053,9 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>73</RPC_CALLID>
+      <ERASURE_CODING_POLICY_ID>0</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>103</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
@@ -1103,21 +1111,30 @@
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
       <TXID>84</TXID>
-      <LEASEHOLDER>DFSClient_NONMAPREDUCE_2134933941_1</LEASEHOLDER>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-923924783_1</LEASEHOLDER>
       <PATH>/hard-lease-recovery-test</PATH>
-      <NEWHOLDER>HDFS_NameNode-2017-11-21 16:13:54,700-0800</NEWHOLDER>
+      <NEWHOLDER>HDFS_NameNode-2017-12-06 16:39:59,951-0800</NEWHOLDER>
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_CLOSE</OPCODE>
+    <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
       <TXID>85</TXID>
+      <LEASEHOLDER>HDFS_NameNode-2017-12-06 16:39:59,951-0800</LEASEHOLDER>
+      <PATH>/hard-lease-recovery-test</PATH>
+      <NEWHOLDER>HDFS_NameNode-2017-12-06 16:40:01,959-0800</NEWHOLDER>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_CLOSE</OPCODE>
+    <DATA>
+      <TXID>86</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1511309634880</MTIME>
-      <ATIME>1511309632689</ATIME>
+      <REPLICATION>3</REPLICATION>
+      <MTIME>1512607202974</MTIME>
+      <ATIME>1512607197925</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME/>
       <CLIENT_MACHINE/>
@@ -1137,7 +1154,7 @@
   <RECORD>
     <OPCODE>OP_ADD_CACHE_POOL</OPCODE>
     <DATA>
-      <TXID>86</TXID>
+      <TXID>87</TXID>
       <POOLNAME>pool1</POOLNAME>
       <OWNERNAME>lei</OWNERNAME>
       <GROUPNAME>staff</GROUPNAME>
@@ -1145,65 +1162,65 @@
       <LIMIT>9223372036854775807</LIMIT>
       <MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY>
       <DEFAULTREPLICATION>1</DEFAULTREPLICATION>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>80</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>138</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_MODIFY_CACHE_POOL</OPCODE>
     <DATA>
-      <TXID>87</TXID>
+      <TXID>88</TXID>
       <POOLNAME>pool1</POOLNAME>
       <LIMIT>99</LIMIT>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>81</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>139</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_CACHE_DIRECTIVE</OPCODE>
     <DATA>
-      <TXID>88</TXID>
+      <TXID>89</TXID>
       <ID>1</ID>
       <PATH>/path</PATH>
       <REPLICATION>1</REPLICATION>
       <POOL>pool1</POOL>
-      <EXPIRATION>2305844520523329692</EXPIRATION>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>82</RPC_CALLID>
+      <EXPIRATION>2305844521820897941</EXPIRATION>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>140</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_MODIFY_CACHE_DIRECTIVE</OPCODE>
     <DATA>
-      <TXID>89</TXID>
+      <TXID>90</TXID>
       <ID>1</ID>
       <REPLICATION>2</REPLICATION>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>83</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>141</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REMOVE_CACHE_DIRECTIVE</OPCODE>
     <DATA>
-      <TXID>90</TXID>
+      <TXID>91</TXID>
       <ID>1</ID>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>84</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>142</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REMOVE_CACHE_POOL</OPCODE>
     <DATA>
-      <TXID>91</TXID>
+      <TXID>92</TXID>
       <POOLNAME>pool1</POOLNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>85</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>143</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_ACL</OPCODE>
     <DATA>
-      <TXID>92</TXID>
+      <TXID>93</TXID>
       <SRC>/file_concat_target</SRC>
       <ENTRY>
         <SCOPE>ACCESS</SCOPE>
@@ -1236,61 +1253,61 @@
   <RECORD>
     <OPCODE>OP_SET_XATTR</OPCODE>
     <DATA>
-      <TXID>93</TXID>
+      <TXID>94</TXID>
       <SRC>/file_concat_target</SRC>
       <XATTR>
         <NAMESPACE>USER</NAMESPACE>
         <NAME>a1</NAME>
         <VALUE>0x313233</VALUE>
       </XATTR>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>87</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>145</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_XATTR</OPCODE>
     <DATA>
-      <TXID>94</TXID>
+      <TXID>95</TXID>
       <SRC>/file_concat_target</SRC>
       <XATTR>
         <NAMESPACE>USER</NAMESPACE>
         <NAME>a2</NAME>
         <VALUE>0x373839</VALUE>
       </XATTR>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>88</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>146</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REMOVE_XATTR</OPCODE>
     <DATA>
-      <TXID>95</TXID>
+      <TXID>96</TXID>
       <SRC>/file_concat_target</SRC>
       <XATTR>
         <NAMESPACE>USER</NAMESPACE>
         <NAME>a2</NAME>
       </XATTR>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>89</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>147</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_ERASURE_CODING_POLICY</OPCODE>
     <DATA>
-      <TXID>96</TXID>
+      <TXID>97</TXID>
       <CODEC>rs</CODEC>
       <DATAUNITS>3</DATAUNITS>
       <PARITYUNITS>2</PARITYUNITS>
       <CELLSIZE>8192</CELLSIZE>
       <EXTRAOPTIONS>0</EXTRAOPTIONS>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>90</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>148</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_ERASURE_CODING_POLICY</OPCODE>
     <DATA>
-      <TXID>97</TXID>
+      <TXID>98</TXID>
       <CODEC>rs</CODEC>
       <DATAUNITS>6</DATAUNITS>
       <PARITYUNITS>10</PARITYUNITS>
@@ -1300,82 +1317,285 @@
         <KEY>dummyKey</KEY>
         <VALUE>dummyValue</VALUE>
       </EXTRAOPTION>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>91</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>149</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ENABLE_ERASURE_CODING_POLICY</OPCODE>
     <DATA>
-      <TXID>98</TXID>
+      <TXID>99</TXID>
       <POLICYNAME>RS-3-2-8k</POLICYNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>92</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>150</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ENABLE_ERASURE_CODING_POLICY</OPCODE>
     <DATA>
-      <TXID>99</TXID>
+      <TXID>100</TXID>
       <POLICYNAME>RS-6-10-4k</POLICYNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>93</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>151</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_DISABLE_ERASURE_CODING_POLICY</OPCODE>
     <DATA>
-      <TXID>100</TXID>
+      <TXID>101</TXID>
       <POLICYNAME>RS-3-2-8k</POLICYNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>94</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>152</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_DISABLE_ERASURE_CODING_POLICY</OPCODE>
     <DATA>
-      <TXID>101</TXID>
+      <TXID>102</TXID>
       <POLICYNAME>RS-6-10-4k</POLICYNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>95</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>153</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REMOVE_ERASURE_CODING_POLICY</OPCODE>
     <DATA>
-      <TXID>102</TXID>
+      <TXID>103</TXID>
       <POLICYNAME>RS-3-2-8k</POLICYNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>96</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>154</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REMOVE_ERASURE_CODING_POLICY</OPCODE>
     <DATA>
-      <TXID>103</TXID>
+      <TXID>104</TXID>
       <POLICYNAME>RS-6-10-4k</POLICYNAME>
-      <RPC_CLIENTID>a4dc081c-6d6f-42d6-af5b-d260228f1aad</RPC_CLIENTID>
-      <RPC_CALLID>97</RPC_CALLID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>155</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_MKDIR</OPCODE>
+    <DATA>
+      <TXID>105</TXID>
+      <LENGTH>0</LENGTH>
+      <INODEID>16396</INODEID>
+      <PATH>/ec</PATH>
+      <TIMESTAMP>1512607204077</TIMESTAMP>
+      <PERMISSION_STATUS>
+        <USERNAME>lei</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <MODE>493</MODE>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ENABLE_ERASURE_CODING_POLICY</OPCODE>
+    <DATA>
+      <TXID>106</TXID>
+      <POLICYNAME>RS-3-2-1024k</POLICYNAME>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>157</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ENABLE_ERASURE_CODING_POLICY</OPCODE>
+    <DATA>
+      <TXID>107</TXID>
+      <POLICYNAME>RS-6-3-1024k</POLICYNAME>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>158</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_SET_XATTR</OPCODE>
+    <DATA>
+      <TXID>108</TXID>
+      <SRC>/ec</SRC>
+      <XATTR>
+        <NAMESPACE>SYSTEM</NAMESPACE>
+        <NAME>hdfs.erasurecoding.policy</NAME>
+        <VALUE>0x0000000c52532d362d332d313032346b</VALUE>
+      </XATTR>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>159</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ADD</OPCODE>
+    <DATA>
+      <TXID>109</TXID>
+      <LENGTH>0</LENGTH>
+      <INODEID>16397</INODEID>
+      <PATH>/ec/replicated</PATH>
+      <REPLICATION>3</REPLICATION>
+      <MTIME>1512607204088</MTIME>
+      <ATIME>1512607204088</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <OVERWRITE>true</OVERWRITE>
+      <PERMISSION_STATUS>
+        <USERNAME>lei</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <MODE>420</MODE>
+      </PERMISSION_STATUS>
+      <ERASURE_CODING_POLICY_ID>0</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>160</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
+    <DATA>
+      <TXID>110</TXID>
+      <BLOCK_ID>1073741838</BLOCK_ID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
+    <DATA>
+      <TXID>111</TXID>
+      <GENSTAMPV2>1015</GENSTAMPV2>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
+    <DATA>
+      <TXID>112</TXID>
+      <PATH>/ec/replicated</PATH>
+      <BLOCK>
+        <BLOCK_ID>1073741838</BLOCK_ID>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENSTAMP>1015</GENSTAMP>
+      </BLOCK>
+      <RPC_CLIENTID/>
+      <RPC_CALLID>-2</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_CLOSE</OPCODE>
+    <DATA>
+      <TXID>113</TXID>
+      <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
+      <PATH>/ec/replicated</PATH>
+      <REPLICATION>3</REPLICATION>
+      <MTIME>1512607204118</MTIME>
+      <ATIME>1512607204088</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
+      <OVERWRITE>false</OVERWRITE>
+      <BLOCK>
+        <BLOCK_ID>1073741838</BLOCK_ID>
+        <NUM_BYTES>10</NUM_BYTES>
+        <GENSTAMP>1015</GENSTAMP>
+      </BLOCK>
+      <PERMISSION_STATUS>
+        <USERNAME>lei</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <MODE>420</MODE>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ADD</OPCODE>
+    <DATA>
+      <TXID>114</TXID>
+      <LENGTH>0</LENGTH>
+      <INODEID>16398</INODEID>
+      <PATH>/ec/RS-3-2</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1512607204120</MTIME>
+      <ATIME>1512607204120</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-923924783_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <OVERWRITE>true</OVERWRITE>
+      <PERMISSION_STATUS>
+        <USERNAME>lei</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <MODE>420</MODE>
+      </PERMISSION_STATUS>
+      <ERASURE_CODING_POLICY_ID>2</ERASURE_CODING_POLICY_ID>
+      <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
+      <RPC_CALLID>166</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
+    <DATA>
+      <TXID>115</TXID>
+      <BLOCK_ID>-9223372036854775792</BLOCK_ID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
+    <DATA>
+      <TXID>116</TXID>
+      <GENSTAMPV2>1016</GENSTAMPV2>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
+    <DATA>
+      <TXID>117</TXID>
+      <PATH>/ec/RS-3-2</PATH>
+      <BLOCK>
+        <BLOCK_ID>-9223372036854775792</BLOCK_ID>
+        <NUM_BYTES>0</NUM_BYTES>
+        <GENSTAMP>1016</GENSTAMP>
+      </BLOCK>
+      <RPC_CLIENTID/>
+      <RPC_CALLID>-2</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_CLOSE</OPCODE>
+    <DATA>
+      <TXID>118</TXID>
+      <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
+      <PATH>/ec/RS-3-2</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1512607204229</MTIME>
+      <ATIME>1512607204120</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
+      <OVERWRITE>false</OVERWRITE>
+      <BLOCK>
+        <BLOCK_ID>-9223372036854775792</BLOCK_ID>
+        <NUM_BYTES>6</NUM_BYTES>
+        <GENSTAMP>1016</GENSTAMP>
+      </BLOCK>
+      <PERMISSION_STATUS>
+        <USERNAME>lei</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <MODE>420</MODE>
+      </PERMISSION_STATUS>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ROLLING_UPGRADE_START</OPCODE>
     <DATA>
-      <TXID>104</TXID>
-      <STARTTIME>1511309635904</STARTTIME>
+      <TXID>119</TXID>
+      <STARTTIME>1512607204230</STARTTIME>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ROLLING_UPGRADE_FINALIZE</OPCODE>
     <DATA>
-      <TXID>105</TXID>
-      <FINALIZETIME>1511309635904</FINALIZETIME>
+      <TXID>120</TXID>
+      <FINALIZETIME>1512607204233</FINALIZETIME>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_END_LOG_SEGMENT</OPCODE>
     <DATA>
-      <TXID>106</TXID>
+      <TXID>121</TXID>
     </DATA>
   </RECORD>
 </EDITS>


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


[30/50] [abbrv] hadoop git commit: YARN-7575. NPE in scheduler UI when max-capacity is not configured. Contributed by Sunil G.

Posted by wa...@apache.org.
YARN-7575. NPE in scheduler UI when max-capacity is not configured. Contributed by Sunil G.


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

Branch: refs/heads/YARN-6592
Commit: 4db4a4a165d3936d59f5e67947c6bfbc9c3270a5
Parents: daa1cdd
Author: Sunil G <su...@apache.org>
Authored: Wed Nov 29 14:50:54 2017 +0530
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../webapp/CapacitySchedulerPage.java           |   9 +-
 .../TestAbsoluteResourceConfiguration.java      | 116 +++++++++++++++++++
 2 files changed, 121 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4db4a4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 3f68e07..4933d34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -161,10 +161,11 @@ class CapacitySchedulerPage extends RmView {
           .__("Configured Capacity:",
               capacities.getConfiguredMinResource().toString())
           .__("Configured Max Capacity:",
-              capacities.getConfiguredMaxResource().getResource()
-                  .equals(Resources.none())
-                      ? "unlimited"
-                      : capacities.getConfiguredMaxResource().toString())
+              (capacities.getConfiguredMaxResource() == null
+                  || capacities.getConfiguredMaxResource().getResource()
+                      .equals(Resources.none()))
+                          ? "unlimited"
+                          : capacities.getConfiguredMaxResource().toString())
           .__("Effective Capacity:",
               appendPercent(capacities.getEffectiveMinResource().toString(),
                   capacities.getCapacity() / 100))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4db4a4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.java
index 80d16bb..5c4a12e 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/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.java
@@ -511,4 +511,120 @@ public class TestAbsoluteResourceConfiguration {
 
     rm.stop();
   }
+
+  @Test
+  public void testEffectiveResourceAfterIncreasinClusterResource()
+      throws Exception {
+    // create conf with basic queue configuration.
+    CapacitySchedulerConfiguration csConf = setupComplexQueueConfiguration(
+        false);
+    setupComplexMinMaxResourceConfig(csConf);
+
+    csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    @SuppressWarnings("resource")
+    MockRM rm = new MockRM(csConf);
+    rm.start();
+
+    // Add few nodes
+    rm.registerNode("127.0.0.1:1234", 125 * GB, 20);
+    rm.registerNode("127.0.0.2:1234", 125 * GB, 20);
+
+    // Get queue object to verify min/max resource configuration.
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    ParentQueue qA = (ParentQueue) cs.getQueue(QUEUEA);
+    Assert.assertNotNull(qA);
+    Assert.assertEquals("Min resource configured for QUEUEA is not correct",
+        QUEUE_A_MINRES, qA.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEA is not correct",
+        QUEUE_A_MAXRES, qA.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEA is not correct",
+        QUEUE_A_MINRES, qA.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA is not correct",
+        QUEUE_A_MAXRES, qA.queueResourceQuotas.getEffectiveMaxResource());
+
+    ParentQueue qB = (ParentQueue) cs.getQueue(QUEUEB);
+    Assert.assertNotNull(qB);
+    Assert.assertEquals("Min resource configured for QUEUEB is not correct",
+        QUEUE_B_MINRES, qB.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEB is not correct",
+        QUEUE_B_MAXRES, qB.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEB is not correct",
+        QUEUE_B_MINRES, qB.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEB is not correct",
+        QUEUE_B_MAXRES, qB.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qC = (LeafQueue) cs.getQueue(QUEUEC);
+    Assert.assertNotNull(qC);
+    Assert.assertEquals("Min resource configured for QUEUEC is not correct",
+        QUEUE_C_MINRES, qC.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEC is not correct",
+        QUEUE_C_MAXRES, qC.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEC is not correct",
+        QUEUE_C_MINRES, qC.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEC is not correct",
+        QUEUE_C_MAXRES, qC.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qA1 = (LeafQueue) cs.getQueue(QUEUEA1);
+    Assert.assertEquals("Effective Min resource for QUEUEA1 is not correct",
+        QUEUE_A1_MINRES, qA1.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA1 is not correct",
+        QUEUE_A_MAXRES, qA1.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qA2 = (LeafQueue) cs.getQueue(QUEUEA2);
+    Assert.assertEquals("Effective Min resource for QUEUEA2 is not correct",
+        QUEUE_A2_MINRES, qA2.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA2 is not correct",
+        QUEUE_A_MAXRES, qA2.queueResourceQuotas.getEffectiveMaxResource());
+
+    LeafQueue qB1 = (LeafQueue) cs.getQueue(QUEUEB1);
+    Assert.assertEquals("Min resource configured for QUEUEB1 is not correct",
+        QUEUE_B1_MINRES, qB1.queueResourceQuotas.getConfiguredMinResource());
+    Assert.assertEquals("Max resource configured for QUEUEB1 is not correct",
+        QUEUE_B_MAXRES, qB1.queueResourceQuotas.getConfiguredMaxResource());
+    Assert.assertEquals("Effective Min resource for QUEUEB1 is not correct",
+        QUEUE_B1_MINRES, qB1.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEB1 is not correct",
+        QUEUE_B_MAXRES, qB1.queueResourceQuotas.getEffectiveMaxResource());
+
+    // add new NM.
+    rm.registerNode("127.0.0.3:1234", 125 * GB, 20);
+
+    // There will be no change in effective resource when nodes are added.
+    // Since configured capacity was based on initial node capacity, a
+    // re configurations is needed to use this added capacity.
+    Assert.assertEquals("Effective Min resource for QUEUEA is not correct",
+        QUEUE_A_MINRES, qA.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA is not correct",
+        QUEUE_A_MAXRES, qA.queueResourceQuotas.getEffectiveMaxResource());
+
+    Assert.assertEquals("Effective Min resource for QUEUEB is not correct",
+        QUEUE_B_MINRES, qB.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEB is not correct",
+        QUEUE_B_MAXRES, qB.queueResourceQuotas.getEffectiveMaxResource());
+
+    Assert.assertEquals("Effective Min resource for QUEUEC is not correct",
+        QUEUE_C_MINRES, qC.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEC is not correct",
+        QUEUE_C_MAXRES, qC.queueResourceQuotas.getEffectiveMaxResource());
+
+    Assert.assertEquals("Effective Min resource for QUEUEB1 is not correct",
+        QUEUE_B1_MINRES, qB1.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEB1 is not correct",
+        QUEUE_B_MAXRES, qB1.queueResourceQuotas.getEffectiveMaxResource());
+
+    Assert.assertEquals("Effective Min resource for QUEUEA1 is not correct",
+        QUEUE_A1_MINRES, qA1.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA1 is not correct",
+        QUEUE_A_MAXRES, qA1.queueResourceQuotas.getEffectiveMaxResource());
+
+    Assert.assertEquals("Effective Min resource for QUEUEA2 is not correct",
+        QUEUE_A2_MINRES, qA2.queueResourceQuotas.getEffectiveMinResource());
+    Assert.assertEquals("Effective Max resource for QUEUEA2 is not correct",
+        QUEUE_A_MAXRES, qA2.queueResourceQuotas.getEffectiveMaxResource());
+
+    rm.stop();
+  }
 }


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


[37/50] [abbrv] hadoop git commit: YARN-7473. Implement Framework and policy for capacity management of auto created queues. (Suma Shivaprasad via wangda)

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b38643c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservationQueue.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/TestReservationQueue.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/TestReservationQueue.java
new file mode 100644
index 0000000..f6b4f2a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservationQueue.java
@@ -0,0 +1,114 @@
+/**
+ * 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.scheduler.capacity;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test class for dynamic auto created leaf queues.
+ * @see ReservationQueue
+ */
+public class TestReservationQueue {
+
+  private CapacitySchedulerConfiguration csConf;
+  private CapacitySchedulerContext csContext;
+  final static int DEF_MAX_APPS = 10000;
+  final static int GB = 1024;
+  private final ResourceCalculator resourceCalculator =
+      new DefaultResourceCalculator();
+  private ReservationQueue autoCreatedLeafQueue;
+
+  @Before
+  public void setup() throws IOException {
+    // setup a context / conf
+    csConf = new CapacitySchedulerConfiguration();
+
+    YarnConfiguration conf = new YarnConfiguration();
+    csContext = mock(CapacitySchedulerContext.class);
+    when(csContext.getConfiguration()).thenReturn(csConf);
+    when(csContext.getConf()).thenReturn(conf);
+    when(csContext.getMinimumResourceCapability()).thenReturn(
+        Resources.createResource(GB, 1));
+    when(csContext.getMaximumResourceCapability()).thenReturn(
+        Resources.createResource(16 * GB, 32));
+    when(csContext.getClusterResource()).thenReturn(
+        Resources.createResource(100 * 16 * GB, 100 * 32));
+    when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    RMContext mockRMContext = TestUtils.getMockRMContext();
+    when(csContext.getRMContext()).thenReturn(mockRMContext);
+
+    // create a queue
+    PlanQueue pq = new PlanQueue(csContext, "root", null, null);
+    autoCreatedLeafQueue = new ReservationQueue(csContext, "a", pq);
+  }
+
+  private void validateAutoCreatedLeafQueue(double capacity) {
+    assertTrue(" actual capacity: " + autoCreatedLeafQueue.getCapacity(),
+        autoCreatedLeafQueue.getCapacity() - capacity < CSQueueUtils.EPSILON);
+    assertEquals(autoCreatedLeafQueue.maxApplications, DEF_MAX_APPS);
+    assertEquals(autoCreatedLeafQueue.maxApplicationsPerUser, DEF_MAX_APPS);
+  }
+
+  @Test
+  public void testAddSubtractCapacity() throws Exception {
+
+    // verify that setting, adding, subtracting capacity works
+    autoCreatedLeafQueue.setCapacity(1.0F);
+    validateAutoCreatedLeafQueue(1);
+    autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(0.9f, 1f));
+    validateAutoCreatedLeafQueue(0.9);
+    autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(1f, 1f));
+    validateAutoCreatedLeafQueue(1);
+    autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(0f, 1f));
+    validateAutoCreatedLeafQueue(0);
+
+    try {
+      autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(1.1f, 1f));
+      fail();
+    } catch (SchedulerDynamicEditException iae) {
+      // expected
+      validateAutoCreatedLeafQueue(1);
+    }
+
+    try {
+      autoCreatedLeafQueue.setEntitlement(new QueueEntitlement(-0.1f, 1f));
+      fail();
+    } catch (SchedulerDynamicEditException iae) {
+      // expected
+      validateAutoCreatedLeafQueue(1);
+    }
+
+  }
+}


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


[47/50] [abbrv] hadoop git commit: YARN-7448. [API] Add SchedulingRequest to the AllocateRequest. (Panagiotis Garefalakis via asuresh)

Posted by wa...@apache.org.
YARN-7448. [API] Add SchedulingRequest to the AllocateRequest. (Panagiotis Garefalakis via asuresh)


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

Branch: refs/heads/YARN-6592
Commit: e2b1dc040ea6e8a2441092570040ef55f1752b84
Parents: c519f0d
Author: Arun Suresh <as...@apache.org>
Authored: Fri Nov 17 10:42:43 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Sun Dec 10 10:27:01 2017 -0800

----------------------------------------------------------------------
 .../api/protocolrecords/AllocateRequest.java    | 42 ++++++++++
 .../hadoop/yarn/api/records/ResourceSizing.java | 27 +++++++
 .../yarn/api/records/SchedulingRequest.java     |  1 +
 .../src/main/proto/yarn_service_protos.proto    |  1 +
 .../impl/pb/AllocateRequestPBImpl.java          | 83 ++++++++++++++++++++
 .../records/impl/pb/ResourceSizingPBImpl.java   |  2 +-
 .../impl/pb/SchedulingRequestPBImpl.java        | 16 ++++
 .../hadoop/yarn/api/TestPBImplRecords.java      | 19 +++++
 8 files changed, 190 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2b1dc04/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 ae0891e..d8d2347 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -28,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -212,6 +214,32 @@ public abstract class AllocateRequest {
   public abstract void setUpdateRequests(
       List<UpdateContainerRequest> updateRequests);
 
+  /**
+   * Get the list of Scheduling requests being sent by the
+   * <code>ApplicationMaster</code>.
+   * @return list of {@link SchedulingRequest} being sent by the
+   *         <code>ApplicationMaster</code>.
+   */
+  @Public
+  @Unstable
+  public List<SchedulingRequest> getSchedulingRequests() {
+    return Collections.EMPTY_LIST;
+  }
+
+  /**
+   * Set the list of Scheduling requests to inform the
+   * <code>ResourceManager</code> about the application's resource requirements
+   * (potentially including allocation tags & placement constraints).
+   * @param schedulingRequests list of <code>SchedulingRequest</code> to update
+   *          the <code>ResourceManager</code> about the application's resource
+   *          requirements.
+   */
+  @Public
+  @Unstable
+  public void setSchedulingRequests(
+      List<SchedulingRequest> schedulingRequests) {
+  }
+
   @Public
   @Unstable
   public static AllocateRequestBuilder newBuilder() {
@@ -314,6 +342,20 @@ public abstract class AllocateRequest {
     }
 
     /**
+     * Set the <code>schedulingRequests</code> of the request.
+     * @see AllocateRequest#setSchedulingRequests(List)
+     * @param schedulingRequests <code>SchedulingRequest</code> of the request
+     * @return {@link AllocateRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public AllocateRequestBuilder schedulingRequests(
+        List<SchedulingRequest> schedulingRequests) {
+      allocateRequest.setSchedulingRequests(schedulingRequests);
+      return this;
+    }
+
+    /**
      * Return generated {@link AllocateRequest} object.
      * @return {@link AllocateRequest}
      */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2b1dc04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
index d82be11..8cdc63f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
@@ -61,4 +61,31 @@ public abstract class ResourceSizing {
   @Public
   @Unstable
   public abstract void setResources(Resource resources);
+
+  @Override
+  public int hashCode() {
+    int result = getResources().hashCode();
+    result = 31 * result + getNumAllocations();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if(obj == null || getClass() != obj.getClass()) {
+      return false;
+    }
+
+    ResourceSizing that = (ResourceSizing) obj;
+
+    if(getNumAllocations() != that.getNumAllocations()) {
+      return  false;
+    }
+    if(!getResources().equals(that.getResources())) {
+      return false;
+    }
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2b1dc04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
index 47a0697..e32dd24 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
@@ -49,6 +49,7 @@ public abstract class SchedulingRequest {
     return SchedulingRequest.newBuilder()
         .allocationRequestId(allocationRequestId).priority(priority)
         .executionType(executionType).allocationTags(allocationTags)
+        .resourceSizing(resourceSizing)
         .placementConstraintExpression(placementConstraintExpression).build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2b1dc04/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 68e585d..e49c4e3 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
@@ -91,6 +91,7 @@ message AllocateRequestProto {
   optional int32 response_id = 4;
   optional float progress = 5;
   repeated UpdateContainerRequestProto update_requests = 7;
+  repeated SchedulingRequestProto scheduling_requests = 10;
 }
 
 message NMTokenProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2b1dc04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
index 0f0f571..b460044 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
@@ -29,14 +29,17 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SchedulingRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.UpdateContainerRequestPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SchedulingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.UpdateContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProtoOrBuilder;
@@ -53,6 +56,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   private List<ResourceRequest> ask = null;
   private List<ContainerId> release = null;
   private List<UpdateContainerRequest> updateRequests = null;
+  private List<SchedulingRequest> schedulingRequests = null;
   private ResourceBlacklistRequest blacklistRequest = null;
   
   public AllocateRequestPBImpl() {
@@ -101,6 +105,9 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     if (this.updateRequests != null) {
       addUpdateRequestsToProto();
     }
+    if (this.schedulingRequests != null) {
+      addSchedulingRequestsToProto();
+    }
     if (this.blacklistRequest != null) {
       builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest));
     }
@@ -178,6 +185,23 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   }
 
   @Override
+  public List<SchedulingRequest> getSchedulingRequests() {
+    initSchedulingRequests();
+    return this.schedulingRequests;
+  }
+
+  @Override
+  public void setSchedulingRequests(
+      List<SchedulingRequest> schedulingRequests) {
+    if (schedulingRequests == null) {
+      return;
+    }
+    initSchedulingRequests();
+    this.schedulingRequests.clear();
+    this.schedulingRequests.addAll(schedulingRequests);
+  }
+
+  @Override
   public ResourceBlacklistRequest getResourceBlacklistRequest() {
     AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (this.blacklistRequest != null) {
@@ -261,6 +285,20 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     }
   }
 
+  private void initSchedulingRequests() {
+    if (this.schedulingRequests != null) {
+      return;
+    }
+    AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<SchedulingRequestProto> list =
+        p.getSchedulingRequestsList();
+    this.schedulingRequests = new ArrayList<>();
+
+    for (SchedulingRequestProto c : list) {
+      this.schedulingRequests.add(convertFromProtoFormat(c));
+    }
+  }
+
   private void addUpdateRequestsToProto() {
     maybeInitBuilder();
     builder.clearUpdateRequests();
@@ -297,6 +335,41 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     builder.addAllUpdateRequests(iterable);
   }
 
+  private void addSchedulingRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearSchedulingRequests();
+    if (schedulingRequests == null) {
+      return;
+    }
+    Iterable<SchedulingRequestProto> iterable =
+        new Iterable<SchedulingRequestProto>() {
+          @Override
+          public Iterator<SchedulingRequestProto> iterator() {
+            return new Iterator<SchedulingRequestProto>() {
+
+              private Iterator<SchedulingRequest> iter =
+                  schedulingRequests.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public SchedulingRequestProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+
+          }
+        };
+    builder.addAllSchedulingRequests(iterable);
+  }
   @Override
   public List<ContainerId> getReleaseList() {
     initReleases();
@@ -377,6 +450,16 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     return ((UpdateContainerRequestPBImpl) t).getProto();
   }
 
+  private SchedulingRequestPBImpl convertFromProtoFormat(
+      SchedulingRequestProto p) {
+    return new SchedulingRequestPBImpl(p);
+  }
+
+  private SchedulingRequestProto convertToProtoFormat(
+      SchedulingRequest t) {
+    return ((SchedulingRequestPBImpl) t).getProto();
+  }
+
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
     return new ContainerIdPBImpl(p);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2b1dc04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
index 05bb3bd..f98e488 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
@@ -112,6 +112,6 @@ public class ResourceSizingPBImpl extends ResourceSizing {
   }
 
   private ResourceProto convertToProtoFormat(Resource r) {
-    return ((ResourcePBImpl) r).getProto();
+    return ProtoUtils.convertToProtoFormat(r);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2b1dc04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
index 7826b36..305856a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
@@ -263,4 +263,20 @@ public class SchedulingRequestPBImpl extends SchedulingRequest {
     this.allocationTags = new HashSet<>();
     this.allocationTags.addAll(p.getAllocationTagsList());
   }
+
+  @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;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2b1dc04/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index c5585c2..a0b907d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -149,8 +149,10 @@ import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
 import org.apache.hadoop.yarn.api.records.ResourceTypeInfo;
 import org.apache.hadoop.yarn.api.records.ResourceUtilization;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
 import org.apache.hadoop.yarn.api.records.Token;
@@ -189,7 +191,9 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceOptionPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourceSizingPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceTypeInfoPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SchedulingRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.StrictPreemptionContractPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
@@ -225,6 +229,8 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceOptionProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceSizingProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SchedulingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StrictPreemptionContractProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
@@ -428,6 +434,8 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     generateByNewInstance(QueueConfigurations.class);
     generateByNewInstance(CollectorInfo.class);
     generateByNewInstance(ResourceTypeInfo.class);
+    generateByNewInstance(ResourceSizing.class);
+    generateByNewInstance(SchedulingRequest.class);
   }
 
   @Test
@@ -907,6 +915,17 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
   }
 
   @Test
+  public void testResourceSizingPBImpl() throws Exception {
+    validatePBImplRecord(ResourceSizingPBImpl.class, ResourceSizingProto.class);
+  }
+
+  @Test
+  public void testSchedulingRequestPBImpl() throws Exception {
+    validatePBImplRecord(SchedulingRequestPBImpl.class,
+        SchedulingRequestProto.class);
+  }
+
+  @Test
   public void testSerializedExceptionPBImpl() throws Exception {
     validatePBImplRecord(SerializedExceptionPBImpl.class,
         SerializedExceptionProto.class);


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


[22/50] [abbrv] hadoop git commit: YARN-6471. Support to add min/max resource configuration for a queue. (Sunil G via wangda)

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/policy/TestPriorityUtilizationQueueOrderingPolicy.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/policy/TestPriorityUtilizationQueueOrderingPolicy.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/policy/TestPriorityUtilizationQueueOrderingPolicy.java
index e3c108a..b9d5b82 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/policy/TestPriorityUtilizationQueueOrderingPolicy.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/policy/TestPriorityUtilizationQueueOrderingPolicy.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableTable;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.junit.Assert;
@@ -52,6 +53,8 @@ public class TestPriorityUtilizationQueueOrderingPolicy {
       when(q.getQueueCapacities()).thenReturn(qc);
       when(q.getPriority()).thenReturn(Priority.newInstance(priorities[i]));
 
+      QueueResourceQuotas qr = new QueueResourceQuotas();
+      when(q.getQueueResourceQuotas()).thenReturn(qr);
       list.add(q);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
index 1108f1a..0132348 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
@@ -354,10 +354,10 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
   private void verifySubQueue(JSONObject info, String q, 
       float parentAbsCapacity, float parentAbsMaxCapacity)
       throws JSONException, Exception {
-    int numExpectedElements = 18;
+    int numExpectedElements = 20;
     boolean isParentQueue = true;
     if (!info.has("queues")) {
-      numExpectedElements = 31;
+      numExpectedElements = 33;
       isParentQueue = false;
     }
     assertEquals("incorrect number of elements", numExpectedElements, info.length());


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


[13/50] [abbrv] hadoop git commit: MAPREDUCE-7000. Moving logging APIs over to slf4j in hadoop-mapreduce-client-nativetask. Contributed by Jinjiang Ling.

Posted by wa...@apache.org.
MAPREDUCE-7000. Moving logging APIs over to slf4j in hadoop-mapreduce-client-nativetask. Contributed by Jinjiang Ling.


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

Branch: refs/heads/YARN-6592
Commit: c2e8a5c2296a1e5033f5750d9d0a0c668ec7162f
Parents: d4cae97
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 7 16:27:08 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 7 16:27:08 2017 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/mapred/nativetask/HadoopPlatform.java | 7 ++++---
 .../hadoop/mapred/nativetask/NativeBatchProcessor.java      | 9 +++++----
 .../nativetask/NativeMapOutputCollectorDelegator.java       | 7 ++++---
 .../org/apache/hadoop/mapred/nativetask/NativeRuntime.java  | 7 ++++---
 .../java/org/apache/hadoop/mapred/nativetask/Platforms.java | 6 +++---
 .../hadoop/mapred/nativetask/StatusReportChecker.java       | 7 ++++---
 .../hadoop/mapred/nativetask/handlers/BufferPuller.java     | 6 +++---
 .../hadoop/mapred/nativetask/handlers/BufferPushee.java     | 6 +++---
 .../hadoop/mapred/nativetask/handlers/BufferPusher.java     | 6 +++---
 .../hadoop/mapred/nativetask/handlers/CombinerHandler.java  | 7 ++++---
 .../nativetask/handlers/NativeCollectorOnlyHandler.java     | 7 ++++---
 .../apache/hadoop/mapred/nativetask/serde/KVSerializer.java | 7 +++----
 .../apache/hadoop/mapred/nativetask/util/OutputUtil.java    | 6 +++---
 .../mapred/nativetask/combinertest/LargeKVCombinerTest.java | 7 ++++---
 .../hadoop/mapred/nativetask/combinertest/WordCount.java    | 6 +++---
 .../org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java   | 6 +++---
 .../org/apache/hadoop/mapred/nativetask/kvtest/KVTest.java  | 7 ++++---
 .../apache/hadoop/mapred/nativetask/kvtest/LargeKVTest.java | 6 +++---
 .../hadoop/mapred/nativetask/kvtest/TestInputFile.java      | 7 ++++---
 .../testutil/EnforceNativeOutputCollectorDelegator.java     | 7 ++++---
 20 files changed, 72 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/HadoopPlatform.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/HadoopPlatform.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/HadoopPlatform.java
index da6f252..d66d2d2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/HadoopPlatform.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/HadoopPlatform.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.mapred.nativetask;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.ByteWritable;
@@ -36,10 +34,13 @@ import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.nativetask.serde.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class HadoopPlatform extends Platform {
-  private static final Log LOG = LogFactory.getLog(HadoopPlatform.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(HadoopPlatform.class);
 
   public HadoopPlatform() throws IOException {
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeBatchProcessor.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeBatchProcessor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeBatchProcessor.java
index 9458c6b..2ec0643 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeBatchProcessor.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeBatchProcessor.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.mapred.nativetask;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
@@ -31,13 +29,16 @@ import org.apache.hadoop.mapred.nativetask.buffer.InputBuffer;
 import org.apache.hadoop.mapred.nativetask.buffer.OutputBuffer;
 import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
 import org.apache.hadoop.mapred.nativetask.util.ConfigUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * used to create channel, transfer data and command between Java and native
  */
 @InterfaceAudience.Private
 public class NativeBatchProcessor implements INativeHandler {
-  private static Log LOG = LogFactory.getLog(NativeBatchProcessor.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NativeBatchProcessor.class);
 
   private final String nativeHandlerName;
   private long nativeHandlerAddr;
@@ -128,7 +129,7 @@ public class NativeBatchProcessor implements INativeHandler {
       NativeRuntime.releaseNativeObject(nativeHandlerAddr);
       nativeHandlerAddr = 0;
     }
-    IOUtils.cleanup(LOG, in);
+    IOUtils.cleanupWithLogger(LOG, in);
     in = null;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java
index 5a516c4..b4755fd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java
@@ -21,8 +21,6 @@ import java.io.IOException;
 
 import com.google.common.base.Charsets;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.mapred.InvalidJobConfException;
@@ -36,6 +34,8 @@ import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskCounter;
 import org.apache.hadoop.util.QuickSort;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * native map output collector wrapped in Java interface
@@ -43,7 +43,8 @@ import org.apache.hadoop.util.QuickSort;
 @InterfaceAudience.Private
 public class NativeMapOutputCollectorDelegator<K, V> implements MapOutputCollector<K, V> {
 
-  private static Log LOG = LogFactory.getLog(NativeMapOutputCollectorDelegator.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NativeMapOutputCollectorDelegator.class);
   private JobConf job;
   private NativeCollectorOnlyHandler<K, V> handler;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java
index cc9adba..8d71ad4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.mapred.nativetask;
 import java.io.IOException;
 
 import com.google.common.base.Charsets;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputBuffer;
@@ -33,6 +31,8 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Task.TaskReporter;
 import org.apache.hadoop.mapred.nativetask.util.ConfigUtil;
 import org.apache.hadoop.util.VersionInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class stands for the native runtime It has three functions:
@@ -42,7 +42,8 @@ import org.apache.hadoop.util.VersionInfo;
  */
 @InterfaceAudience.Private
 public class NativeRuntime {
-  private static Log LOG = LogFactory.getLog(NativeRuntime.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NativeRuntime.class);
   private static boolean nativeLibraryLoaded = false;
 
   private static Configuration conf = new Configuration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/Platforms.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/Platforms.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/Platforms.java
index 9fad3a5..350449d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/Platforms.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/Platforms.java
@@ -20,13 +20,13 @@ package org.apache.hadoop.mapred.nativetask;
 import java.io.IOException;
 import java.util.ServiceLoader;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.nativetask.serde.INativeSerializer;
 import org.apache.hadoop.mapred.nativetask.serde.NativeSerialization;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
@@ -37,7 +37,7 @@ import org.apache.hadoop.mapred.nativetask.serde.NativeSerialization;
 @InterfaceAudience.Private
 public class Platforms {
 
-  private static final Log LOG = LogFactory.getLog(Platforms.class);
+  private static final Logger LOG = LoggerFactory.getLogger(Platforms.class);
   private static final ServiceLoader<Platform> platforms = ServiceLoader.load(Platform.class);
   
   public static void init(Configuration conf) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/StatusReportChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/StatusReportChecker.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/StatusReportChecker.java
index 1e76d39..6a437b1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/StatusReportChecker.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/StatusReportChecker.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.mapred.nativetask;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapred.Task.TaskReporter;
 import org.apache.hadoop.mapreduce.TaskCounter;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Will periodically check status from native and report to MR framework.
@@ -31,7 +31,8 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
  */
 class StatusReportChecker implements Runnable {
 
-  private static Log LOG = LogFactory.getLog(StatusReportChecker.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StatusReportChecker.class);
   public static final int INTERVAL = 1000; // milliseconds
 
   private Thread checker;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPuller.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPuller.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPuller.java
index b5dff92..6c43a65 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPuller.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPuller.java
@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
@@ -33,6 +31,8 @@ import org.apache.hadoop.mapred.nativetask.buffer.BufferType;
 import org.apache.hadoop.mapred.nativetask.buffer.ByteBufferDataReader;
 import org.apache.hadoop.mapred.nativetask.buffer.InputBuffer;
 import org.apache.hadoop.util.Progress;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * actively signal a {@link BufferPullee} to load data into buffer and receive
@@ -40,7 +40,7 @@ import org.apache.hadoop.util.Progress;
 @InterfaceAudience.Private
 public class BufferPuller implements RawKeyValueIterator, DataReceiver {
   
-  private static Log LOG = LogFactory.getLog(BufferPuller.class);
+  private static final Logger LOG = LoggerFactory.getLogger(BufferPuller.class);
 
   public final static int KV_HEADER_LENGTH = Constants.SIZEOF_KV_LENGTH;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPushee.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPushee.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPushee.java
index 090882d..a78789e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPushee.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPushee.java
@@ -21,8 +21,6 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.mapred.RecordWriter;
 import org.apache.hadoop.mapred.nativetask.Constants;
@@ -31,6 +29,8 @@ import org.apache.hadoop.mapred.nativetask.buffer.ByteBufferDataReader;
 import org.apache.hadoop.mapred.nativetask.buffer.InputBuffer;
 import org.apache.hadoop.mapred.nativetask.serde.KVSerializer;
 import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * collect data when signaled
@@ -38,7 +38,7 @@ import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
 @InterfaceAudience.Private
 public class BufferPushee<OK, OV> implements Closeable {
 
-  private static Log LOG = LogFactory.getLog(BufferPushee.class);
+  private static final Logger LOG = LoggerFactory.getLogger(BufferPushee.class);
   
   public final static int KV_HEADER_LENGTH = Constants.SIZEOF_KV_LENGTH;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPusher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPusher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPusher.java
index 454127c..787e63f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPusher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPusher.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.mapred.nativetask.handlers;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.nativetask.NativeDataTarget;
@@ -29,6 +27,8 @@ import org.apache.hadoop.mapred.nativetask.buffer.ByteBufferDataWriter;
 import org.apache.hadoop.mapred.nativetask.serde.IKVSerializer;
 import org.apache.hadoop.mapred.nativetask.serde.KVSerializer;
 import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * actively push data into a buffer and signal a {@link BufferPushee} to collect it
@@ -36,7 +36,7 @@ import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
 @InterfaceAudience.Private
 public class BufferPusher<K, V> implements OutputCollector<K, V> {
   
-  private static Log LOG = LogFactory.getLog(BufferPusher.class);
+  private static final Logger LOG = LoggerFactory.getLogger(BufferPusher.class);
 
   private final SizedWritable<K> tmpInputKey;
   private final SizedWritable<V> tmpInputValue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/CombinerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/CombinerHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/CombinerHandler.java
index fcaeca1..78ae00d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/CombinerHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/CombinerHandler.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.mapred.nativetask.handlers;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Task.CombinerRunner;
@@ -36,10 +34,13 @@ import org.apache.hadoop.mapred.nativetask.serde.SerializationFramework;
 import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskCounter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class CombinerHandler<K, V> implements ICombineHandler, CommandDispatcher {
   public static final String NAME = "NativeTask.CombineHandler";
-  private static Log LOG = LogFactory.getLog(NativeCollectorOnlyHandler.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NativeCollectorOnlyHandler.class);
   public static final Command LOAD = new Command(1, "Load");
   public static final Command COMBINE = new Command(4, "Combine");
   public final CombinerRunner<K, V> combinerRunner;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/NativeCollectorOnlyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/NativeCollectorOnlyHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/NativeCollectorOnlyHandler.java
index 34238ae..8bf4d8b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/NativeCollectorOnlyHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/NativeCollectorOnlyHandler.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.mapred.nativetask.handlers;
 import java.io.Closeable;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -37,6 +35,8 @@ import org.apache.hadoop.mapred.nativetask.TaskContext;
 import org.apache.hadoop.mapred.nativetask.util.NativeTaskOutput;
 import org.apache.hadoop.mapred.nativetask.util.OutputUtil;
 import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Java Record Reader + Java Mapper + Native Collector
@@ -46,7 +46,8 @@ import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
 public class NativeCollectorOnlyHandler<K, V> implements CommandDispatcher, Closeable {
 
   public static final String NAME = "NativeTask.MCollectorOutputHandler";
-  private static Log LOG = LogFactory.getLog(NativeCollectorOnlyHandler.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NativeCollectorOnlyHandler.class);
   public static final Command GET_OUTPUT_PATH =
       new Command(100, "GET_OUTPUT_PATH");
   public static final Command GET_OUTPUT_INDEX_PATH =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/KVSerializer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/KVSerializer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/KVSerializer.java
index e6609b8..4485ed0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/KVSerializer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/KVSerializer.java
@@ -20,21 +20,20 @@ package org.apache.hadoop.mapred.nativetask.serde;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.nativetask.Constants;
 import org.apache.hadoop.mapred.nativetask.buffer.DataInputStream;
 import org.apache.hadoop.mapred.nativetask.buffer.DataOutputStream;
 import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 @InterfaceAudience.Private
 public class KVSerializer<K, V> implements IKVSerializer {
 
-  private static final Log LOG = LogFactory.getLog(KVSerializer.class);
+  private static final Logger LOG = LoggerFactory.getLogger(KVSerializer.class);
   
   public static final int KV_HEAD_LENGTH = Constants.SIZEOF_KV_LENGTH;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/OutputUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/OutputUtil.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/OutputUtil.java
index dd45ede..9809c8e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/OutputUtil.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/OutputUtil.java
@@ -20,15 +20,15 @@ package org.apache.hadoop.mapred.nativetask.util;
 
 import java.lang.reflect.Constructor;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class OutputUtil {
 
-  private static Log LOG = LogFactory.getLog(OutputUtil.class);
+  private static final Logger LOG = LoggerFactory.getLogger(OutputUtil.class);
   public static final String NATIVE_TASK_OUTPUT_MANAGER = "nativetask.output.manager";
 
   public static NativeTaskOutput createNativeTaskOutput(Configuration conf, String id) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/LargeKVCombinerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/LargeKVCombinerTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/LargeKVCombinerTest.java
index a4eecbe..299acbd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/LargeKVCombinerTest.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/LargeKVCombinerTest.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.mapred.nativetask.combinertest;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -41,11 +39,14 @@ import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
 public class LargeKVCombinerTest {
-  private static final Log LOG = LogFactory.getLog(LargeKVCombinerTest.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(LargeKVCombinerTest.class);
 
   @Before
   public void startUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/WordCount.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/WordCount.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/WordCount.java
index c022848..c38f7e8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/WordCount.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/combinertest/WordCount.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.mapred.nativetask.combinertest;
 import java.io.IOException;
 import java.util.StringTokenizer;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
@@ -33,10 +31,12 @@ import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.GenericOptionsParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class WordCount {
 
-  private static Log LOG = LogFactory.getLog(WordCount.class);
+  private static final Logger LOG = LoggerFactory.getLogger(WordCount.class);
   
   public static class TokenizerMapper extends Mapper<Object, Text, Text, IntWritable> {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java
index 3b4c9c0..802259f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java
@@ -23,8 +23,6 @@ import java.util.zip.CRC32;
 
 import com.google.common.primitives.Longs;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -37,11 +35,13 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.StopWatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class KVJob {
   public static final String INPUTPATH = "nativetask.kvtest.inputfile.path";
   public static final String OUTPUTPATH = "nativetask.kvtest.outputfile.path";
-  private static final Log LOG = LogFactory.getLog(KVJob.class);
+  private static final Logger LOG = LoggerFactory.getLogger(KVJob.class);
   Job job = null;
 
   public static class ValueMapper<KTYPE, VTYPE> extends Mapper<KTYPE, VTYPE, KTYPE, VTYPE> {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVTest.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVTest.java
index 6b658ac..5364401 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVTest.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVTest.java
@@ -23,8 +23,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -41,13 +39,16 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Splitter;
 import com.google.common.collect.Lists;
 
 @RunWith(Parameterized.class)
 public class KVTest {
-  private static final Log LOG = LogFactory.getLog(KVTest.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(KVTest.class);
 
   private static Configuration nativekvtestconf = ScenarioConfiguration.getNativeConfiguration();
   private static Configuration hadoopkvtestconf = ScenarioConfiguration.getNormalConfiguration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/LargeKVTest.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/LargeKVTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/LargeKVTest.java
index 5f1619e..4a7b963 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/LargeKVTest.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/LargeKVTest.java
@@ -22,8 +22,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -38,9 +36,11 @@ import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class LargeKVTest {
-  private static final Log LOG = LogFactory.getLog(LargeKVTest.class);
+  private static final Logger LOG = LoggerFactory.getLogger(LargeKVTest.class);
 
   @Before
   public void startUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/TestInputFile.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/TestInputFile.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/TestInputFile.java
index 04db67a..4cf534b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/TestInputFile.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/TestInputFile.java
@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -40,10 +38,13 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.nativetask.testutil.BytesFactory;
 import org.apache.hadoop.mapred.nativetask.testutil.ScenarioConfiguration;
 import org.apache.hadoop.mapred.nativetask.testutil.TestConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 public class TestInputFile {
-	private static Log LOG = LogFactory.getLog(TestInputFile.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestInputFile.class);
 
   public static class KVSizeScope {
     private static final int DefaultMinNum = 1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2e8a5c2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/testutil/EnforceNativeOutputCollectorDelegator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/testutil/EnforceNativeOutputCollectorDelegator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/testutil/EnforceNativeOutputCollectorDelegator.java
index 7212236..d38056d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/testutil/EnforceNativeOutputCollectorDelegator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/testutil/EnforceNativeOutputCollectorDelegator.java
@@ -19,13 +19,14 @@ package org.apache.hadoop.mapred.nativetask.testutil;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapred.nativetask.NativeMapOutputCollectorDelegator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class EnforceNativeOutputCollectorDelegator<K, V>
   extends NativeMapOutputCollectorDelegator<K, V> {
-  private static final Log LOG = LogFactory.getLog(EnforceNativeOutputCollectorDelegator.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(EnforceNativeOutputCollectorDelegator.class);
   private boolean nativetaskloaded = false;
 
   @Override


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


[45/50] [abbrv] hadoop git commit: YARN-6704. Add support for work preserving NM restart when FederationInterceptor is enabled in AMRMProxyService. (Botong Huang via Subru).

Posted by wa...@apache.org.
YARN-6704. Add support for work preserving NM restart when FederationInterceptor is enabled in AMRMProxyService. (Botong Huang via Subru).


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

Branch: refs/heads/YARN-6592
Commit: 670e8d4ec7e71fc3b054cd3b2826f869b649a788
Parents: 04b84da
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Dec 8 15:39:18 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Fri Dec 8 15:39:18 2017 -0800

----------------------------------------------------------------------
 .../yarn/server/MockResourceManagerFacade.java  |  16 +-
 .../nodemanager/amrmproxy/AMRMProxyService.java |   5 +-
 .../amrmproxy/FederationInterceptor.java        | 271 +++++++++++++++++--
 .../amrmproxy/BaseAMRMProxyTest.java            |  15 +
 .../amrmproxy/TestFederationInterceptor.java    | 104 +++++++
 .../TestableFederationInterceptor.java          |   8 +-
 6 files changed, 387 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
index b5727aa..15e1cea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
@@ -111,6 +111,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@@ -622,7 +623,20 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
 
     validateRunning();
 
-    return GetContainersResponse.newInstance(null);
+    ApplicationAttemptId attemptId = request.getApplicationAttemptId();
+    List<ContainerReport> containers = new ArrayList<>();
+    synchronized (applicationContainerIdMap) {
+      // Return the list of running containers that were being tracked for this
+      // application
+      Assert.assertTrue("The application id is NOT registered: " + attemptId,
+          applicationContainerIdMap.containsKey(attemptId));
+      List<ContainerId> ids = applicationContainerIdMap.get(attemptId);
+      for (ContainerId c : ids) {
+        containers.add(ContainerReport.newInstance(c, null, null, null, 0, 0,
+            null, null, 0, null, null));
+      }
+    }
+    return GetContainersResponse.newInstance(containers);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.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/amrmproxy/AMRMProxyService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
index ebd85bf..815e39b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
@@ -128,11 +128,8 @@ public class AMRMProxyService extends CompositeService implements
         new AMRMProxyTokenSecretManager(this.nmContext.getNMStateStore());
     this.secretManager.init(conf);
 
-    // Both second app attempt and NM restart within Federation need registry
     if (conf.getBoolean(YarnConfiguration.AMRM_PROXY_HA_ENABLED,
-        YarnConfiguration.DEFAULT_AMRM_PROXY_HA_ENABLED)
-        || conf.getBoolean(YarnConfiguration.NM_RECOVERY_ENABLED,
-            YarnConfiguration.DEFAULT_NM_RECOVERY_ENABLED)) {
+        YarnConfiguration.DEFAULT_AMRM_PROXY_HA_ENABLED)) {
       this.registry = FederationStateStoreFacade.createInstance(conf,
           YarnConfiguration.YARN_REGISTRY_CLASS,
           YarnConfiguration.DEFAULT_YARN_REGISTRY_CLASS,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.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/amrmproxy/FederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
index ef5e061..9a53a50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
@@ -37,16 +37,23 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.PreemptionContract;
@@ -59,6 +66,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProto;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
@@ -90,6 +99,22 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private static final Logger LOG =
       LoggerFactory.getLogger(FederationInterceptor.class);
 
+  public static final String NMSS_CLASS_PREFIX = "FederationInterceptor/";
+
+  public static final String NMSS_REG_REQUEST_KEY =
+      NMSS_CLASS_PREFIX + "registerRequest";
+  public static final String NMSS_REG_RESPONSE_KEY =
+      NMSS_CLASS_PREFIX + "registerResponse";
+
+  /*
+   * When AMRMProxy HA is enabled, secondary AMRMTokens will be stored in Yarn
+   * Registry. Otherwise if NM recovery is enabled, the UAM token are store in
+   * local NMSS instead under this directory name.
+   */
+  public static final String NMSS_SECONDARY_SC_PREFIX =
+      NMSS_CLASS_PREFIX + "secondarySC/";
+  public static final String STRING_TO_BYTE_FORMAT = "UTF-8";
+
   /**
    * The home sub-cluster is the sub-cluster where the AM container is running
    * in.
@@ -187,14 +212,20 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     } catch (Exception ex) {
       throw new YarnRuntimeException(ex);
     }
-    // Add all app tokens for Yarn Registry access
-    if (this.registryClient != null && appContext.getCredentials() != null) {
-      this.appOwner.addCredentials(appContext.getCredentials());
+
+    if (appContext.getRegistryClient() != null) {
+      this.registryClient = new FederationRegistryClient(conf,
+          appContext.getRegistryClient(), this.appOwner);
+      // Add all app tokens for Yarn Registry access
+      if (appContext.getCredentials() != null) {
+        this.appOwner.addCredentials(appContext.getCredentials());
+      }
     }
 
     this.homeSubClusterId =
         SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
-    this.homeRM = createHomeRMProxy(appContext);
+    this.homeRM = createHomeRMProxy(appContext, ApplicationMasterProtocol.class,
+        this.appOwner);
 
     this.federationFacade = FederationStateStoreFacade.getInstance();
     this.subClusterResolver = this.federationFacade.getSubClusterResolver();
@@ -204,11 +235,137 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
 
     this.uamPool.init(conf);
     this.uamPool.start();
+  }
 
-    if (appContext.getRegistryClient() != null) {
-      this.registryClient = new FederationRegistryClient(conf,
-          appContext.getRegistryClient(), this.appOwner);
+  @Override
+  public void recover(Map<String, byte[]> recoveredDataMap) {
+    super.recover(recoveredDataMap);
+    LOG.info("Recovering data for FederationInterceptor");
+    if (recoveredDataMap == null) {
+      return;
     }
+
+    ApplicationAttemptId attemptId =
+        getApplicationContext().getApplicationAttemptId();
+    try {
+      if (recoveredDataMap.containsKey(NMSS_REG_REQUEST_KEY)) {
+        RegisterApplicationMasterRequestProto pb =
+            RegisterApplicationMasterRequestProto
+                .parseFrom(recoveredDataMap.get(NMSS_REG_REQUEST_KEY));
+        this.amRegistrationRequest =
+            new RegisterApplicationMasterRequestPBImpl(pb);
+        LOG.info("amRegistrationRequest recovered for {}", attemptId);
+      }
+      if (recoveredDataMap.containsKey(NMSS_REG_RESPONSE_KEY)) {
+        RegisterApplicationMasterResponseProto pb =
+            RegisterApplicationMasterResponseProto
+                .parseFrom(recoveredDataMap.get(NMSS_REG_RESPONSE_KEY));
+        this.amRegistrationResponse =
+            new RegisterApplicationMasterResponsePBImpl(pb);
+        LOG.info("amRegistrationResponse recovered for {}", attemptId);
+      }
+
+      // Recover UAM amrmTokens from registry or NMSS
+      Map<String, Token<AMRMTokenIdentifier>> uamMap;
+      if (this.registryClient != null) {
+        uamMap = this.registryClient
+            .loadStateFromRegistry(attemptId.getApplicationId());
+        LOG.info("Found {} existing UAMs for application {} in Yarn Registry",
+            uamMap.size(), attemptId.getApplicationId());
+      } else {
+        uamMap = new HashMap<>();
+        for (Entry<String, byte[]> entry : recoveredDataMap.entrySet()) {
+          if (entry.getKey().startsWith(NMSS_SECONDARY_SC_PREFIX)) {
+            // entry for subClusterId -> UAM amrmToken
+            String scId =
+                entry.getKey().substring(NMSS_SECONDARY_SC_PREFIX.length());
+            Token<AMRMTokenIdentifier> amrmToken = new Token<>();
+            amrmToken.decodeFromUrlString(
+                new String(entry.getValue(), STRING_TO_BYTE_FORMAT));
+            uamMap.put(scId, amrmToken);
+            LOG.debug("Recovered UAM in " + scId + " from NMSS");
+          }
+        }
+        LOG.info("Found {} existing UAMs for application {} in NMStateStore",
+            uamMap.size(), attemptId.getApplicationId());
+      }
+
+      // Re-attach the UAMs
+      int containers = 0;
+      for (Map.Entry<String, Token<AMRMTokenIdentifier>> entry : uamMap
+          .entrySet()) {
+        SubClusterId subClusterId = SubClusterId.newInstance(entry.getKey());
+
+        // Create a config loaded with federation on and subclusterId
+        // for each UAM
+        YarnConfiguration config = new YarnConfiguration(getConf());
+        FederationProxyProviderUtil.updateConfForFederation(config,
+            subClusterId.getId());
+
+        try {
+          this.uamPool.reAttachUAM(subClusterId.getId(), config,
+              attemptId.getApplicationId(),
+              this.amRegistrationResponse.getQueue(),
+              getApplicationContext().getUser(), this.homeSubClusterId.getId(),
+              entry.getValue());
+
+          RegisterApplicationMasterResponse response =
+              this.uamPool.registerApplicationMaster(subClusterId.getId(),
+                  this.amRegistrationRequest);
+
+          // Running containers from secondary RMs
+          for (Container container : response
+              .getContainersFromPreviousAttempts()) {
+            containerIdToSubClusterIdMap.put(container.getId(), subClusterId);
+            containers++;
+          }
+          LOG.info("Recovered {} running containers from UAM in {}",
+              response.getContainersFromPreviousAttempts().size(),
+              subClusterId);
+
+        } catch (Exception e) {
+          LOG.error(
+              "Error reattaching UAM to " + subClusterId + " for " + attemptId,
+              e);
+        }
+      }
+
+      // Get the running containers from home RM, note that we will also get the
+      // AM container itself from here. We don't need it, but no harm to put the
+      // map as well.
+      UserGroupInformation appSubmitter = UserGroupInformation
+          .createRemoteUser(getApplicationContext().getUser());
+      ApplicationClientProtocol rmClient =
+          createHomeRMProxy(getApplicationContext(),
+              ApplicationClientProtocol.class, appSubmitter);
+
+      GetContainersResponse response =
+          rmClient.getContainers(GetContainersRequest.newInstance(attemptId));
+      for (ContainerReport container : response.getContainerList()) {
+        containerIdToSubClusterIdMap.put(container.getContainerId(),
+            this.homeSubClusterId);
+        containers++;
+        LOG.debug("  From home RM " + this.homeSubClusterId
+            + " running container " + container.getContainerId());
+      }
+      LOG.info("{} running containers including AM recovered from home RM ",
+          response.getContainerList().size(), this.homeSubClusterId);
+
+      LOG.info(
+          "In all {} UAMs {} running containers including AM recovered for {}",
+          uamMap.size(), containers, attemptId);
+
+      if (this.amRegistrationResponse != null) {
+        // Initialize the AMRMProxyPolicy
+        String queue = this.amRegistrationResponse.getQueue();
+        this.policyInterpreter =
+            FederationPolicyUtils.loadAMRMPolicy(queue, this.policyInterpreter,
+                getConf(), this.federationFacade, this.homeSubClusterId);
+      }
+    } catch (IOException | YarnException e) {
+      throw new YarnRuntimeException(e);
+    }
+
   }
 
   /**
@@ -242,6 +399,19 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       // Save the registration request. This will be used for registering with
       // secondary sub-clusters using UAMs, as well as re-register later
       this.amRegistrationRequest = request;
+      if (getNMStateStore() != null) {
+        try {
+          RegisterApplicationMasterRequestPBImpl pb =
+              (RegisterApplicationMasterRequestPBImpl)
+                  this.amRegistrationRequest;
+          getNMStateStore().storeAMRMProxyAppContextEntry(
+              getApplicationContext().getApplicationAttemptId(),
+              NMSS_REG_REQUEST_KEY, pb.getProto().toByteArray());
+        } catch (Exception e) {
+          LOG.error("Error storing AMRMProxy application context entry for "
+              + getApplicationContext().getApplicationAttemptId(), e);
+        }
+      }
     }
 
     /*
@@ -278,6 +448,20 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
         getApplicationContext().getApplicationAttemptId().getApplicationId();
     reAttachUAMAndMergeRegisterResponse(this.amRegistrationResponse, appId);
 
+    if (getNMStateStore() != null) {
+      try {
+        RegisterApplicationMasterResponsePBImpl pb =
+            (RegisterApplicationMasterResponsePBImpl)
+                this.amRegistrationResponse;
+        getNMStateStore().storeAMRMProxyAppContextEntry(
+            getApplicationContext().getApplicationAttemptId(),
+            NMSS_REG_RESPONSE_KEY, pb.getProto().toByteArray());
+      } catch (Exception e) {
+        LOG.error("Error storing AMRMProxy application context entry for "
+            + getApplicationContext().getApplicationAttemptId(), e);
+      }
+    }
+
     // the queue this application belongs will be used for getting
     // AMRMProxy policy from state store.
     String queue = this.amRegistrationResponse.getQueue();
@@ -437,6 +621,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           if (uamResponse.getResponse() == null
               || !uamResponse.getResponse().getIsUnregistered()) {
             failedToUnRegister = true;
+          } else if (getNMStateStore() != null) {
+            getNMStateStore().removeAMRMProxyAppContextEntry(
+                getApplicationContext().getApplicationAttemptId(),
+                NMSS_SECONDARY_SC_PREFIX + uamResponse.getSubClusterId());
           }
         } catch (Throwable e) {
           failedToUnRegister = true;
@@ -496,6 +684,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     }
   }
 
+  @VisibleForTesting
+  protected FederationRegistryClient getRegistryClient() {
+    return this.registryClient;
+  }
+
   /**
    * Create the UAM pool manager for secondary sub-clsuters. For unit test to
    * override.
@@ -510,18 +703,20 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   }
 
   /**
-   * Returns instance of the ApplicationMasterProtocol proxy class that is used
-   * to connect to the Home resource manager.
+   * Create a proxy instance that is used to connect to the Home resource
+   * manager.
    *
    * @param appContext AMRMProxyApplicationContext
+   * @param protocol the protocol class for the proxy
+   * @param user the ugi for the proxy
+   * @param <T> the type of the proxy
    * @return the proxy created
    */
-  protected ApplicationMasterProtocol createHomeRMProxy(
-      AMRMProxyApplicationContext appContext) {
+  protected <T> T createHomeRMProxy(AMRMProxyApplicationContext appContext,
+      Class<T> protocol, UserGroupInformation user) {
     try {
       return FederationProxyProviderUtil.createRMProxy(appContext.getConf(),
-          ApplicationMasterProtocol.class, this.homeSubClusterId, this.appOwner,
-          appContext.getAMRMToken());
+          protocol, this.homeSubClusterId, user, appContext.getAMRMToken());
     } catch (Exception ex) {
       throw new YarnRuntimeException(ex);
     }
@@ -810,17 +1005,31 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                 responses.add(response);
               }
 
-              // Save the new AMRMToken for the UAM in registry if present
+              // Save the new AMRMToken for the UAM if present
               if (response.getAMRMToken() != null) {
                 Token<AMRMTokenIdentifier> newToken = ConverterUtils
                     .convertFromYarn(response.getAMRMToken(), (Text) null);
-                // Update the token in registry
+                // Update the token in registry or NMSS
                 if (registryClient != null) {
                   registryClient
                       .writeAMRMTokenForUAM(
                           getApplicationContext().getApplicationAttemptId()
                               .getApplicationId(),
                           subClusterId.getId(), newToken);
+                } else if (getNMStateStore() != null) {
+                  try {
+                    getNMStateStore().storeAMRMProxyAppContextEntry(
+                        getApplicationContext().getApplicationAttemptId(),
+                        NMSS_SECONDARY_SC_PREFIX + subClusterId.getId(),
+                        newToken.encodeToUrlString()
+                            .getBytes(STRING_TO_BYTE_FORMAT));
+                  } catch (IOException e) {
+                    LOG.error(
+                        "Error storing UAM token as AMRMProxy "
+                            + "context entry in NMSS for "
+                            + getApplicationContext().getApplicationAttemptId(),
+                        e);
+                  }
                 }
               }
 
@@ -925,12 +1134,20 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
             successfulRegistrations.put(uamResponse.getSubClusterId(),
                 uamResponse.getResponse());
 
+            // Save the UAM token in registry or NMSS
             if (registryClient != null) {
               registryClient.writeAMRMTokenForUAM(
                   getApplicationContext().getApplicationAttemptId()
                       .getApplicationId(),
                   uamResponse.getSubClusterId().getId(),
                   uamResponse.getUamToken());
+            } else if (getNMStateStore() != null) {
+              getNMStateStore().storeAMRMProxyAppContextEntry(
+                  getApplicationContext().getApplicationAttemptId(),
+                  NMSS_SECONDARY_SC_PREFIX
+                      + uamResponse.getSubClusterId().getId(),
+                  uamResponse.getUamToken().encodeToUrlString()
+                      .getBytes(STRING_TO_BYTE_FORMAT));
             }
           }
         } catch (Exception e) {
@@ -952,11 +1169,6 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private AllocateResponse mergeAllocateResponses(
       AllocateResponse homeResponse) {
     // Timing issue, we need to remove the completed and then save the new ones.
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Remove containers: "
-          + homeResponse.getCompletedContainersStatuses());
-      LOG.debug("Adding containers: " + homeResponse.getAllocatedContainers());
-    }
     removeFinishedContainersFromCache(
         homeResponse.getCompletedContainersStatuses());
     cacheAllocatedContainers(homeResponse.getAllocatedContainers(),
@@ -989,6 +1201,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private void removeFinishedContainersFromCache(
       List<ContainerStatus> finishedContainers) {
     for (ContainerStatus container : finishedContainers) {
+      LOG.debug("Completed container {}", container);
       if (containerIdToSubClusterIdMap
           .containsKey(container.getContainerId())) {
         containerIdToSubClusterIdMap.remove(container.getContainerId());
@@ -1146,12 +1359,22 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private void cacheAllocatedContainers(List<Container> containers,
       SubClusterId subClusterId) {
     for (Container container : containers) {
+      LOG.debug("Adding container {}", container);
       if (containerIdToSubClusterIdMap.containsKey(container.getId())) {
         SubClusterId existingSubClusterId =
             containerIdToSubClusterIdMap.get(container.getId());
         if (existingSubClusterId.equals(subClusterId)) {
-          // When RM fails over, the new RM master might send out the same
-          // container allocation more than once. Just move on in this case.
+          /*
+           * When RM fails over, the new RM master might send out the same
+           * container allocation more than once.
+           *
+           * It is also possible because of a recent NM restart with NM recovery
+           * enabled. We recover running containers from RM. But RM might not
+           * notified AM of some of these containers yet. When RM dose notify,
+           * we will already have these containers in the map.
+           *
+           * Either case, just warn and move on.
+           */
           LOG.warn(
               "Duplicate containerID: {} found in the allocated containers"
                   + " from same sub-cluster: {}, so ignoring.",
@@ -1226,7 +1449,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
    */
   private boolean warnIfNotExists(ContainerId containerId, String actionName) {
     if (!this.containerIdToSubClusterIdMap.containsKey(containerId)) {
-      LOG.error("AM is trying to {} a container {} that does not exist. ",
+      LOG.error(
+          "AM is trying to {} a container {} that does not exist. Might happen "
+              + "shortly after NM restart when NM recovery is enabled",
           actionName, containerId.toString());
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
index da1d047..0319dbe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMMemoryStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredAMRMProxyState;
 import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
@@ -180,6 +181,20 @@ public abstract class BaseAMRMProxyTest {
     return new NMContext(null, null, null, null, stateStore, false, this.conf);
   }
 
+  // A utility method for intercepter recover unit test
+  protected Map<String, byte[]> recoverDataMapForAppAttempt(
+      NMStateStoreService nmStateStore, ApplicationAttemptId attemptId)
+      throws IOException {
+    RecoveredAMRMProxyState state = nmStateStore.loadAMRMProxyState();
+    for (Map.Entry<ApplicationAttemptId, Map<String, byte[]>> entry : state
+        .getAppContexts().entrySet()) {
+      if (entry.getKey().equals(attemptId)) {
+        return entry.getValue();
+      }
+    }
+    return null;
+  }
+
   protected List<ContainerId> getCompletedContainerIds(
       List<ContainerStatus> containerStatus) {
     List<ContainerId> ret = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
index aa7ed69..eefaba1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
@@ -450,6 +450,104 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
   }
 
   @Test
+  public void testRecoverWithAMRMProxyHA() throws Exception {
+    testRecover(registry);
+  }
+
+  @Test
+  public void testRecoverWithoutAMRMProxyHA() throws Exception {
+    testRecover(null);
+  }
+
+  public void testRecover(RegistryOperations registryObj) throws Exception {
+    ApplicationUserInfo userInfo = getApplicationUserInfo(testAppId);
+    userInfo.getUser().doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        interceptor = new TestableFederationInterceptor();
+        interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
+            getConf(), attemptId, "test-user", null, null, null, registryObj));
+        interceptor.cleanupRegistry();
+
+        // Register the application
+        RegisterApplicationMasterRequest registerReq =
+            Records.newRecord(RegisterApplicationMasterRequest.class);
+        registerReq.setHost(Integer.toString(testAppId));
+        registerReq.setRpcPort(testAppId);
+        registerReq.setTrackingUrl("");
+
+        RegisterApplicationMasterResponse registerResponse =
+            interceptor.registerApplicationMaster(registerReq);
+        Assert.assertNotNull(registerResponse);
+
+        Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
+
+        // Allocate one batch of containers
+        registerSubCluster(SubClusterId.newInstance("SC-1"));
+        registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
+
+        int numberOfContainers = 3;
+        List<Container> containers =
+            getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
+        Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
+
+        // Prepare for Federation Interceptor restart and recover
+        Map<String, byte[]> recoveredDataMap =
+            recoverDataMapForAppAttempt(nmStateStore, attemptId);
+        String scEntry =
+            FederationInterceptor.NMSS_SECONDARY_SC_PREFIX + "SC-1";
+        if (registryObj == null) {
+          Assert.assertTrue(recoveredDataMap.containsKey(scEntry));
+        } else {
+          // When AMRMPRoxy HA is enabled, NMSS should not have the UAM token,
+          // it should be in Registry
+          Assert.assertFalse(recoveredDataMap.containsKey(scEntry));
+        }
+
+        // Preserve the mock RM instances
+        MockResourceManagerFacade homeRM = interceptor.getHomeRM();
+        ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
+            interceptor.getSecondaryRMs();
+
+        // Create a new intercepter instance and recover
+        interceptor = new TestableFederationInterceptor(homeRM, secondaries);
+        interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
+            getConf(), attemptId, "test-user", null, null, null, registryObj));
+        interceptor.recover(recoveredDataMap);
+
+        Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
+
+        // Release all containers
+        releaseContainersAndAssert(containers);
+
+        // Finish the application
+        FinishApplicationMasterRequest finishReq =
+            Records.newRecord(FinishApplicationMasterRequest.class);
+        finishReq.setDiagnostics("");
+        finishReq.setTrackingUrl("");
+        finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
+
+        FinishApplicationMasterResponse finshResponse =
+            interceptor.finishApplicationMaster(finishReq);
+        Assert.assertNotNull(finshResponse);
+        Assert.assertEquals(true, finshResponse.getIsUnregistered());
+
+        // After the application succeeds, the registry/NMSS entry should be
+        // cleaned up
+        if (registryObj != null) {
+          Assert.assertEquals(0,
+              interceptor.getRegistryClient().getAllApplications().size());
+        } else {
+          recoveredDataMap =
+              recoverDataMapForAppAttempt(nmStateStore, attemptId);
+          Assert.assertFalse(recoveredDataMap.containsKey(scEntry));
+        }
+        return null;
+      }
+    });
+  }
+
+  @Test
   public void testRequestInterceptorChainCreation() throws Exception {
     RequestInterceptor root =
         super.getAMRMProxyService().createRequestInterceptorChain();
@@ -636,6 +734,12 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
             interceptor.finishApplicationMaster(finishReq);
         Assert.assertNotNull(finshResponse);
         Assert.assertEquals(true, finshResponse.getIsUnregistered());
+
+        // After the application succeeds, the registry entry should be deleted
+        if (interceptor.getRegistryClient() != null) {
+          Assert.assertEquals(0,
+              interceptor.getRegistryClient().getAllApplications().size());
+        }
         return null;
       }
     });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
index 23c80ae..1088c69 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
@@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
@@ -59,16 +58,17 @@ public class TestableFederationInterceptor extends FederationInterceptor {
     return new TestableUnmanagedAMPoolManager(threadPool);
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  protected ApplicationMasterProtocol createHomeRMProxy(
-      AMRMProxyApplicationContext appContext) {
+  protected <T> T createHomeRMProxy(AMRMProxyApplicationContext appContext,
+      Class<T> protocol, UserGroupInformation user) {
     synchronized (this) {
       if (mockRm == null) {
         mockRm = new MockResourceManagerFacade(
             new YarnConfiguration(super.getConf()), 0);
       }
     }
-    return mockRm;
+    return (T) mockRm;
   }
 
   @SuppressWarnings("unchecked")


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


[42/50] [abbrv] hadoop git commit: YARN-7520. Queue Ordering policy changes for ordering auto created leaf queues within Managed parent Queues. (Suma Shivaprasad via wangda)

Posted by wa...@apache.org.
YARN-7520. Queue Ordering policy changes for ordering auto created leaf queues within Managed parent Queues. (Suma Shivaprasad via wangda)

Change-Id: I482f086945bd448d512cb5b3879d7371e37ee134


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

Branch: refs/heads/YARN-6592
Commit: a8316df8c05a7b3d1a5577174b838711a49ef971
Parents: f548bff
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Dec 8 15:11:28 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Dec 8 15:11:28 2017 -0800

----------------------------------------------------------------------
 .../PriorityUtilizationQueueOrderingPolicy.java | 83 ++++++++++++++------
 ...tPriorityUtilizationQueueOrderingPolicy.java | 78 ++++++++++++------
 2 files changed, 113 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8316df8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/policy/PriorityUtilizationQueueOrderingPolicy.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/policy/PriorityUtilizationQueueOrderingPolicy.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/policy/PriorityUtilizationQueueOrderingPolicy.java
index 4985a1a..e684c2b 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/policy/PriorityUtilizationQueueOrderingPolicy.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/policy/PriorityUtilizationQueueOrderingPolicy.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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.
@@ -21,9 +21,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels
+    .RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import java.util.ArrayList;
@@ -49,7 +51,8 @@ import java.util.function.Supplier;
  *   other is under: The queue that is under its capacity guarantee gets the
  *   resources.
  */
-public class PriorityUtilizationQueueOrderingPolicy implements QueueOrderingPolicy {
+public class PriorityUtilizationQueueOrderingPolicy
+    implements QueueOrderingPolicy {
   private List<CSQueue> queues;
   private boolean respectPriority;
 
@@ -78,7 +81,7 @@ public class PriorityUtilizationQueueOrderingPolicy implements QueueOrderingPoli
     if (priority1 == priority2) {
       // The queue with less relative used-capacity goes first
       return Double.compare(relativeAssigned1, relativeAssigned2);
-    } else {
+    } else{
       // When priority is different:
       if ((relativeAssigned1 < 1.0f && relativeAssigned2 < 1.0f) || (
           relativeAssigned1 >= 1.0f && relativeAssigned2 >= 1.0f)) {
@@ -86,7 +89,7 @@ public class PriorityUtilizationQueueOrderingPolicy implements QueueOrderingPoli
         // Or both the queues are over or meeting their guaranteed capacities
         // queue with higher used-capacity goes first
         return Integer.compare(priority2, priority1);
-      } else {
+      } else{
         // Otherwise, when one of the queues is over or meeting their
         // guaranteed capacities and the other is under: The queue that is
         // under its capacity guarantee gets the resources.
@@ -98,7 +101,7 @@ public class PriorityUtilizationQueueOrderingPolicy implements QueueOrderingPoli
   /**
    * Comparator that both looks at priority and utilization
    */
-  private class PriorityQueueComparator implements  Comparator<CSQueue> {
+  private class PriorityQueueComparator implements Comparator<CSQueue> {
 
     @Override
     public int compare(CSQueue q1, CSQueue q2) {
@@ -109,8 +112,38 @@ public class PriorityUtilizationQueueOrderingPolicy implements QueueOrderingPoli
         return rc;
       }
 
-      float used1 = q1.getQueueCapacities().getUsedCapacity(p);
-      float used2 = q2.getQueueCapacities().getUsedCapacity(p);
+      float q1AbsCapacity = q1.getQueueCapacities().getAbsoluteCapacity(p);
+      float q2AbsCapacity = q2.getQueueCapacities().getAbsoluteCapacity(p);
+
+      //If q1's abs capacity > 0 and q2 is 0, then prioritize q1
+      if (Float.compare(q1AbsCapacity, 0f) > 0 && Float.compare(q2AbsCapacity,
+          0f) == 0) {
+        return -1;
+        //If q2's abs capacity > 0 and q1 is 0, then prioritize q2
+      } else if (Float.compare(q2AbsCapacity, 0f) > 0 && Float.compare(
+          q1AbsCapacity, 0f) == 0) {
+        return 1;
+      } else if (Float.compare(q1AbsCapacity, 0f) == 0 && Float.compare(
+          q2AbsCapacity, 0f) == 0) {
+        // both q1 has 0 and q2 has 0 capacity, then fall back to using
+        // priority, abs used capacity to prioritize
+        float used1 = q1.getQueueCapacities().getAbsoluteUsedCapacity(p);
+        float used2 = q2.getQueueCapacities().getAbsoluteUsedCapacity(p);
+
+        return compare(q1, q2, used1, used2, p);
+      } else{
+        // both q1 has positive abs capacity and q2 has positive abs
+        // capacity
+        float used1 = q1.getQueueCapacities().getUsedCapacity(p);
+        float used2 = q2.getQueueCapacities().getUsedCapacity(p);
+
+        return compare(q1, q2, used1, used2, p);
+      }
+    }
+
+    private int compare(CSQueue q1, CSQueue q2, float q1Used, float q2Used,
+        String partition) {
+
       int p1 = 0;
       int p2 = 0;
       if (respectPriority) {
@@ -118,29 +151,31 @@ public class PriorityUtilizationQueueOrderingPolicy implements QueueOrderingPoli
         p2 = q2.getPriority().getPriority();
       }
 
-      rc = PriorityUtilizationQueueOrderingPolicy.compare(used1, used2, p1, p2);
+      int rc = PriorityUtilizationQueueOrderingPolicy.compare(q1Used, q2Used,
+          p1, p2);
 
       // For queue with same used ratio / priority, queue with higher configured
       // capacity goes first
       if (0 == rc) {
-        Resource minEffRes1 = q1.getQueueResourceQuotas()
-            .getConfiguredMinResource(p);
-        Resource minEffRes2 = q2.getQueueResourceQuotas()
-            .getConfiguredMinResource(p);
-        if (!minEffRes1.equals(Resources.none())
-            && !minEffRes2.equals(Resources.none())) {
+        Resource minEffRes1 =
+            q1.getQueueResourceQuotas().getConfiguredMinResource(partition);
+        Resource minEffRes2 =
+            q2.getQueueResourceQuotas().getConfiguredMinResource(partition);
+        if (!minEffRes1.equals(Resources.none()) && !minEffRes2.equals(
+            Resources.none())) {
           return minEffRes2.compareTo(minEffRes1);
         }
 
-        float abs1 = q1.getQueueCapacities().getAbsoluteCapacity(p);
-        float abs2 = q2.getQueueCapacities().getAbsoluteCapacity(p);
+        float abs1 = q1.getQueueCapacities().getAbsoluteCapacity(partition);
+        float abs2 = q2.getQueueCapacities().getAbsoluteCapacity(partition);
         return Float.compare(abs2, abs1);
       }
 
       return rc;
     }
 
-    private int compareQueueAccessToPartition(CSQueue q1, CSQueue q2, String partition) {
+    private int compareQueueAccessToPartition(CSQueue q1, CSQueue q2,
+        String partition) {
       // Everybody has access to default partition
       if (StringUtils.equals(partition, RMNodeLabelsManager.NO_LABEL)) {
         return 0;
@@ -190,9 +225,11 @@ public class PriorityUtilizationQueueOrderingPolicy implements QueueOrderingPoli
   @Override
   public String getConfigName() {
     if (respectPriority) {
-      return CapacitySchedulerConfiguration.QUEUE_PRIORITY_UTILIZATION_ORDERING_POLICY;
+      return CapacitySchedulerConfiguration.
+          QUEUE_PRIORITY_UTILIZATION_ORDERING_POLICY;
     } else{
-      return CapacitySchedulerConfiguration.QUEUE_UTILIZATION_ORDERING_POLICY;
+      return CapacitySchedulerConfiguration.
+          QUEUE_UTILIZATION_ORDERING_POLICY;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8316df8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/policy/TestPriorityUtilizationQueueOrderingPolicy.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/policy/TestPriorityUtilizationQueueOrderingPolicy.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/policy/TestPriorityUtilizationQueueOrderingPolicy.java
index b9d5b82..ca9a84b 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/policy/TestPriorityUtilizationQueueOrderingPolicy.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/policy/TestPriorityUtilizationQueueOrderingPolicy.java
@@ -36,7 +36,7 @@ import static org.mockito.Mockito.when;
 
 public class TestPriorityUtilizationQueueOrderingPolicy {
   private List<CSQueue> mockCSQueues(String[] queueNames, int[] priorities,
-      float[] utilizations, String partition) {
+      float[] utilizations, float[] absCapacities, String partition) {
     // sanity check
     assert queueNames != null && priorities != null && utilizations != null
         && queueNames.length > 0 && queueNames.length == priorities.length
@@ -48,6 +48,7 @@ public class TestPriorityUtilizationQueueOrderingPolicy {
       when(q.getQueueName()).thenReturn(queueNames[i]);
 
       QueueCapacities qc = new QueueCapacities(false);
+      qc.setAbsoluteCapacity(partition, absCapacities[i]);
       qc.setUsedCapacity(partition, utilizations[i]);
 
       when(q.getQueueCapacities()).thenReturn(qc);
@@ -81,41 +82,45 @@ public class TestPriorityUtilizationQueueOrderingPolicy {
 
     // Case 1, one queue
     policy.setQueues(mockCSQueues(new String[] { "a" }, new int[] { 0 },
-        new float[] { 0.1f }, ""));
+        new float[] { 0.1f }, new float[] {0.2f},  ""));
     verifyOrder(policy, "", new String[] { "a" });
 
     // Case 2, 2 queues
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 0, 0 },
-        new float[] { 0.1f, 0.0f }, ""));
+        new float[] { 0.1f, 0.0f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "b", "a" });
 
     // Case 3, 3 queues
     policy.setQueues(
         mockCSQueues(new String[] { "a", "b", "c" }, new int[] { 0, 0, 0 },
-            new float[] { 0.1f, 0.0f, 0.2f }, ""));
+            new float[] { 0.1f, 0.0f, 0.2f }, new float[] {0.2f, 0.3f, 0.4f},
+            ""));
     verifyOrder(policy, "", new String[] { "b", "a", "c" });
 
     // Case 4, 3 queues, ignore priority
     policy.setQueues(
         mockCSQueues(new String[] { "a", "b", "c" }, new int[] { 2, 1, 0 },
-            new float[] { 0.1f, 0.0f, 0.2f }, ""));
+            new float[] { 0.1f, 0.0f, 0.2f }, new float[] {0.2f, 0.3f, 0.4f},
+            ""));
     verifyOrder(policy, "", new String[] { "b", "a", "c" });
 
     // Case 5, 3 queues, look at partition (default)
     policy.setQueues(
         mockCSQueues(new String[] { "a", "b", "c" }, new int[] { 2, 1, 0 },
-            new float[] { 0.1f, 0.0f, 0.2f }, "x"));
+            new float[] { 0.1f, 0.0f, 0.2f }, new float[] {0.2f, 0.3f, 0.4f},
+            "x"));
     verifyOrder(policy, "", new String[] { "a", "b", "c" });
 
     // Case 5, 3 queues, look at partition (x)
     policy.setQueues(
         mockCSQueues(new String[] { "a", "b", "c" }, new int[] { 2, 1, 0 },
-            new float[] { 0.1f, 0.0f, 0.2f }, "x"));
+            new float[] { 0.1f, 0.0f, 0.2f }, new float[] {0.2f, 0.3f, 0.4f},
+            "x"));
     verifyOrder(policy, "x", new String[] { "b", "a", "c" });
 
     // Case 6, 3 queues, with different accessibility to partition
     List<CSQueue> queues = mockCSQueues(new String[] { "a", "b", "c" }, new int[] { 2, 1, 0 },
-        new float[] { 0.1f, 0.0f, 0.2f }, "x");
+        new float[] { 0.1f, 0.0f, 0.2f }, new float[] {0.2f, 0.3f, 0.4f},  "x");
     // a can access "x"
     when(queues.get(0).getAccessibleNodeLabels()).thenReturn(ImmutableSet.of("x", "y"));
     // c can access "x"
@@ -131,89 +136,94 @@ public class TestPriorityUtilizationQueueOrderingPolicy {
 
     // Case 1, one queue
     policy.setQueues(mockCSQueues(new String[] { "a" }, new int[] { 1 },
-        new float[] { 0.1f }, ""));
+        new float[] { 0.1f }, new float[] {0.2f}, ""));
     verifyOrder(policy, "", new String[] { "a" });
 
     // Case 2, 2 queues, both under utilized, same priority
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 1 },
-        new float[] { 0.2f, 0.1f }, ""));
+        new float[] { 0.2f, 0.1f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "b", "a" });
 
     // Case 3, 2 queues, both over utilized, same priority
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 1 },
-        new float[] { 1.1f, 1.2f }, ""));
+        new float[] { 1.1f, 1.2f },new float[] {0.2f, 0.3f},  ""));
     verifyOrder(policy, "", new String[] { "a", "b" });
 
     // Case 4, 2 queues, one under and one over, same priority
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 1 },
-        new float[] { 0.1f, 1.2f }, ""));
+        new float[] { 0.1f, 1.2f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "a", "b" });
 
     // Case 5, 2 queues, both over utilized, different priority
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 2 },
-        new float[] { 1.1f, 1.2f }, ""));
+        new float[] { 1.1f, 1.2f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "b", "a" });
 
     // Case 6, 2 queues, both under utilized, different priority
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 2 },
-        new float[] { 0.1f, 0.2f }, ""));
+        new float[] { 0.1f, 0.2f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "b", "a" });
 
     // Case 7, 2 queues, one under utilized and one over utilized,
     // different priority (1)
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 2 },
-        new float[] { 0.1f, 1.2f }, ""));
+        new float[] { 0.1f, 1.2f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "a", "b" });
 
     // Case 8, 2 queues, one under utilized and one over utilized,
     // different priority (1)
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 2, 1 },
-        new float[] { 0.1f, 1.2f }, ""));
+        new float[] { 0.1f, 1.2f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "a", "b" });
 
     // Case 9, 2 queues, one under utilized and one meet, different priority (1)
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 2 },
-        new float[] { 0.1f, 1.0f }, ""));
+        new float[] { 0.1f, 1.0f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "a", "b" });
 
     // Case 10, 2 queues, one under utilized and one meet, different priority (2)
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 2, 1 },
-        new float[] { 0.1f, 1.0f }, ""));
+        new float[] { 0.1f, 1.0f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "a", "b" });
 
     // Case 11, 2 queues, one under utilized and one meet, same priority
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 1 },
-        new float[] { 0.1f, 1.0f }, ""));
+        new float[] { 0.1f, 1.0f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "a", "b" });
 
     // Case 12, 2 queues, both meet, different priority
     policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 2 },
-        new float[] { 1.0f, 1.0f }, ""));
+        new float[] { 1.0f, 1.0f }, new float[] {0.2f, 0.3f}, ""));
     verifyOrder(policy, "", new String[] { "b", "a" });
 
     // Case 13, 5 queues, different priority
     policy.setQueues(mockCSQueues(new String[] { "a", "b", "c", "d", "e" },
         new int[] { 1, 2, 0, 0, 3 },
-        new float[] { 1.2f, 1.0f, 0.2f, 1.1f, 0.2f }, ""));
+        new float[] { 1.2f, 1.0f, 0.2f, 1.1f, 0.2f },
+        new float[] { 0.2f, 0.1f, 0.1f, 0.3f, 0.3f }, ""));
     verifyOrder(policy, "", new String[] { "e", "c", "b", "a", "d" });
 
-    // Case 14, 5 queues, different priority, partition default;
+    // Case 14, 5 queues, different priority,
+    // partition default - abs capacity is 0;
     policy.setQueues(mockCSQueues(new String[] { "a", "b", "c", "d", "e" },
         new int[] { 1, 2, 0, 0, 3 },
-        new float[] { 1.2f, 1.0f, 0.2f, 1.1f, 0.2f }, "x"));
+        new float[] { 1.2f, 1.0f, 0.2f, 1.1f, 0.2f },
+        new float[] { 0.2f, 0.1f, 0.1f, 0.3f, 0.3f }, "x"));
     verifyOrder(policy, "", new String[] { "e", "b", "a", "c", "d" });
 
     // Case 15, 5 queues, different priority, partition x;
     policy.setQueues(mockCSQueues(new String[] { "a", "b", "c", "d", "e" },
         new int[] { 1, 2, 0, 0, 3 },
-        new float[] { 1.2f, 1.0f, 0.2f, 1.1f, 0.2f }, "x"));
+        new float[] { 1.2f, 1.0f, 0.2f, 1.1f, 0.2f },
+        new float[] { 0.2f, 0.1f, 0.1f, 0.3f, 0.3f }, "x"));
     verifyOrder(policy, "x", new String[] { "e", "c", "b", "a", "d" });
 
     // Case 16, 5 queues, different priority, partition x; and different
     // accessibility
     List<CSQueue> queues = mockCSQueues(new String[] { "a", "b", "c", "d", "e" },
         new int[] { 1, 2, 0, 0, 3 },
-        new float[] { 1.2f, 1.0f, 0.2f, 1.1f, 0.2f }, "x");
+        new float[] { 1.2f, 1.0f, 0.2f, 1.1f, 0.2f },
+        new float[] { 0.2f, 0.1f, 0.1f, 0.3f, 0.3f }, "x");
     // Only a/d has access to x
     when(queues.get(0).getAccessibleNodeLabels()).thenReturn(
         ImmutableSet.of("x"));
@@ -221,5 +231,23 @@ public class TestPriorityUtilizationQueueOrderingPolicy {
         ImmutableSet.of("x"));
     policy.setQueues(queues);
     verifyOrder(policy, "x", new String[] { "a", "d", "e", "c", "b" });
+
+    // Case 17, 2 queues, one's abs capacity is 0
+    policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 1 },
+        new float[] { 0.1f, 1.2f }, new float[] {0.0f, 0.3f}, ""));
+    verifyOrder(policy, "", new String[] { "b", "a" });
+
+    // Case 18, 2 queues, one's abs capacity is 0
+    policy.setQueues(mockCSQueues(new String[] { "a", "b" }, new int[] { 1, 1 },
+        new float[] { 0.1f, 1.2f }, new float[] {0.3f, 0.0f}, ""));
+    verifyOrder(policy, "", new String[] { "a", "b" });
+
+    //Case 19, 5 queues with 2 having abs capacity 0 are prioritized last
+    policy.setQueues(mockCSQueues(new String[] { "a", "b", "c", "d", "e" },
+        new int[] { 1, 2, 0, 0, 3 },
+        new float[] { 1.2f, 1.0f, 0.2f, 1.1f, 0.2f },
+        new float[] { 0.0f, 0.0f, 0.1f, 0.3f, 0.3f }, "x"));
+    verifyOrder(policy, "x", new String[] { "e", "c", "d", "b", "a" });
+
   }
 }


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


[34/50] [abbrv] hadoop git commit: HADOOP-15059. Undoing the switch of Credentials to PB format as default - done via HADOOP-12563 for supporting 2.x to 3.x upgrades.

Posted by wa...@apache.org.
HADOOP-15059. Undoing the switch of Credentials to PB format as default - done via HADOOP-12563 for supporting 2.x to 3.x upgrades.


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

Branch: refs/heads/YARN-6592
Commit: f19638333b11da6dcab9a964e73a49947b8390fd
Parents: ce04340
Author: Vinod Kumar Vavilapalli (I am also known as @tshooter.) <vi...@apache.org>
Authored: Fri Dec 8 08:00:21 2017 -0800
Committer: Vinod Kumar Vavilapalli (I am also known as @tshooter.) <vi...@apache.org>
Committed: Fri Dec 8 08:00:21 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/security/Credentials.java | 112 ++++++++++++-------
 .../hadoop/security/token/DtFileOperations.java |  10 +-
 .../hadoop/security/token/TestDtUtilShell.java  |  15 +--
 .../hdfs/tools/DelegationTokenFetcher.java      |   3 +-
 4 files changed, 86 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f1963833/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
index 4d58981..3e51249 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
@@ -27,7 +27,6 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
@@ -60,6 +59,28 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class Credentials implements Writable {
+  public enum SerializedFormat {
+    WRITABLE((byte) 0x00),
+    PROTOBUF((byte) 0x01);
+
+    // Caching to avoid reconstructing the array each time.
+    private static final SerializedFormat[] FORMATS = values();
+
+    final byte value;
+
+    SerializedFormat(byte val) {
+      this.value = val;
+    }
+
+    public static SerializedFormat valueOf(int val) {
+      try {
+        return FORMATS[val];
+      } catch (ArrayIndexOutOfBoundsException e) {
+        throw new IllegalArgumentException("Unknown credential format: " + val);
+      }
+    }
+  }
+
   private static final Logger LOG = LoggerFactory.getLogger(Credentials.class);
 
   private  Map<Text, byte[]> secretKeysMap = new HashMap<Text, byte[]>();
@@ -224,63 +245,74 @@ public class Credentials implements Writable {
     if (!Arrays.equals(magic, TOKEN_STORAGE_MAGIC)) {
       throw new IOException("Bad header found in token storage.");
     }
-    byte version = in.readByte();
-    if (version != TOKEN_STORAGE_VERSION &&
-        version != OLD_TOKEN_STORAGE_VERSION) {
-      throw new IOException("Unknown version " + version +
-                            " in token storage.");
+    SerializedFormat format;
+    try {
+      format = SerializedFormat.valueOf(in.readByte());
+    } catch (IllegalArgumentException e) {
+      throw new IOException(e);
     }
-    if (version == OLD_TOKEN_STORAGE_VERSION) {
+    switch (format) {
+    case WRITABLE:
       readFields(in);
-    } else if (version == TOKEN_STORAGE_VERSION) {
+      break;
+    case PROTOBUF:
       readProto(in);
+      break;
+    default:
+      throw new IOException("Unsupported format " + format);
     }
   }
 
   private static final byte[] TOKEN_STORAGE_MAGIC =
       "HDTS".getBytes(StandardCharsets.UTF_8);
-  private static final byte TOKEN_STORAGE_VERSION = 1;
-
-  /**
-   *  For backward compatibility.
-   */
-  private static final byte OLD_TOKEN_STORAGE_VERSION = 0;
-
 
   public void writeTokenStorageToStream(DataOutputStream os)
       throws IOException {
-    os.write(TOKEN_STORAGE_MAGIC);
-    os.write(TOKEN_STORAGE_VERSION);
-    writeProto(os);
+    // by default store in the oldest supported format for compatibility
+    writeTokenStorageToStream(os, SerializedFormat.WRITABLE);
   }
 
-  public void writeTokenStorageFile(Path filename,
-                                    Configuration conf) throws IOException {
-    FSDataOutputStream os = filename.getFileSystem(conf).create(filename);
-    writeTokenStorageToStream(os);
-    os.close();
+  public void writeTokenStorageToStream(DataOutputStream os,
+      SerializedFormat format) throws IOException {
+    switch (format) {
+    case WRITABLE:
+      writeWritableOutputStream(os);
+      break;
+    case PROTOBUF:
+      writeProtobufOutputStream(os);
+      break;
+    default:
+      throw new IllegalArgumentException("Unsupported serialized format: "
+          + format);
+    }
   }
 
-  /**
-   *  For backward compatibility.
-   */
-  public void writeLegacyTokenStorageLocalFile(File f) throws IOException {
-    writeLegacyOutputStream(new DataOutputStream(new FileOutputStream(f)));
+  private void writeWritableOutputStream(DataOutputStream os)
+      throws IOException {
+    os.write(TOKEN_STORAGE_MAGIC);
+    os.write(SerializedFormat.WRITABLE.value);
+    write(os);
   }
 
-  /**
-   *  For backward compatibility.
-   */
-  public void writeLegacyTokenStorageFile(Path filename, Configuration conf)
+  private void writeProtobufOutputStream(DataOutputStream os)
       throws IOException {
-    writeLegacyOutputStream(filename.getFileSystem(conf).create(filename));
+    os.write(TOKEN_STORAGE_MAGIC);
+    os.write(SerializedFormat.PROTOBUF.value);
+    writeProto(os);
   }
 
-  private void writeLegacyOutputStream(DataOutputStream os) throws IOException {
-    os.write(TOKEN_STORAGE_MAGIC);
-    os.write(OLD_TOKEN_STORAGE_VERSION);
-    write(os);
-    os.close();
+  public void writeTokenStorageFile(Path filename,
+                                    Configuration conf) throws IOException {
+    // by default store in the oldest supported format for compatibility
+    writeTokenStorageFile(filename, conf, SerializedFormat.WRITABLE);
+  }
+
+  public void writeTokenStorageFile(Path filename, Configuration conf,
+      SerializedFormat format) throws IOException {
+    try (FSDataOutputStream os =
+             filename.getFileSystem(conf).create(filename)) {
+      writeTokenStorageToStream(os, format);
+    }
   }
 
   /**
@@ -312,7 +344,7 @@ public class Credentials implements Writable {
    * @param out
    * @throws IOException
    */
-  public void writeProto(DataOutput out) throws IOException {
+  void writeProto(DataOutput out) throws IOException {
     CredentialsProto.Builder storage = CredentialsProto.newBuilder();
     for (Map.Entry<Text, Token<? extends TokenIdentifier>> e :
                                                          tokenMap.entrySet()) {
@@ -337,7 +369,7 @@ public class Credentials implements Writable {
    * Populates keys/values from proto buffer storage.
    * @param in - stream ready to read a serialized proto buffer message
    */
-  public void readProto(DataInput in) throws IOException {
+  void readProto(DataInput in) throws IOException {
     CredentialsProto storage = CredentialsProto.parseDelimitedFrom((DataInputStream)in);
     for (CredentialsKVProto kv : storage.getTokensList()) {
       addToken(new Text(kv.getAliasBytes().toByteArray()),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f1963833/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java
index d128cc9..d36ad9b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java
@@ -102,11 +102,13 @@ public final class DtFileOperations {
   public static void doFormattedWrite(
       File f, String format, Credentials creds, Configuration conf)
       throws IOException {
-    if (format == null || format.equals(FORMAT_PB)) {
-      creds.writeTokenStorageFile(fileToPath(f), conf);
-    } else { // if (format != null && format.equals(FORMAT_JAVA)) {
-      creds.writeLegacyTokenStorageLocalFile(f);
+    // default to oldest supported format for compatibility
+    Credentials.SerializedFormat credsFormat =
+        Credentials.SerializedFormat.WRITABLE;
+    if (format.equals(FORMAT_PB)) {
+      credsFormat = Credentials.SerializedFormat.PROTOBUF;
     }
+    creds.writeTokenStorageFile(fileToPath(f), conf, credsFormat);
   }
 
   /** Print out a Credentials file from the local filesystem.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f1963833/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
index 1c25912..53b0d3d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.security.token;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.FileInputStream;
-import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 
@@ -29,14 +28,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.security.token.DtFetcher;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -54,7 +51,6 @@ public class TestDtUtilShell {
   private static Configuration defaultConf = new Configuration();
   private static FileSystem localFs = null;
   private final String alias = "proxy_ip:1234";
-  private final String renewer = "yarn";
   private final String getUrl = SERVICE_GET.toString() + "://localhost:9000/";
   private final String getUrl2 = "http://localhost:9000/";
   public static Text SERVICE_GET = new Text("testTokenServiceGet");
@@ -111,11 +107,12 @@ public class TestDtUtilShell {
     Token<? extends TokenIdentifier> tok = (Token<? extends TokenIdentifier>)
         new Token(IDENTIFIER, PASSWORD, KIND, service);
     creds.addToken(tok.getService(), tok);
+    Credentials.SerializedFormat format =
+        Credentials.SerializedFormat.PROTOBUF;
     if (legacy) {
-      creds.writeLegacyTokenStorageLocalFile(new File(tokenPath.toString()));
-    } else {
-      creds.writeTokenStorageFile(tokenPath, defaultConf);
+      format = Credentials.SerializedFormat.WRITABLE;
     }
+    creds.writeTokenStorageFile(tokenPath, defaultConf, format);
   }
 
   @Test
@@ -284,6 +281,6 @@ public class TestDtUtilShell {
     DataInputStream in = new DataInputStream(
         new FileInputStream(tokenFilenameGet));
     spyCreds.readTokenStorageStream(in);
-    Mockito.verify(spyCreds).readProto(in);
+    Mockito.verify(spyCreds, Mockito.never()).readFields(in);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f1963833/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
index cf77804..c6ea91c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
@@ -182,7 +182,8 @@ public class DelegationTokenFetcher {
       Credentials cred = new Credentials();
       cred.addToken(token.getService(), token);
       // dtutil is replacing this tool; preserve legacy functionality
-      cred.writeLegacyTokenStorageFile(tokenFile, conf);
+      cred.writeTokenStorageFile(tokenFile, conf,
+          Credentials.SerializedFormat.WRITABLE);
 
       if (LOG.isDebugEnabled()) {
         LOG.debug("Fetched token " + fs.getUri() + " for " +


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


[24/50] [abbrv] hadoop git commit: YARN-6471. Support to add min/max resource configuration for a queue. (Sunil G via wangda)

Posted by wa...@apache.org.
YARN-6471. Support to add min/max resource configuration for a queue. (Sunil G via wangda)

Change-Id: I9213f5297a6841fab5c573e85ee4c4e5f4a0b7ff


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

Branch: refs/heads/YARN-6592
Commit: 5e798b1a0ddceeaf54703b94052501867156e979
Parents: d6c31a3
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Aug 11 10:30:23 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 7 18:56:54 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/util/StringUtils.java     |  31 ++
 .../resource/DefaultResourceCalculator.java     |   6 +
 .../resource/DominantResourceCalculator.java    |   7 +
 .../yarn/util/resource/ResourceCalculator.java  |  12 +
 .../hadoop/yarn/util/resource/Resources.java    |   5 +
 .../capacity/FifoCandidatesSelector.java        |   9 +-
 .../ProportionalCapacityPreemptionPolicy.java   |  10 +-
 .../monitor/capacity/TempQueuePerPartition.java |  16 +-
 .../scheduler/AbstractResourceUsage.java        | 198 +++++++
 .../scheduler/QueueResourceQuotas.java          | 153 ++++++
 .../scheduler/ResourceUsage.java                | 237 ++-------
 .../scheduler/capacity/AbstractCSQueue.java     | 162 +++++-
 .../scheduler/capacity/CSQueue.java             |  39 ++
 .../scheduler/capacity/CSQueueUtils.java        |  24 +-
 .../CapacitySchedulerConfiguration.java         | 179 ++++++-
 .../scheduler/capacity/LeafQueue.java           |  31 +-
 .../scheduler/capacity/ParentQueue.java         | 201 +++++++-
 .../scheduler/capacity/UsersManager.java        |   5 +-
 .../PriorityUtilizationQueueOrderingPolicy.java |  11 +
 .../webapp/dao/CapacitySchedulerQueueInfo.java  |  15 +
 .../yarn/server/resourcemanager/MockNM.java     |   8 +
 .../yarn/server/resourcemanager/MockRM.java     |   6 +
 ...alCapacityPreemptionPolicyMockFramework.java |  13 +
 ...estProportionalCapacityPreemptionPolicy.java |  28 +-
 ...pacityPreemptionPolicyIntraQueueWithDRF.java |   6 +-
 .../TestAbsoluteResourceConfiguration.java      | 516 +++++++++++++++++++
 .../capacity/TestApplicationLimits.java         |  30 +-
 .../TestApplicationLimitsByPartition.java       |   4 +
 .../capacity/TestCapacityScheduler.java         |   2 +-
 .../scheduler/capacity/TestChildQueueOrder.java |   2 +
 .../scheduler/capacity/TestLeafQueue.java       | 261 ++++------
 .../scheduler/capacity/TestParentQueue.java     |   8 +
 .../scheduler/capacity/TestReservations.java    |  19 +
 ...tPriorityUtilizationQueueOrderingPolicy.java |   3 +
 .../webapp/TestRMWebServicesCapacitySched.java  |   4 +-
 35 files changed, 1831 insertions(+), 430 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
index cda5ec7..1be8a08 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
@@ -1152,4 +1152,35 @@ public class StringUtils {
     return s1.equalsIgnoreCase(s2);
   }
 
+  /**
+   * <p>Checks if the String contains only unicode letters.</p>
+   *
+   * <p><code>null</code> will return <code>false</code>.
+   * An empty String (length()=0) will return <code>true</code>.</p>
+   *
+   * <pre>
+   * StringUtils.isAlpha(null)   = false
+   * StringUtils.isAlpha("")     = true
+   * StringUtils.isAlpha("  ")   = false
+   * StringUtils.isAlpha("abc")  = true
+   * StringUtils.isAlpha("ab2c") = false
+   * StringUtils.isAlpha("ab-c") = false
+   * </pre>
+   *
+   * @param str  the String to check, may be null
+   * @return <code>true</code> if only contains letters, and is non-null
+   */
+  public static boolean isAlpha(String str) {
+      if (str == null) {
+          return false;
+      }
+      int sz = str.length();
+      for (int i = 0; i < sz; i++) {
+          if (Character.isLetter(str.charAt(i)) == false) {
+              return false;
+          }
+      }
+      return true;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
index 7f155e7..aefa85c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
@@ -131,4 +131,10 @@ public class DefaultResourceCalculator extends ResourceCalculator {
   public boolean isAnyMajorResourceZero(Resource resource) {
     return resource.getMemorySize() == 0f;
   }
+
+  @Override
+  public Resource normalizeDown(Resource r, Resource stepFactor) {
+    return Resources.createResource(
+        roundDown((r.getMemorySize()), stepFactor.getMemorySize()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index 6b284e3..16e4527 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -567,4 +567,11 @@ public class DominantResourceCalculator extends ResourceCalculator {
     }
     return false;
   }
+
+  @Override
+  public Resource normalizeDown(Resource r, Resource stepFactor) {
+    return Resources.createResource(
+        roundDown(r.getMemorySize(), stepFactor.getMemorySize()),
+        roundDown(r.getVirtualCores(), stepFactor.getVirtualCores()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
index d59560f..a816290 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
@@ -235,4 +235,16 @@ public abstract class ResourceCalculator {
    * @return returns true if any resource is zero.
    */
   public abstract boolean isAnyMajorResourceZero(Resource resource);
+
+  /**
+   * Get resource <code>r</code>and normalize down using step-factor
+   * <code>stepFactor</code>.
+   *
+   * @param r
+   *          resource to be multiplied
+   * @param stepFactor
+   *          factor by which to normalize down
+   * @return resulting normalized resource
+   */
+  public abstract Resource normalizeDown(Resource r, Resource stepFactor);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index 068e7f1..0426ab7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -546,4 +546,9 @@ public class Resources {
       Resource resource) {
     return rc.isAnyMajorResourceZero(resource);
   }
+
+  public static Resource normalizeDown(ResourceCalculator calculator,
+      Resource resource, Resource factor) {
+    return calculator.normalizeDown(resource, factor);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.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/monitor/capacity/FifoCandidatesSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java
index f843db4..748548a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java
@@ -140,10 +140,10 @@ public class FifoCandidatesSelector
         // Can try preempting AMContainers (still saving atmost
         // maxAMCapacityForThisQueue AMResource's) if more resources are
         // required to be preemptionCandidates from this Queue.
-        Resource maxAMCapacityForThisQueue = Resources.multiply(
-            Resources.multiply(clusterResource,
-                leafQueue.getAbsoluteCapacity()),
-            leafQueue.getMaxAMResourcePerQueuePercent());
+        Resource maxAMCapacityForThisQueue = Resources
+            .multiply(
+                leafQueue.getEffectiveCapacity(RMNodeLabelsManager.NO_LABEL),
+                leafQueue.getMaxAMResourcePerQueuePercent());
 
         preemptAMContainers(clusterResource, selectedCandidates, skippedAMContainerlist,
             resToObtainByPartition, skippedAMSize, maxAMCapacityForThisQueue,
@@ -199,7 +199,6 @@ public class FifoCandidatesSelector
    * Given a target preemption for a specific application, select containers
    * to preempt (after unreserving all reservation for that app).
    */
-  @SuppressWarnings("unchecked")
   private void preemptFrom(FiCaSchedulerApp app,
       Resource clusterResource, Map<String, Resource> resToObtainByPartition,
       List<RMContainer> skippedAMContainerlist, Resource skippedAMSize,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.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/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 2c072d2..8327cb9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolic
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
@@ -525,6 +526,13 @@ public class ProportionalCapacityPreemptionPolicy
       float absMaxCap = qc.getAbsoluteMaximumCapacity(partitionToLookAt);
       boolean preemptionDisabled = curQueue.getPreemptionDisabled();
 
+      QueueResourceQuotas queueResourceQuotas = curQueue
+          .getQueueResourceQuotas();
+      Resource effMinRes = queueResourceQuotas
+          .getEffectiveMinResource(partitionToLookAt);
+      Resource effMaxRes = queueResourceQuotas
+          .getEffectiveMaxResource(partitionToLookAt);
+
       Resource current = Resources
           .clone(curQueue.getQueueResourceUsage().getUsed(partitionToLookAt));
       Resource killable = Resources.none();
@@ -550,7 +558,7 @@ public class ProportionalCapacityPreemptionPolicy
 
       ret = new TempQueuePerPartition(queueName, current, preemptionDisabled,
           partitionToLookAt, killable, absCap, absMaxCap, partitionResource,
-          reserved, curQueue);
+          reserved, curQueue, effMinRes, effMaxRes);
 
       if (curQueue instanceof ParentQueue) {
         String configuredOrderingPolicy =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.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/monitor/capacity/TempQueuePerPartition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
index 89452f9..bd236fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
@@ -48,6 +48,9 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
 
   double normalizedGuarantee;
 
+  private Resource effMinRes;
+  private Resource effMaxRes;
+
   final ArrayList<TempQueuePerPartition> children;
   private Collection<TempAppPerPartition> apps;
   LeafQueue leafQueue;
@@ -68,7 +71,8 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
   TempQueuePerPartition(String queueName, Resource current,
       boolean preemptionDisabled, String partition, Resource killable,
       float absCapacity, float absMaxCapacity, Resource totalPartitionResource,
-      Resource reserved, CSQueue queue) {
+      Resource reserved, CSQueue queue, Resource effMinRes,
+      Resource effMaxRes) {
     super(queueName, current, Resource.newInstance(0, 0), reserved,
         Resource.newInstance(0, 0));
 
@@ -95,6 +99,8 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
     this.absCapacity = absCapacity;
     this.absMaxCapacity = absMaxCapacity;
     this.totalPartitionResource = totalPartitionResource;
+    this.effMinRes = effMinRes;
+    this.effMaxRes = effMaxRes;
   }
 
   public void setLeafQueue(LeafQueue l) {
@@ -177,10 +183,18 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
   }
 
   public Resource getGuaranteed() {
+    if(!effMinRes.equals(Resources.none())) {
+      return Resources.clone(effMinRes);
+    }
+
     return Resources.multiply(totalPartitionResource, absCapacity);
   }
 
   public Resource getMax() {
+    if(!effMaxRes.equals(Resources.none())) {
+      return Resources.clone(effMaxRes);
+    }
+
     return Resources.multiply(totalPartitionResource, absMaxCapacity);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.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/AbstractResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java
new file mode 100644
index 0000000..c295323
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java
@@ -0,0 +1,198 @@
+/**
+ * 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.scheduler;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+/**
+ * This class can be used to track resource usage in queue/user/app.
+ *
+ * And it is thread-safe
+ */
+public class AbstractResourceUsage {
+  protected ReadLock readLock;
+  protected WriteLock writeLock;
+  protected Map<String, UsageByLabel> usages;
+  // short for no-label :)
+  private static final String NL = CommonNodeLabelsManager.NO_LABEL;
+
+  public AbstractResourceUsage() {
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+
+    usages = new HashMap<String, UsageByLabel>();
+    usages.put(NL, new UsageByLabel(NL));
+  }
+
+  // Usage enum here to make implement cleaner
+  public enum ResourceType {
+    // CACHED_USED and CACHED_PENDING may be read by anyone, but must only
+    // be written by ordering policies
+    USED(0), PENDING(1), AMUSED(2), RESERVED(3), CACHED_USED(4), CACHED_PENDING(
+        5), AMLIMIT(6), MIN_RESOURCE(7), MAX_RESOURCE(8), EFF_MIN_RESOURCE(
+            9), EFF_MAX_RESOURCE(
+                10), EFF_MIN_RESOURCE_UP(11), EFF_MAX_RESOURCE_UP(12);
+
+    private int idx;
+
+    private ResourceType(int value) {
+      this.idx = value;
+    }
+  }
+
+  public static class UsageByLabel {
+    // usage by label, contains all UsageType
+    private Resource[] resArr;
+
+    public UsageByLabel(String label) {
+      resArr = new Resource[ResourceType.values().length];
+      for (int i = 0; i < resArr.length; i++) {
+        resArr[i] = Resource.newInstance(0, 0);
+      };
+    }
+
+    public Resource getUsed() {
+      return resArr[ResourceType.USED.idx];
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("{used=" + resArr[0] + "%, ");
+      sb.append("pending=" + resArr[1] + "%, ");
+      sb.append("am_used=" + resArr[2] + "%, ");
+      sb.append("reserved=" + resArr[3] + "%}");
+      sb.append("min_eff=" + resArr[9] + "%, ");
+      sb.append("max_eff=" + resArr[10] + "%}");
+      sb.append("min_effup=" + resArr[11] + "%, ");
+      return sb.toString();
+    }
+  }
+
+  private static Resource normalize(Resource res) {
+    if (res == null) {
+      return Resources.none();
+    }
+    return res;
+  }
+
+  protected Resource _get(String label, ResourceType type) {
+    if (label == null) {
+      label = RMNodeLabelsManager.NO_LABEL;
+    }
+
+    try {
+      readLock.lock();
+      UsageByLabel usage = usages.get(label);
+      if (null == usage) {
+        return Resources.none();
+      }
+      return normalize(usage.resArr[type.idx]);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  protected Resource _getAll(ResourceType type) {
+    try {
+      readLock.lock();
+      Resource allOfType = Resources.createResource(0);
+      for (Map.Entry<String, UsageByLabel> usageEntry : usages.entrySet()) {
+        //all usages types are initialized
+        Resources.addTo(allOfType, usageEntry.getValue().resArr[type.idx]);
+      }
+      return allOfType;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  private UsageByLabel getAndAddIfMissing(String label) {
+    if (label == null) {
+      label = RMNodeLabelsManager.NO_LABEL;
+    }
+    if (!usages.containsKey(label)) {
+      UsageByLabel u = new UsageByLabel(label);
+      usages.put(label, u);
+      return u;
+    }
+
+    return usages.get(label);
+  }
+
+  protected void _set(String label, ResourceType type, Resource res) {
+    try {
+      writeLock.lock();
+      UsageByLabel usage = getAndAddIfMissing(label);
+      usage.resArr[type.idx] = res;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  protected void _inc(String label, ResourceType type, Resource res) {
+    try {
+      writeLock.lock();
+      UsageByLabel usage = getAndAddIfMissing(label);
+      Resources.addTo(usage.resArr[type.idx], res);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  protected void _dec(String label, ResourceType type, Resource res) {
+    try {
+      writeLock.lock();
+      UsageByLabel usage = getAndAddIfMissing(label);
+      Resources.subtractFrom(usage.resArr[type.idx], res);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public String toString() {
+    try {
+      readLock.lock();
+      return usages.toString();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public Set<String> getNodePartitionsSet() {
+    try {
+      readLock.lock();
+      return usages.keySet();
+    } finally {
+      readLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueResourceQuotas.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/QueueResourceQuotas.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueResourceQuotas.java
new file mode 100644
index 0000000..2e653fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueResourceQuotas.java
@@ -0,0 +1,153 @@
+/**
+ * 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.scheduler;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+
+/**
+ * QueueResourceQuotas by Labels for following fields by label
+ * - EFFECTIVE_MIN_CAPACITY
+ * - EFFECTIVE_MAX_CAPACITY
+ * This class can be used to track resource usage in queue/user/app.
+ *
+ * And it is thread-safe
+ */
+public class QueueResourceQuotas extends AbstractResourceUsage {
+  // short for no-label :)
+  private static final String NL = CommonNodeLabelsManager.NO_LABEL;
+
+  public QueueResourceQuotas() {
+    super();
+  }
+
+  /*
+   * Configured Minimum Resource
+   */
+  public Resource getConfiguredMinResource() {
+    return _get(NL, ResourceType.MIN_RESOURCE);
+  }
+
+  public Resource getConfiguredMinResource(String label) {
+    return _get(label, ResourceType.MIN_RESOURCE);
+  }
+
+  public void setConfiguredMinResource(String label, Resource res) {
+    _set(label, ResourceType.MIN_RESOURCE, res);
+  }
+
+  public void setConfiguredMinResource(Resource res) {
+    _set(NL, ResourceType.MIN_RESOURCE, res);
+  }
+
+  /*
+   * Configured Maximum Resource
+   */
+  public Resource getConfiguredMaxResource() {
+    return getConfiguredMaxResource(NL);
+  }
+
+  public Resource getConfiguredMaxResource(String label) {
+    return _get(label, ResourceType.MAX_RESOURCE);
+  }
+
+  public void setConfiguredMaxResource(Resource res) {
+    setConfiguredMaxResource(NL, res);
+  }
+
+  public void setConfiguredMaxResource(String label, Resource res) {
+    _set(label, ResourceType.MAX_RESOURCE, res);
+  }
+
+  /*
+   * Effective Minimum Resource
+   */
+  public Resource getEffectiveMinResource() {
+    return _get(NL, ResourceType.EFF_MIN_RESOURCE);
+  }
+
+  public Resource getEffectiveMinResource(String label) {
+    return _get(label, ResourceType.EFF_MIN_RESOURCE);
+  }
+
+  public void setEffectiveMinResource(String label, Resource res) {
+    _set(label, ResourceType.EFF_MIN_RESOURCE, res);
+  }
+
+  public void setEffectiveMinResource(Resource res) {
+    _set(NL, ResourceType.EFF_MIN_RESOURCE, res);
+  }
+
+  /*
+   * Effective Maximum Resource
+   */
+  public Resource getEffectiveMaxResource() {
+    return getEffectiveMaxResource(NL);
+  }
+
+  public Resource getEffectiveMaxResource(String label) {
+    return _get(label, ResourceType.EFF_MAX_RESOURCE);
+  }
+
+  public void setEffectiveMaxResource(Resource res) {
+    setEffectiveMaxResource(NL, res);
+  }
+
+  public void setEffectiveMaxResource(String label, Resource res) {
+    _set(label, ResourceType.EFF_MAX_RESOURCE, res);
+  }
+
+  /*
+   * Effective Minimum Resource
+   */
+  public Resource getEffectiveMinResourceUp() {
+    return _get(NL, ResourceType.EFF_MIN_RESOURCE_UP);
+  }
+
+  public Resource getEffectiveMinResourceUp(String label) {
+    return _get(label, ResourceType.EFF_MIN_RESOURCE_UP);
+  }
+
+  public void setEffectiveMinResourceUp(String label, Resource res) {
+    _set(label, ResourceType.EFF_MIN_RESOURCE_UP, res);
+  }
+
+  public void setEffectiveMinResourceUp(Resource res) {
+    _set(NL, ResourceType.EFF_MIN_RESOURCE_UP, res);
+  }
+
+  /*
+   * Effective Maximum Resource
+   */
+  public Resource getEffectiveMaxResourceUp() {
+    return getEffectiveMaxResourceUp(NL);
+  }
+
+  public Resource getEffectiveMaxResourceUp(String label) {
+    return _get(label, ResourceType.EFF_MAX_RESOURCE_UP);
+  }
+
+  public void setEffectiveMaxResourceUp(Resource res) {
+    setEffectiveMaxResourceUp(NL, res);
+  }
+
+  public void setEffectiveMaxResourceUp(String label, Resource res) {
+    _set(label, ResourceType.EFF_MAX_RESOURCE_UP, res);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.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/ResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
index 6f0c7d2..ede4aec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
@@ -39,63 +39,12 @@ import org.apache.hadoop.yarn.util.resource.Resources;
  * 
  * And it is thread-safe
  */
-public class ResourceUsage {
-  private ReadLock readLock;
-  private WriteLock writeLock;
-  private Map<String, UsageByLabel> usages;
+public class ResourceUsage extends AbstractResourceUsage {
   // short for no-label :)
   private static final String NL = CommonNodeLabelsManager.NO_LABEL;
-  private final UsageByLabel usageNoLabel;
 
   public ResourceUsage() {
-    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    readLock = lock.readLock();
-    writeLock = lock.writeLock();
-
-    usages = new HashMap<String, UsageByLabel>();
-    usageNoLabel = new UsageByLabel(NL);
-    usages.put(NL, usageNoLabel);
-  }
-
-  // Usage enum here to make implement cleaner
-  private enum ResourceType {
-    //CACHED_USED and CACHED_PENDING may be read by anyone, but must only
-    //be written by ordering policies
-    USED(0), PENDING(1), AMUSED(2), RESERVED(3), CACHED_USED(4),
-      CACHED_PENDING(5), AMLIMIT(6);
-
-    private int idx;
-
-    private ResourceType(int value) {
-      this.idx = value;
-    }
-  }
-
-  private static class UsageByLabel {
-    // usage by label, contains all UsageType
-    private Resource[] resArr;
-
-    public UsageByLabel(String label) {
-      resArr = new Resource[ResourceType.values().length];
-      for (int i = 0; i < resArr.length; i++) {
-        resArr[i] = Resource.newInstance(0, 0);
-      };
-    }
-    
-    public Resource getUsed() {
-      return resArr[ResourceType.USED.idx];
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder();
-      sb.append("{used=" + resArr[0] + "%, ");
-      sb.append("pending=" + resArr[1] + "%, ");
-      sb.append("am_used=" + resArr[2] + "%, ");
-      sb.append("reserved=" + resArr[3] + "%}");
-      sb.append("am_limit=" + resArr[6] + "%, ");
-      return sb.toString();
-    }
+    super();
   }
 
   /*
@@ -109,22 +58,6 @@ public class ResourceUsage {
     return _get(label, ResourceType.USED);
   }
 
-  public Resource getCachedUsed() {
-    return _get(NL, ResourceType.CACHED_USED);
-  }
-
-  public Resource getCachedUsed(String label) {
-    return _get(label, ResourceType.CACHED_USED);
-  }
-
-  public Resource getCachedPending() {
-    return _get(NL, ResourceType.CACHED_PENDING);
-  }
-
-  public Resource getCachedPending(String label) {
-    return _get(label, ResourceType.CACHED_PENDING);
-  }
-
   public void incUsed(String label, Resource res) {
     _inc(label, ResourceType.USED, res);
   }
@@ -145,7 +78,7 @@ public class ResourceUsage {
     setUsed(NL, res);
   }
   
-  public void copyAllUsed(ResourceUsage other) {
+  public void copyAllUsed(AbstractResourceUsage other) {
     try {
       writeLock.lock();
       for (Entry<String, UsageByLabel> entry : other.usages.entrySet()) {
@@ -160,22 +93,6 @@ public class ResourceUsage {
     _set(label, ResourceType.USED, res);
   }
 
-  public void setCachedUsed(String label, Resource res) {
-    _set(label, ResourceType.CACHED_USED, res);
-  }
-
-  public void setCachedUsed(Resource res) {
-    _set(NL, ResourceType.CACHED_USED, res);
-  }
-
-  public void setCachedPending(String label, Resource res) {
-    _set(label, ResourceType.CACHED_PENDING, res);
-  }
-
-  public void setCachedPending(Resource res) {
-    _set(NL, ResourceType.CACHED_PENDING, res);
-  }
-
   /*
    * Pending
    */
@@ -281,6 +198,47 @@ public class ResourceUsage {
     _set(label, ResourceType.AMUSED, res);
   }
 
+  public Resource getAllPending() {
+    return _getAll(ResourceType.PENDING);
+  }
+
+  public Resource getAllUsed() {
+    return _getAll(ResourceType.USED);
+  }
+
+  // Cache Used
+  public Resource getCachedUsed() {
+    return _get(NL, ResourceType.CACHED_USED);
+  }
+
+  public Resource getCachedUsed(String label) {
+    return _get(label, ResourceType.CACHED_USED);
+  }
+
+  public Resource getCachedPending() {
+    return _get(NL, ResourceType.CACHED_PENDING);
+  }
+
+  public Resource getCachedPending(String label) {
+    return _get(label, ResourceType.CACHED_PENDING);
+  }
+
+  public void setCachedUsed(String label, Resource res) {
+    _set(label, ResourceType.CACHED_USED, res);
+  }
+
+  public void setCachedUsed(Resource res) {
+    _set(NL, ResourceType.CACHED_USED, res);
+  }
+
+  public void setCachedPending(String label, Resource res) {
+    _set(label, ResourceType.CACHED_PENDING, res);
+  }
+
+  public void setCachedPending(Resource res) {
+    _set(NL, ResourceType.CACHED_PENDING, res);
+  }
+
   /*
    * AM-Resource Limit
    */
@@ -316,94 +274,6 @@ public class ResourceUsage {
     _set(label, ResourceType.AMLIMIT, res);
   }
 
-  private static Resource normalize(Resource res) {
-    if (res == null) {
-      return Resources.none();
-    }
-    return res;
-  }
-
-  private Resource _get(String label, ResourceType type) {
-    if (label == null || label.equals(NL)) {
-      return normalize(usageNoLabel.resArr[type.idx]);
-    }
-    try {
-      readLock.lock();
-      UsageByLabel usage = usages.get(label);
-      if (null == usage) {
-        return Resources.none();
-      }
-      return normalize(usage.resArr[type.idx]);
-    } finally {
-      readLock.unlock();
-    }
-  }
-  
-  private Resource _getAll(ResourceType type) {
-    try {
-      readLock.lock();
-      Resource allOfType = Resources.createResource(0);
-      for (Map.Entry<String, UsageByLabel> usageEntry : usages.entrySet()) {
-        //all usages types are initialized
-        Resources.addTo(allOfType, usageEntry.getValue().resArr[type.idx]);
-      }
-      return allOfType;
-    } finally {
-      readLock.unlock();
-    }
-  }
-  
-  public Resource getAllPending() {
-    return _getAll(ResourceType.PENDING);
-  }
-  
-  public Resource getAllUsed() {
-    return _getAll(ResourceType.USED);
-  }
-
-  private UsageByLabel getAndAddIfMissing(String label) {
-    if (label == null || label.equals(NL)) {
-      return usageNoLabel;
-    }
-    if (!usages.containsKey(label)) {
-      UsageByLabel u = new UsageByLabel(label);
-      usages.put(label, u);
-      return u;
-    }
-
-    return usages.get(label);
-  }
-
-  private void _set(String label, ResourceType type, Resource res) {
-    try {
-      writeLock.lock();
-      UsageByLabel usage = getAndAddIfMissing(label);
-      usage.resArr[type.idx] = res;
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
-  private void _inc(String label, ResourceType type, Resource res) {
-    try {
-      writeLock.lock();
-      UsageByLabel usage = getAndAddIfMissing(label);
-      Resources.addTo(usage.resArr[type.idx], res);
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
-  private void _dec(String label, ResourceType type, Resource res) {
-    try {
-      writeLock.lock();
-      UsageByLabel usage = getAndAddIfMissing(label);
-      Resources.subtractFrom(usage.resArr[type.idx], res);
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
   public Resource getCachedDemand(String label) {
     try {
       readLock.lock();
@@ -415,23 +285,4 @@ public class ResourceUsage {
       readLock.unlock();
     }
   }
-  
-  @Override
-  public String toString() {
-    try {
-      readLock.lock();
-      return usages.toString();
-    } finally {
-      readLock.unlock();
-    }
-  }
-  
-  public Set<String> getNodePartitionsSet() {
-    try {
-      readLock.lock();
-      return usages.keySet();
-    } finally {
-      readLock.unlock();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 74c85ce..dddac4d 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
@@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AbsoluteResourceType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
@@ -87,6 +89,7 @@ public abstract class AbstractCSQueue implements CSQueue {
 
   final ResourceCalculator resourceCalculator;
   Set<String> accessibleLabels;
+  Set<String> resourceTypes;
   final RMNodeLabelsManager labelManager;
   String defaultLabelExpression;
   
@@ -102,6 +105,14 @@ public abstract class AbstractCSQueue implements CSQueue {
   // etc.
   QueueCapacities queueCapacities;
 
+  QueueResourceQuotas queueResourceQuotas;
+
+  protected enum CapacityConfigType {
+    NONE, PERCENTAGE, ABSOLUTE_RESOURCE
+  };
+  protected CapacityConfigType capacityConfigType =
+      CapacityConfigType.NONE;
+
   private final RecordFactory recordFactory = 
       RecordFactoryProvider.getRecordFactory(null);
   protected CapacitySchedulerContext csContext;
@@ -141,6 +152,9 @@ public abstract class AbstractCSQueue implements CSQueue {
     // initialize QueueCapacities
     queueCapacities = new QueueCapacities(parent == null);
 
+    // initialize queueResourceQuotas
+    queueResourceQuotas = new QueueResourceQuotas();
+
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     readLock = lock.readLock();
     writeLock = lock.writeLock();
@@ -276,6 +290,10 @@ public abstract class AbstractCSQueue implements CSQueue {
       this.defaultLabelExpression =
           csContext.getConfiguration().getDefaultNodeLabelExpression(
               getQueuePath());
+      this.resourceTypes = new HashSet<String>();
+      for (AbsoluteResourceType type : AbsoluteResourceType.values()) {
+        resourceTypes.add(type.toString().toLowerCase());
+      }
 
       // inherit from parent if labels not set
       if (this.accessibleLabels == null && parent != null) {
@@ -292,6 +310,11 @@ public abstract class AbstractCSQueue implements CSQueue {
       // After we setup labels, we can setup capacities
       setupConfigurableCapacities();
 
+      // Also fetch minimum/maximum resource constraint for this queue if
+      // configured.
+      capacityConfigType = CapacityConfigType.NONE;
+      updateConfigurableResourceRequirement(getQueuePath(), clusterResource);
+
       this.maximumAllocation =
           csContext.getConfiguration().getMaximumAllocationPerQueue(
               getQueuePath());
@@ -364,6 +387,125 @@ public abstract class AbstractCSQueue implements CSQueue {
     return unionInheritedWeights;
   }
 
+  protected void updateConfigurableResourceRequirement(String queuePath,
+      Resource clusterResource) {
+    CapacitySchedulerConfiguration conf = csContext.getConfiguration();
+    Set<String> configuredNodelabels = conf.getConfiguredNodeLabels(queuePath);
+
+    for (String label : configuredNodelabels) {
+      Resource minResource = conf.getMinimumResourceRequirement(label,
+          queuePath, resourceTypes);
+      Resource maxResource = conf.getMaximumResourceRequirement(label,
+          queuePath, resourceTypes);
+
+      if (this.capacityConfigType.equals(CapacityConfigType.NONE)) {
+        this.capacityConfigType = (!minResource.equals(Resources.none())
+            && queueCapacities.getAbsoluteCapacity(label) == 0f)
+                ? CapacityConfigType.ABSOLUTE_RESOURCE
+                : CapacityConfigType.PERCENTAGE;
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("capacityConfigType is updated as '" + capacityConfigType
+              + "' for queue '" + getQueueName());
+        }
+      }
+
+      validateAbsoluteVsPercentageCapacityConfig(minResource);
+
+      // If min resource for a resource type is greater than its max resource,
+      // throw exception to handle such error configs.
+      if (!maxResource.equals(Resources.none()) && Resources.greaterThan(
+          resourceCalculator, clusterResource, minResource, maxResource)) {
+        throw new IllegalArgumentException("Min resource configuration "
+            + minResource + " is greater than its max value:" + maxResource
+            + " in queue:" + getQueueName());
+      }
+
+      // If parent's max resource is lesser to a specific child's max
+      // resource, throw exception to handle such error configs.
+      if (parent != null) {
+        Resource parentMaxRes = parent.getQueueResourceQuotas()
+            .getConfiguredMaxResource(label);
+        if (Resources.greaterThan(resourceCalculator, clusterResource,
+            parentMaxRes, Resources.none())) {
+          if (Resources.greaterThan(resourceCalculator, clusterResource,
+              maxResource, parentMaxRes)) {
+            throw new IllegalArgumentException("Max resource configuration "
+                + maxResource + " is greater than parents max value:"
+                + parentMaxRes + " in queue:" + getQueueName());
+          }
+
+          // If child's max resource is not set, but its parent max resource is
+          // set, we must set child max resource to its parent's.
+          if (maxResource.equals(Resources.none())
+              && !minResource.equals(Resources.none())) {
+            maxResource = Resources.clone(parentMaxRes);
+          }
+        }
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Updating absolute resource configuration for queue:"
+            + getQueueName() + " as minResource=" + minResource
+            + " and maxResource=" + maxResource);
+      }
+
+      queueResourceQuotas.setConfiguredMinResource(label, minResource);
+      queueResourceQuotas.setConfiguredMaxResource(label, maxResource);
+    }
+  }
+
+  private void validateAbsoluteVsPercentageCapacityConfig(
+      Resource minResource) {
+    CapacityConfigType localType = CapacityConfigType.PERCENTAGE;
+    if (!minResource.equals(Resources.none())) {
+      localType = CapacityConfigType.ABSOLUTE_RESOURCE;
+    }
+
+    if (!queueName.equals("root")
+        && !this.capacityConfigType.equals(localType)) {
+      throw new IllegalArgumentException("Queue '" + getQueueName()
+          + "' should use either percentage based capacity"
+          + " configuration or absolute resource.");
+    }
+  }
+
+  @Override
+  public CapacityConfigType getCapacityConfigType() {
+    return capacityConfigType;
+  }
+
+  @Override
+  public Resource getEffectiveCapacity(String label) {
+    return Resources
+        .clone(getQueueResourceQuotas().getEffectiveMinResource(label));
+  }
+
+  @Override
+  public Resource getEffectiveCapacityUp(String label) {
+    return Resources
+        .clone(getQueueResourceQuotas().getEffectiveMinResourceUp(label));
+  }
+
+  @Override
+  public Resource getEffectiveCapacityDown(String label, Resource factor) {
+    return Resources.normalizeDown(resourceCalculator,
+        getQueueResourceQuotas().getEffectiveMinResource(label),
+        minimumAllocation);
+  }
+
+  @Override
+  public Resource getEffectiveMaxCapacity(String label) {
+    return Resources
+        .clone(getQueueResourceQuotas().getEffectiveMaxResource(label));
+  }
+
+  @Override
+  public Resource getEffectiveMaxCapacityDown(String label, Resource factor) {
+    return Resources.normalizeDown(resourceCalculator,
+        getQueueResourceQuotas().getEffectiveMaxResource(label),
+        minimumAllocation);
+  }
+
   private void initializeQueueState(QueueState previousState,
       QueueState configuredState, QueueState parentState) {
     // verify that we can not any value for State other than RUNNING/STOPPED
@@ -555,6 +697,11 @@ public abstract class AbstractCSQueue implements CSQueue {
   }
 
   @Override
+  public QueueResourceQuotas getQueueResourceQuotas() {
+    return queueResourceQuotas;
+  }
+
+  @Override
   public ReentrantReadWriteLock.ReadLock getReadLock() {
     return readLock;
   }
@@ -604,7 +751,7 @@ public abstract class AbstractCSQueue implements CSQueue {
        * limit-set-by-parent)
        */
       Resource queueMaxResource =
-          getQueueMaxResource(nodePartition, clusterResource);
+          getQueueMaxResource(nodePartition);
 
       return Resources.min(resourceCalculator, clusterResource,
           queueMaxResource, currentResourceLimits.getLimit());
@@ -617,11 +764,8 @@ public abstract class AbstractCSQueue implements CSQueue {
     return Resources.none();
   }
 
-  Resource getQueueMaxResource(String nodePartition, Resource clusterResource) {
-    return Resources.multiplyAndNormalizeDown(resourceCalculator,
-        labelManager.getResourceByLabel(nodePartition, clusterResource),
-        queueCapacities.getAbsoluteMaximumCapacity(nodePartition),
-        minimumAllocation);
+  Resource getQueueMaxResource(String nodePartition) {
+    return getEffectiveMaxCapacity(nodePartition);
   }
 
   public boolean hasChildQueues() {
@@ -782,7 +926,7 @@ public abstract class AbstractCSQueue implements CSQueue {
     queueUsage.incUsed(nodeLabel, resourceToInc);
     CSQueueUtils.updateUsedCapacity(resourceCalculator,
         labelManager.getResourceByLabel(nodeLabel, Resources.none()),
-        nodeLabel, this);
+        Resources.none(), nodeLabel, this);
     if (null != parent) {
       parent.incUsedResource(nodeLabel, resourceToInc, null);
     }
@@ -798,7 +942,7 @@ public abstract class AbstractCSQueue implements CSQueue {
     queueUsage.decUsed(nodeLabel, resourceToDec);
     CSQueueUtils.updateUsedCapacity(resourceCalculator,
         labelManager.getResourceByLabel(nodeLabel, Resources.none()),
-        nodeLabel, this);
+        Resources.none(), nodeLabel, this);
     if (null != parent) {
       parent.decUsedResource(nodeLabel, resourceToDec, null);
     }
@@ -904,7 +1048,7 @@ public abstract class AbstractCSQueue implements CSQueue {
         Resource maxResourceLimit;
         if (allocation.getSchedulingMode()
             == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
-          maxResourceLimit = getQueueMaxResource(partition, cluster);
+          maxResourceLimit = getQueueMaxResource(partition);
         } else{
           maxResourceLimit = labelManager.getResourceByLabel(
               schedulerContainer.getNodePartition(), cluster);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 43e7f53..2e29a71 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
@@ -41,10 +41,12 @@ import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractUsersManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue.CapacityConfigType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
@@ -355,4 +357,41 @@ public interface CSQueue extends SchedulerQueue<CSQueue> {
    * @return map of usernames and corresponding weight
    */
   Map<String, Float> getUserWeights();
+
+  /**
+   * Get QueueResourceQuotas associated with each queue.
+   * @return QueueResourceQuotas
+   */
+  public QueueResourceQuotas getQueueResourceQuotas();
+
+  /**
+   * Get CapacityConfigType as PERCENTAGE or ABSOLUTE_RESOURCE
+   * @return CapacityConfigType
+   */
+  public CapacityConfigType getCapacityConfigType();
+
+  /**
+   * Get effective capacity of queue. If min/max resource is configured,
+   * preference will be given to absolute configuration over normal capacity.
+   * Also round down the result to normalizeDown.
+   *
+   * @param label
+   *          partition
+   * @return effective queue capacity
+   */
+  Resource getEffectiveCapacity(String label);
+  Resource getEffectiveCapacityUp(String label);
+  Resource getEffectiveCapacityDown(String label, Resource factor);
+
+  /**
+   * Get effective max capacity of queue. If min/max resource is configured,
+   * preference will be given to absolute configuration over normal capacity.
+   * Also round down the result to normalizeDown.
+   *
+   * @param label
+   *          partition
+   * @return effective max queue capacity
+   */
+  Resource getEffectiveMaxCapacity(String label);
+  Resource getEffectiveMaxCapacityDown(String label, Resource factor);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.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/CSQueueUtils.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/CSQueueUtils.java
index e1014c1..81dec80 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/CSQueueUtils.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/CSQueueUtils.java
@@ -150,7 +150,7 @@ class CSQueueUtils {
       }
     }
   }
-  
+
   // Set absolute capacities for {capacity, maximum-capacity}
   private static void updateAbsoluteCapacitiesByNodeLabels(
       QueueCapacities queueCapacities, QueueCapacities parentQueueCapacities) {
@@ -180,8 +180,8 @@ class CSQueueUtils {
    * used resource for all partitions of this queue.
    */
   public static void updateUsedCapacity(final ResourceCalculator rc,
-      final Resource totalPartitionResource, String nodePartition,
-      AbstractCSQueue childQueue) {
+      final Resource totalPartitionResource, Resource clusterResource,
+      String nodePartition, AbstractCSQueue childQueue) {
     QueueCapacities queueCapacities = childQueue.getQueueCapacities();
     CSQueueMetrics queueMetrics = childQueue.getMetrics();
     ResourceUsage queueResourceUsage = childQueue.getQueueResourceUsage();
@@ -193,11 +193,8 @@ class CSQueueUtils {
 
     if (Resources.greaterThan(rc, totalPartitionResource,
         totalPartitionResource, Resources.none())) {
-      // queueGuaranteed = totalPartitionedResource *
-      // absolute_capacity(partition)
-      Resource queueGuranteedResource =
-          Resources.multiply(totalPartitionResource,
-              queueCapacities.getAbsoluteCapacity(nodePartition));
+      Resource queueGuranteedResource = childQueue
+          .getEffectiveCapacity(nodePartition);
 
       // make queueGuranteed >= minimum_allocation to avoid divided by 0.
       queueGuranteedResource =
@@ -248,9 +245,7 @@ class CSQueueUtils {
     for (String partition : nodeLabels) {
       // Calculate guaranteed resource for a label in a queue by below logic.
       // (total label resource) * (absolute capacity of label in that queue)
-      Resource queueGuranteedResource = Resources.multiply(nlm
-          .getResourceByLabel(partition, cluster), queue.getQueueCapacities()
-          .getAbsoluteCapacity(partition));
+      Resource queueGuranteedResource = queue.getEffectiveCapacity(partition);
 
       // Available resource in queue for a specific label will be calculated as
       // {(guaranteed resource for a label in a queue) -
@@ -289,15 +284,14 @@ class CSQueueUtils {
     ResourceUsage queueResourceUsage = childQueue.getQueueResourceUsage();
 
     if (nodePartition == null) {
-      for (String partition : Sets.union(
-          queueCapacities.getNodePartitionsSet(),
+      for (String partition : Sets.union(queueCapacities.getNodePartitionsSet(),
           queueResourceUsage.getNodePartitionsSet())) {
         updateUsedCapacity(rc, nlm.getResourceByLabel(partition, cluster),
-            partition, childQueue);
+            cluster, partition, childQueue);
       }
     } else {
       updateUsedCapacity(rc, nlm.getResourceByLabel(nodePartition, cluster),
-          nodePartition, childQueue);
+          cluster, nodePartition, childQueue);
     }
 
     // Update queue metrics w.r.t node labels. In a generic way, we can

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index 4515453..f81e097 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderi
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
+import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -60,6 +61,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import java.util.Set;
 import java.util.StringTokenizer;
 
@@ -317,6 +320,21 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @Private
   public static final int DEFAULT_MAX_ASSIGN_PER_HEARTBEAT = -1;
 
+  /** Configuring absolute min/max resources in a queue **/
+  @Private
+  public static final String MINIMUM_RESOURCE = "min-resource";
+
+  @Private
+  public static final String MAXIMUM_RESOURCE = "max-resource";
+
+  public static final String DEFAULT_RESOURCE_TYPES = "memory,vcores";
+
+  public static final String PATTERN_FOR_ABSOLUTE_RESOURCE = "\\[([^\\]]+)";
+
+  public enum AbsoluteResourceType {
+    MEMORY, VCORES;
+  }
+
   AppPriorityACLConfigurationParser priorityACLConfig = new AppPriorityACLConfigurationParser();
 
   public CapacitySchedulerConfiguration() {
@@ -394,7 +412,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   
   public float getNonLabeledQueueCapacity(String queue) {
     float capacity = queue.equals("root") ? 100.0f : getFloat(
-        getQueuePrefix(queue) + CAPACITY, UNDEFINED);
+        getQueuePrefix(queue) + CAPACITY, 0f);
     if (capacity < MINIMUM_CAPACITY_VALUE || capacity > MAXIMUM_CAPACITY_VALUE) {
       throw new IllegalArgumentException("Illegal " +
       		"capacity of " + capacity + " for queue " + queue);
@@ -1676,4 +1694,163 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
         queuePath);
     setMaximumCapacity(leafQueueConfPrefix, val);
   }
+
+  public static String getUnits(String resourceValue) {
+    String units;
+    for (int i = 0; i < resourceValue.length(); i++) {
+      if (Character.isAlphabetic(resourceValue.charAt(i))) {
+        units = resourceValue.substring(i);
+        if (StringUtils.isAlpha(units)) {
+          return units;
+        }
+      }
+    }
+    return "";
+  }
+
+  /**
+   * Get absolute minimum resource requirement for a queue.
+   *
+   * @param label
+   *          NodeLabel
+   * @param queue
+   *          queue path
+   * @param resourceTypes
+   *          Resource types
+   * @return ResourceInformation
+   */
+  public Resource getMinimumResourceRequirement(String label, String queue,
+      Set<String> resourceTypes) {
+    return internalGetLabeledResourceRequirementForQueue(queue, label,
+        resourceTypes, MINIMUM_RESOURCE);
+  }
+
+  /**
+   * Get absolute maximum resource requirement for a queue.
+   *
+   * @param label
+   *          NodeLabel
+   * @param queue
+   *          queue path
+   * @param resourceTypes
+   *          Resource types
+   * @return Resource
+   */
+  public Resource getMaximumResourceRequirement(String label, String queue,
+      Set<String> resourceTypes) {
+    return internalGetLabeledResourceRequirementForQueue(queue, label,
+        resourceTypes, MAXIMUM_RESOURCE);
+  }
+
+  @VisibleForTesting
+  public void setMinimumResourceRequirement(String label, String queue,
+      Resource resource) {
+    updateMinMaxResourceToConf(label, queue, resource, MINIMUM_RESOURCE);
+  }
+
+  @VisibleForTesting
+  public void setMaximumResourceRequirement(String label, String queue,
+      Resource resource) {
+    updateMinMaxResourceToConf(label, queue, resource, MAXIMUM_RESOURCE);
+  }
+
+  private void updateMinMaxResourceToConf(String label, String queue,
+      Resource resource, String type) {
+    if (queue.equals("root")) {
+      throw new IllegalArgumentException(
+          "Cannot set resource, root queue will take 100% of cluster capacity");
+    }
+
+    StringBuilder resourceString = new StringBuilder();
+    resourceString
+        .append("[" + AbsoluteResourceType.MEMORY.toString().toLowerCase() + "="
+            + resource.getMemorySize() + ","
+            + AbsoluteResourceType.VCORES.toString().toLowerCase() + "="
+            + resource.getVirtualCores() + "]");
+
+    String prefix = getQueuePrefix(queue) + type;
+    if (!label.isEmpty()) {
+      prefix = getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS + DOT + label
+          + DOT + type;
+    }
+    set(prefix, resourceString.toString());
+  }
+
+  private Resource internalGetLabeledResourceRequirementForQueue(String queue,
+      String label, Set<String> resourceTypes, String suffix) {
+    String propertyName = getNodeLabelPrefix(queue, label) + suffix;
+    String resourceString = get(propertyName);
+    if (resourceString == null || resourceString.isEmpty()) {
+      return Resources.none();
+    }
+
+    // Define resource here.
+    Resource resource = Resource.newInstance(0l, 0);
+    Matcher matcher = Pattern.compile(PATTERN_FOR_ABSOLUTE_RESOURCE)
+        .matcher(resourceString);
+    /*
+     * Absolute resource configuration for a queue will be grouped by "[]".
+     * Syntax of absolute resource config could be like below
+     * "memory=4Gi vcores=2". Ideally this means "4GB of memory and 2 vcores".
+     */
+    if (matcher.find()) {
+      // Get the sub-group.
+      String subGroup = matcher.group(1);
+      if (subGroup.trim().isEmpty()) {
+        return Resources.none();
+      }
+
+      for (String kvPair : subGroup.trim().split(",")) {
+        String[] splits = kvPair.split("=");
+
+        // Ensure that each sub string is key value pair separated by '='.
+        if (splits != null && splits.length > 1) {
+          updateResourceValuesFromConfig(resourceTypes, resource, splits);
+        }
+      }
+    }
+
+    // Memory has to be configured always.
+    if (resource.getMemorySize() == 0l) {
+      return Resources.none();
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("CSConf - getAbsolueResourcePerQueue: prefix="
+          + getNodeLabelPrefix(queue, label) + ", capacity=" + resource);
+    }
+    return resource;
+  }
+
+  private void updateResourceValuesFromConfig(Set<String> resourceTypes,
+      Resource resource, String[] splits) {
+
+    // If key is not a valid type, skip it.
+    if (!resourceTypes.contains(splits[0])) {
+      return;
+    }
+
+    String units = getUnits(splits[1]);
+    Long resourceValue = Long
+        .valueOf(splits[1].substring(0, splits[1].length() - units.length()));
+
+    // Convert all incoming units to MB if units is configured.
+    if (!units.isEmpty()) {
+      resourceValue = UnitsConversionUtil.convert(units, "Mi", resourceValue);
+    }
+
+    // map it based on key.
+    AbsoluteResourceType resType = AbsoluteResourceType
+        .valueOf(StringUtils.toUpperCase(splits[0].trim()));
+    switch (resType) {
+      case MEMORY :
+        resource.setMemorySize(resourceValue);
+        break;
+      case VCORES :
+        resource.setVirtualCores(resourceValue.intValue());
+        break;
+      default :
+        break;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 e8342d9..1460121 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
@@ -671,12 +671,7 @@ public class LeafQueue extends AbstractCSQueue {
           1.0f / Math.max(getAbstractUsersManager().getNumActiveUsers(), 1));
       effectiveUserLimit = Math.min(effectiveUserLimit * userWeight, 1.0f);
 
-      Resource queuePartitionResource = Resources
-          .multiplyAndNormalizeUp(resourceCalculator,
-              labelManager.getResourceByLabel(nodePartition,
-                  lastClusterResource),
-              queueCapacities.getAbsoluteCapacity(nodePartition),
-              minimumAllocation);
+      Resource queuePartitionResource = getEffectiveCapacityUp(nodePartition);
 
       Resource userAMLimit = Resources.multiplyAndNormalizeUp(
           resourceCalculator, queuePartitionResource,
@@ -705,11 +700,7 @@ public class LeafQueue extends AbstractCSQueue {
        * non-labeled), * with per-partition am-resource-percent to get the max am
        * resource limit for this queue and partition.
        */
-      Resource queuePartitionResource = Resources.multiplyAndNormalizeUp(
-          resourceCalculator,
-          labelManager.getResourceByLabel(nodePartition, lastClusterResource),
-          queueCapacities.getAbsoluteCapacity(nodePartition),
-          minimumAllocation);
+      Resource queuePartitionResource = getEffectiveCapacityUp(nodePartition);
 
       Resource queueCurrentLimit = Resources.none();
       // For non-labeled partition, we need to consider the current queue
@@ -965,6 +956,14 @@ public class LeafQueue extends AbstractCSQueue {
   private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) {
     // Set preemption-allowed:
     // For leaf queue, only under-utilized queue is allowed to preempt resources from other queues
+    if (!queueResourceQuotas.getEffectiveMinResource(nodePartition)
+        .equals(Resources.none())) {
+      limits.setIsAllowPreemption(Resources.lessThan(resourceCalculator,
+          csContext.getClusterResource(), queueUsage.getUsed(nodePartition),
+          queueResourceQuotas.getEffectiveMinResource(nodePartition)));
+      return;
+    }
+
     float usedCapacity = queueCapacities.getAbsoluteUsedCapacity(nodePartition);
     float guaranteedCapacity = queueCapacities.getAbsoluteCapacity(nodePartition);
     limits.setIsAllowPreemption(usedCapacity < guaranteedCapacity);
@@ -1344,7 +1343,7 @@ public class LeafQueue extends AbstractCSQueue {
     currentPartitionResourceLimit =
         partition.equals(RMNodeLabelsManager.NO_LABEL)
             ? currentPartitionResourceLimit
-            : getQueueMaxResource(partition, clusterResource);
+            : getQueueMaxResource(partition);
 
     Resource headroom = Resources.componentwiseMin(
         Resources.subtract(userLimitResource, user.getUsed(partition)),
@@ -1716,12 +1715,8 @@ public class LeafQueue extends AbstractCSQueue {
     // this. So need cap limits by queue's max capacity here.
     this.cachedResourceLimitsForHeadroom =
         new ResourceLimits(currentResourceLimits.getLimit());
-    Resource queueMaxResource =
-        Resources.multiplyAndNormalizeDown(resourceCalculator, labelManager
-            .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
-            queueCapacities
-                .getAbsoluteMaximumCapacity(RMNodeLabelsManager.NO_LABEL),
-            minimumAllocation);
+    Resource queueMaxResource = getEffectiveMaxCapacityDown(
+        RMNodeLabelsManager.NO_LABEL, minimumAllocation);
     this.cachedResourceLimitsForHeadroom.setLimit(Resources.min(
         resourceCalculator, clusterResource, queueMaxResource,
         currentResourceLimits.getLimit()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/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 959ca51..c770fac 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
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
@@ -67,6 +68,7 @@ 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.server.resourcemanager.scheduler.placement.CandidateNodeSet;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.CandidateNodeSetUtils;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 @Private
@@ -162,31 +164,78 @@ public class ParentQueue extends AbstractCSQueue {
       writeLock.lock();
       // Validate
       float childCapacities = 0;
+      Resource minResDefaultLabel = Resources.createResource(0, 0);
       for (CSQueue queue : childQueues) {
         childCapacities += queue.getCapacity();
+        Resources.addTo(minResDefaultLabel, queue.getQueueResourceQuotas()
+            .getConfiguredMinResource());
+
+        // If any child queue is using percentage based capacity model vs parent
+        // queues' absolute configuration or vice versa, throw back an
+        // exception.
+        if (!queueName.equals("root") && getCapacity() != 0f
+            && !queue.getQueueResourceQuotas().getConfiguredMinResource()
+                .equals(Resources.none())) {
+          throw new IllegalArgumentException("Parent queue '" + getQueueName()
+              + "' and child queue '" + queue.getQueueName()
+              + "' should use either percentage based capacity"
+              + " configuration or absolute resource together.");
+        }
       }
+
       float delta = Math.abs(1.0f - childCapacities);  // crude way to check
       // allow capacities being set to 0, and enforce child 0 if parent is 0
-      if (((queueCapacities.getCapacity() > 0) && (delta > PRECISION)) || (
-          (queueCapacities.getCapacity() == 0) && (childCapacities > 0))) {
-        throw new IllegalArgumentException(
-            "Illegal" + " capacity of " + childCapacities
-                + " for children of queue " + queueName);
+      if ((minResDefaultLabel.equals(Resources.none())
+          && (queueCapacities.getCapacity() > 0) && (delta > PRECISION))
+          || ((queueCapacities.getCapacity() == 0) && (childCapacities > 0))) {
+        throw new IllegalArgumentException("Illegal" + " capacity of "
+            + childCapacities + " for children of queue " + queueName);
       }
       // check label capacities
       for (String nodeLabel : queueCapacities.getExistingNodeLabels()) {
         float capacityByLabel = queueCapacities.getCapacity(nodeLabel);
         // check children's labels
         float sum = 0;
+        Resource minRes = Resources.createResource(0, 0);
+        Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel,
+            scheduler.getClusterResource());
         for (CSQueue queue : childQueues) {
           sum += queue.getQueueCapacities().getCapacity(nodeLabel);
+
+          // If any child queue of a label is using percentage based capacity
+          // model vs parent queues' absolute configuration or vice versa, throw
+          // back an exception
+          if (!queueName.equals("root") && !this.capacityConfigType
+              .equals(queue.getCapacityConfigType())) {
+            throw new IllegalArgumentException("Parent queue '" + getQueueName()
+                + "' and child queue '" + queue.getQueueName()
+                + "' should use either percentage based capacity"
+                + "configuration or absolute resource together for label:"
+                + nodeLabel);
+          }
+
+          // Accumulate all min/max resource configured for all child queues.
+          Resources.addTo(minRes, queue.getQueueResourceQuotas()
+              .getConfiguredMinResource(nodeLabel));
         }
-        if ((capacityByLabel > 0 && Math.abs(1.0f - sum) > PRECISION)
+        if ((minResDefaultLabel.equals(Resources.none()) && capacityByLabel > 0
+            && Math.abs(1.0f - sum) > PRECISION)
             || (capacityByLabel == 0) && (sum > 0)) {
           throw new IllegalArgumentException(
               "Illegal" + " capacity of " + sum + " for children of queue "
                   + queueName + " for label=" + nodeLabel);
         }
+
+        // Ensure that for each parent queue: parent.min-resource >=
+        // Σ(child.min-resource).
+        Resource parentMinResource = queueResourceQuotas
+            .getConfiguredMinResource(nodeLabel);
+        if (!parentMinResource.equals(Resources.none()) && Resources.lessThan(
+            resourceCalculator, resourceByLabel, parentMinResource, minRes)) {
+          throw new IllegalArgumentException("Parent Queues" + " capacity: "
+              + parentMinResource + " is less than" + " to its children:"
+              + minRes + " for queue:" + queueName);
+        }
       }
 
       this.childQueues.clear();
@@ -687,11 +736,8 @@ public class ParentQueue extends AbstractCSQueue {
         child.getQueueResourceUsage().getUsed(nodePartition));
 
     // Get child's max resource
-    Resource childConfiguredMaxResource = Resources.multiplyAndNormalizeDown(
-        resourceCalculator,
-        labelManager.getResourceByLabel(nodePartition, clusterResource),
-        child.getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition),
-        minimumAllocation);
+    Resource childConfiguredMaxResource = getEffectiveMaxCapacityDown(
+        nodePartition, minimumAllocation);
 
     // Child's limit should be capped by child configured max resource
     childLimit =
@@ -827,6 +873,14 @@ public class ParentQueue extends AbstractCSQueue {
       ResourceLimits resourceLimits) {
     try {
       writeLock.lock();
+
+      // Update effective capacity in all parent queue.
+      Set<String> configuredNodelabels = csContext.getConfiguration()
+          .getConfiguredNodeLabels(getQueuePath());
+      for (String label : configuredNodelabels) {
+        calculateEffectiveResourcesAndCapacity(label, clusterResource);
+      }
+
       // Update all children
       for (CSQueue childQueue : childQueues) {
         // Get ResourceLimits of child queue before assign containers
@@ -848,6 +902,110 @@ public class ParentQueue extends AbstractCSQueue {
     return true;
   }
 
+  private void calculateEffectiveResourcesAndCapacity(String label,
+      Resource clusterResource) {
+
+    // For root queue, ensure that max/min resource is updated to latest
+    // cluster resource.
+    Resource resourceByLabel = labelManager.getResourceByLabel(label,
+        clusterResource);
+    if (getQueueName().equals("root")) {
+      queueResourceQuotas.setConfiguredMinResource(label, resourceByLabel);
+      queueResourceQuotas.setConfiguredMaxResource(label, resourceByLabel);
+      queueResourceQuotas.setEffectiveMinResource(label, resourceByLabel);
+      queueResourceQuotas.setEffectiveMaxResource(label, resourceByLabel);
+    }
+
+    // Total configured min resources of direct children of queue
+    Resource configuredMinResources = Resource.newInstance(0L, 0);
+    for (CSQueue childQueue : getChildQueues()) {
+      Resources.addTo(configuredMinResources,
+          childQueue.getQueueResourceQuotas().getConfiguredMinResource(label));
+    }
+
+    // Factor to scale down effective resource: When cluster has sufficient
+    // resources, effective_min_resources will be same as configured
+    // min_resources.
+    float effectiveMinRatio = 1;
+    ResourceCalculator rc = this.csContext.getResourceCalculator();
+    if (getQueueName().equals("root")) {
+      if (!resourceByLabel.equals(Resources.none()) && Resources.lessThan(rc,
+          clusterResource, resourceByLabel, configuredMinResources)) {
+        effectiveMinRatio = Resources.divide(rc, clusterResource,
+            resourceByLabel, configuredMinResources);
+      }
+    } else {
+      if (Resources.lessThan(rc, clusterResource,
+          queueResourceQuotas.getEffectiveMinResource(label),
+          configuredMinResources)) {
+        effectiveMinRatio = Resources.divide(rc, clusterResource,
+            queueResourceQuotas.getEffectiveMinResource(label),
+            configuredMinResources);
+      }
+    }
+
+    // loop and do this for all child queues
+    for (CSQueue childQueue : getChildQueues()) {
+      Resource minResource = childQueue.getQueueResourceQuotas()
+          .getConfiguredMinResource(label);
+
+      // Update effective resource (min/max) to each child queue.
+      if (childQueue.getCapacityConfigType()
+          .equals(CapacityConfigType.ABSOLUTE_RESOURCE)) {
+        childQueue.getQueueResourceQuotas().setEffectiveMinResource(label,
+            Resources.multiply(minResource, effectiveMinRatio));
+
+        // Max resource of a queue should be a minimum of {configuredMaxRes,
+        // parentMaxRes}. parentMaxRes could be configured value. But if not
+        // present could also be taken from effective max resource of parent.
+        Resource parentMaxRes = queueResourceQuotas
+            .getConfiguredMaxResource(label);
+        if (parentMaxRes.equals(Resources.none())) {
+          parentMaxRes = parent.getQueueResourceQuotas()
+              .getEffectiveMaxResource(label);
+        }
+
+        // Minimum of {childMaxResource, parentMaxRes}. However if
+        // childMaxResource is empty, consider parent's max resource alone.
+        Resource childMaxResource = childQueue.getQueueResourceQuotas()
+            .getConfiguredMaxResource(label);
+        Resource effMaxResource = Resources.min(resourceCalculator,
+            resourceByLabel, childMaxResource.equals(Resources.none())
+                ? parentMaxRes
+                : childMaxResource,
+            parentMaxRes);
+        childQueue.getQueueResourceQuotas().setEffectiveMaxResource(label,
+            Resources.clone(effMaxResource));
+      } else {
+        childQueue.getQueueResourceQuotas().setEffectiveMinResource(label,
+            Resources.multiply(resourceByLabel,
+                childQueue.getQueueCapacities().getAbsoluteCapacity(label)));
+        childQueue.getQueueResourceQuotas().setEffectiveMaxResource(label,
+            Resources.multiply(resourceByLabel, childQueue.getQueueCapacities()
+                .getAbsoluteMaximumCapacity(label)));
+
+        childQueue.getQueueResourceQuotas().setEffectiveMinResourceUp(label,
+            Resources.multiplyAndNormalizeUp(rc, resourceByLabel,
+                childQueue.getQueueCapacities().getAbsoluteCapacity(label),
+                minimumAllocation));
+        childQueue.getQueueResourceQuotas().setEffectiveMaxResourceUp(label,
+            Resources.multiplyAndNormalizeUp(rc,
+                resourceByLabel, childQueue.getQueueCapacities()
+                    .getAbsoluteMaximumCapacity(label),
+                    minimumAllocation));
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Updating effective min resource for queue:"
+            + childQueue.getQueueName() + " as effMinResource="
+            + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label)
+            + "and Updating effective max resource as effMaxResource="
+            + childQueue.getQueueResourceQuotas()
+                .getEffectiveMaxResource(label));
+      }
+    }
+  }
+
   @Override
   public List<CSQueue> getChildQueues() {
     try {
@@ -980,9 +1138,21 @@ public class ParentQueue extends AbstractCSQueue {
        * When this happens, we have to preempt killable container (on same or different
        * nodes) of parent queue to avoid violating parent's max resource.
        */
-      if (getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition)
-          < getQueueCapacities().getAbsoluteUsedCapacity(nodePartition)) {
-        killContainersToEnforceMaxQueueCapacity(nodePartition, clusterResource);
+      if (!queueResourceQuotas.getEffectiveMaxResource(nodePartition)
+          .equals(Resources.none())) {
+        if (Resources.lessThan(resourceCalculator, clusterResource,
+            queueResourceQuotas.getEffectiveMaxResource(nodePartition),
+            queueUsage.getUsed(nodePartition))) {
+          killContainersToEnforceMaxQueueCapacity(nodePartition,
+              clusterResource);
+        }
+      } else {
+        if (getQueueCapacities()
+            .getAbsoluteMaximumCapacity(nodePartition) < getQueueCapacities()
+                .getAbsoluteUsedCapacity(nodePartition)) {
+          killContainersToEnforceMaxQueueCapacity(nodePartition,
+              clusterResource);
+        }
       }
     } finally {
       writeLock.unlock();
@@ -999,8 +1169,7 @@ public class ParentQueue extends AbstractCSQueue {
 
     Resource partitionResource = labelManager.getResourceByLabel(partition,
         null);
-    Resource maxResource = Resources.multiply(partitionResource,
-        getQueueCapacities().getAbsoluteMaximumCapacity(partition));
+    Resource maxResource = getEffectiveMaxCapacity(partition);
 
     while (Resources.greaterThan(resourceCalculator, partitionResource,
         queueUsage.getUsed(partition), maxResource)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e798b1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.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/UsersManager.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/UsersManager.java
index 33f30b0..efc20e9 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/UsersManager.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/UsersManager.java
@@ -686,10 +686,7 @@ public class UsersManager implements AbstractUsersManager {
      * * If we're running over capacity, then its (usedResources + required)
      * (which extra resources we are allocating)
      */
-    Resource queueCapacity = Resources.multiplyAndNormalizeUp(
-        resourceCalculator, partitionResource,
-        lQueue.getQueueCapacities().getAbsoluteCapacity(nodePartition),
-        lQueue.getMinimumAllocation());
+    Resource queueCapacity = lQueue.getEffectiveCapacityUp(nodePartition);
 
     /*
      * Assume we have required resource equals to minimumAllocation, this can


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