Browse Source

YARN-2496. Enhanced Capacity Scheduler to have basic support for allocating resources based on node-labels. Contributed by Wangda Tan.
YARN-2500. Ehnaced ResourceManager to support schedulers allocating resources based on node-labels. Contributed by Wangda Tan.

(cherry picked from commit f2ea555ac6c06a3f2f6559731f48711fff05d3f1)

Vinod Kumar Vavilapalli 10 năm trước cách đây
mục cha
commit
e8e3a36213
57 tập tin đã thay đổi với 2843 bổ sung793 xóa
  1. 5 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
  2. 5 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
  3. 6 0
      hadoop-yarn-project/CHANGES.txt
  4. 17 0
      hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
  5. 27 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  6. 25 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  7. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
  8. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
  9. 15 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
  10. 10 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  11. 13 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
  12. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  13. 26 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  14. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
  15. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
  16. 19 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
  17. 120 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
  18. 448 0
      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
  19. 47 14
      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
  20. 55 2
      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
  21. 43 5
      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
  22. 147 1
      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
  23. 289 280
      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
  24. 139 252
      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. 7 1
      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
  26. 1 1
      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
  27. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
  28. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  29. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
  30. 28 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
  31. 6 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
  32. 32 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  33. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
  34. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
  35. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java
  36. 30 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  37. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
  38. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
  39. 30 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
  40. 9 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
  41. 62 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  42. 234 45
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
  43. 10 1
      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
  44. 8 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCSQueueUtils.java
  45. 20 8
      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
  46. 3 2
      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
  47. 460 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
  48. 24 13
      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
  49. 3 2
      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
  50. 2 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueMappings.java
  51. 259 8
      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
  52. 7 2
      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
  53. 20 10
      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
  54. 29 2
      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
  55. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
  56. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  57. 24 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java

+ 5 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.sls.nodemanager;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -159,6 +160,10 @@ public class NodeInfo {
       return null;
     }
 
+    @Override
+    public Set<String> getNodeLabels() {
+      return null;
+    }
   }
 
   public static RMNode newNodeInfo(String rackName, String hostName,

+ 5 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode
 
 import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 
 @Private
 @Unstable
@@ -147,4 +148,8 @@ public class RMNodeWrapper implements RMNode {
     return node.getNodeManagerVersion();
   }
 
+  @Override
+  public Set<String> getNodeLabels() {
+    return null;
+  }
 }

+ 6 - 0
hadoop-yarn-project/CHANGES.txt

@@ -138,6 +138,12 @@ Release 2.6.0 - UNRELEASED
     YARN-2656. Made RM web services authentication filter support proxy user.
     (Varun Vasudev and Zhijie Shen via zjshen)
 
+    YARN-2496. Enhanced Capacity Scheduler to have basic support for allocating
+    resources based on node-labels. (Wangda Tan via vinodkv)
+
+    YARN-2500. Ehnaced ResourceManager to support schedulers allocating resources
+    based on node-labels. (Wangda Tan via vinodkv)
+
   IMPROVEMENTS
 
     YARN-2242. Improve exception information on AM launch crashes. (Li Lu 

+ 17 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml

@@ -188,6 +188,23 @@
     </Or>
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue" />
+    <Or>
+      <Field name="absoluteCapacity" />
+      <Field name="absoluteMaxCapacity" />
+      <Field name="acls" />
+      <Field name="capacity" />
+      <Field name="maximumCapacity" />
+      <Field name="state" />
+      <Field name="labelManager" />
+      <Field name="defaultLabelExpression" />
+      <Field name="accessibleLabels" />
+      <Field name="absoluteNodeLabelCapacities" />
+      <Field name="reservationsContinueLooking" />
+    </Or>
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
   <!-- Inconsistent sync warning - scheduleAsynchronously is only initialized once and never changed -->
   <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler" />

+ 27 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NMToken;
@@ -254,13 +255,13 @@ public class ApplicationMasterService extends AbstractService implements
       if (hasApplicationMasterRegistered(applicationAttemptId)) {
         String message =
             "Application Master is already registered : "
-                + applicationAttemptId.getApplicationId();
+                + appID;
         LOG.warn(message);
         RMAuditLogger.logFailure(
           this.rmContext.getRMApps()
-            .get(applicationAttemptId.getApplicationId()).getUser(),
+            .get(appID).getUser(),
           AuditConstants.REGISTER_AM, "", "ApplicationMasterService", message,
-          applicationAttemptId.getApplicationId(), applicationAttemptId);
+          appID, applicationAttemptId);
         throw new InvalidApplicationMasterRequestException(message);
       }
       
@@ -340,6 +341,7 @@ public class ApplicationMasterService extends AbstractService implements
 
     ApplicationAttemptId applicationAttemptId =
         authorizeRequest().getApplicationAttemptId();
+    ApplicationId appId = applicationAttemptId.getApplicationId();
 
     AllocateResponseLock lock = responseMap.get(applicationAttemptId);
     if (lock == null) {
@@ -351,13 +353,13 @@ public class ApplicationMasterService extends AbstractService implements
       if (!hasApplicationMasterRegistered(applicationAttemptId)) {
         String message =
             "Application Master is trying to unregister before registering for: "
-                + applicationAttemptId.getApplicationId();
+                + appId;
         LOG.error(message);
         RMAuditLogger.logFailure(
             this.rmContext.getRMApps()
-                .get(applicationAttemptId.getApplicationId()).getUser(),
+                .get(appId).getUser(),
             AuditConstants.UNREGISTER_AM, "", "ApplicationMasterService",
-            message, applicationAttemptId.getApplicationId(),
+            message, appId,
             applicationAttemptId);
         throw new ApplicationMasterNotRegisteredException(message);
       }
@@ -365,7 +367,7 @@ public class ApplicationMasterService extends AbstractService implements
       this.amLivelinessMonitor.receivedPing(applicationAttemptId);
 
       RMApp rmApp =
-          rmContext.getRMApps().get(applicationAttemptId.getApplicationId());
+          rmContext.getRMApps().get(appId);
 
       if (rmApp.isAppFinalStateStored()) {
         return FinishApplicationMasterResponse.newInstance(true);
@@ -418,6 +420,7 @@ public class ApplicationMasterService extends AbstractService implements
 
     ApplicationAttemptId appAttemptId =
         amrmTokenIdentifier.getApplicationAttemptId();
+    ApplicationId applicationId = appAttemptId.getApplicationId();
 
     this.amLivelinessMonitor.receivedPing(appAttemptId);
 
@@ -432,14 +435,14 @@ public class ApplicationMasterService extends AbstractService implements
       if (!hasApplicationMasterRegistered(appAttemptId)) {
         String message =
             "Application Master is not registered for known application: "
-                + appAttemptId.getApplicationId()
+                + applicationId
                 + ". Let AM resync.";
         LOG.info(message);
         RMAuditLogger.logFailure(
-            this.rmContext.getRMApps().get(appAttemptId.getApplicationId())
+            this.rmContext.getRMApps().get(applicationId)
                 .getUser(), AuditConstants.REGISTER_AM, "",
             "ApplicationMasterService", message,
-            appAttemptId.getApplicationId(),
+            applicationId,
             appAttemptId);
         return resync;
       }
@@ -481,11 +484,22 @@ public class ApplicationMasterService extends AbstractService implements
       List<String> blacklistRemovals =
           (blacklistRequest != null) ?
               blacklistRequest.getBlacklistRemovals() : Collections.EMPTY_LIST;
-
+      RMApp app =
+          this.rmContext.getRMApps().get(applicationId);
+      
+      // set label expression for Resource Requests
+      ApplicationSubmissionContext asc = app.getApplicationSubmissionContext();
+      for (ResourceRequest req : ask) {
+        if (null == req.getNodeLabelExpression()) {
+          req.setNodeLabelExpression(asc.getNodeLabelExpression());
+        }
+      }
+              
       // sanity check
       try {
         RMServerUtils.validateResourceRequests(ask,
-            rScheduler.getMaximumResourceCapability());
+            rScheduler.getMaximumResourceCapability(), app.getQueue(),
+            rScheduler);
       } catch (InvalidResourceRequestException e) {
         LOG.warn("Invalid resource ask by application " + appAttemptId, e);
         throw e;
@@ -498,8 +512,6 @@ public class ApplicationMasterService extends AbstractService implements
         throw e;
       }
 
-      RMApp app =
-          this.rmContext.getRMApps().get(appAttemptId.getApplicationId());
       // In the case of work-preserving AM restart, it's possible for the
       // AM to release containers from the earlier attempt.
       if (!app.getApplicationSubmissionContext()
@@ -582,7 +594,7 @@ public class ApplicationMasterService extends AbstractService implements
             .toString(), amrmToken.getPassword(), amrmToken.getService()
             .toString()));
         LOG.info("The AMRMToken has been rolled-over. Send new AMRMToken back"
-            + " to application: " + appAttemptId.getApplicationId());
+            + " to application: " + applicationId);
       }
 
       /*

+ 25 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -343,7 +343,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       long submitTime, String user)
       throws YarnException {
     ApplicationId applicationId = submissionContext.getApplicationId();
-    validateResourceRequest(submissionContext);
+    ResourceRequest amReq = validateAndCreateResourceRequest(submissionContext);
     // Create RMApp
     RMAppImpl application =
         new RMAppImpl(applicationId, rmContext, this.conf,
@@ -351,7 +351,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
             submissionContext.getQueue(),
             submissionContext, this.scheduler, this.masterService,
             submitTime, submissionContext.getApplicationType(),
-            submissionContext.getApplicationTags());
+            submissionContext.getApplicationTags(), amReq);
 
     // Concurrent app submissions with same applicationId will fail here
     // Concurrent app submissions with different applicationIds will not
@@ -373,7 +373,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     return application;
   }
 
-  private void validateResourceRequest(
+  private ResourceRequest validateAndCreateResourceRequest(
       ApplicationSubmissionContext submissionContext)
       throws InvalidResourceRequestException {
     // Validation of the ApplicationSubmissionContext needs to be completed
@@ -383,18 +383,36 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
 
     // Check whether AM resource requirements are within required limits
     if (!submissionContext.getUnmanagedAM()) {
-      ResourceRequest amReq = BuilderUtils.newResourceRequest(
-          RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
-          submissionContext.getResource(), 1);
+      ResourceRequest amReq;
+      if (submissionContext.getAMContainerResourceRequest() != null) {
+        amReq = submissionContext.getAMContainerResourceRequest();
+      } else {
+        amReq =
+            BuilderUtils.newResourceRequest(
+                RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
+                submissionContext.getResource(), 1);
+      }
+      
+      // set label expression for AM container
+      if (null == amReq.getNodeLabelExpression()) {
+        amReq.setNodeLabelExpression(submissionContext
+            .getNodeLabelExpression());
+      }
+
       try {
         SchedulerUtils.validateResourceRequest(amReq,
-            scheduler.getMaximumResourceCapability());
+            scheduler.getMaximumResourceCapability(),
+            submissionContext.getQueue(), scheduler);
       } catch (InvalidResourceRequestException e) {
         LOG.warn("RM app submission failed in validating AM resource request"
             + " for application " + submissionContext.getApplicationId(), e);
         throw e;
       }
+      
+      return amReq;
     }
+    
+    return null;
   }
 
   private boolean isApplicationInFinalState(RMAppState rmAppState) {

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.conf.ConfigurationProvider;
 import org.apache.hadoop.yarn.event.Dispatcher;
 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.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -108,6 +109,10 @@ public interface RMContext {
 
   boolean isWorkPreservingRecoveryEnabled();
   
+  RMNodeLabelsManager getNodeLabelManager();
+  
+  public void setNodeLabelManager(RMNodeLabelsManager mgr);
+
   long getEpoch();
 
   ReservationSystem getReservationSystem();

+ 12 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 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.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
@@ -91,6 +92,7 @@ public class RMContextImpl implements RMContext {
   private RMApplicationHistoryWriter rmApplicationHistoryWriter;
   private SystemMetricsPublisher systemMetricsPublisher;
   private ConfigurationProvider configurationProvider;
+  private RMNodeLabelsManager nodeLabelManager;
   private long epoch;
   private Clock systemClock = new SystemClock();
   private long schedulerRecoveryStartTime = 0;
@@ -406,6 +408,16 @@ public class RMContextImpl implements RMContext {
     this.epoch = epoch;
   }
 
+  @Override
+  public RMNodeLabelsManager getNodeLabelManager() {
+    return nodeLabelManager;
+  }
+  
+  @Override
+  public void setNodeLabelManager(RMNodeLabelsManager mgr) {
+    nodeLabelManager = mgr;
+  }
+
   public void setSchedulerRecoveryStartAndWaitTime(long waitTime) {
     this.schedulerRecoveryStartTime = systemClock.getTime();
     this.schedulerRecoveryWaitTime = waitTime;

+ 15 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java

@@ -44,6 +44,7 @@ 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.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -84,9 +85,11 @@ public class RMServerUtils {
    * requested memory/vcore is non-negative and not greater than max
    */
   public static void validateResourceRequests(List<ResourceRequest> ask,
-      Resource maximumResource) throws InvalidResourceRequestException {
+      Resource maximumResource, String queueName, YarnScheduler scheduler)
+      throws InvalidResourceRequestException {
     for (ResourceRequest resReq : ask) {
-      SchedulerUtils.validateResourceRequest(resReq, maximumResource);
+      SchedulerUtils.validateResourceRequest(resReq, maximumResource,
+          queueName, scheduler);
     }
   }
 
@@ -132,17 +135,25 @@ public class RMServerUtils {
     }
   }
 
+  public static UserGroupInformation verifyAccess(
+      AccessControlList acl, String method, final Log LOG)
+      throws IOException {
+    // by default, this method will use AdminService as module name
+    return verifyAccess(acl, method, "AdminService", LOG);
+  }
+
   /**
    * Utility method to verify if the current user has access based on the
    * passed {@link AccessControlList}
    * @param acl the {@link AccessControlList} to check against
    * @param method the method name to be logged
+   * @param module, like AdminService or NodeLabelManager
    * @param LOG the logger to use
    * @return {@link UserGroupInformation} of the current user
    * @throws IOException
    */
   public static UserGroupInformation verifyAccess(
-      AccessControlList acl, String method, final Log LOG)
+      AccessControlList acl, String method, String module, final Log LOG)
       throws IOException {
     UserGroupInformation user;
     try {
@@ -159,7 +170,7 @@ public class RMServerUtils {
           " to call '" + method + "'");
 
       RMAuditLogger.logFailure(user.getShortUserName(), method,
-          acl.toString(), "AdminService",
+          acl.toString(), module,
           RMAuditLogger.AuditConstants.UNAUTHORIZED_USER);
 
       throw new AccessControlException("User " + user.getShortUserName() +

+ 10 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMaste
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 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;
@@ -320,6 +321,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return new AMLivelinessMonitor(this.rmDispatcher);
   }
   
+  protected RMNodeLabelsManager createNodeLabelManager() {
+    return new RMNodeLabelsManager();
+  }
+  
   protected DelegationTokenRenewer createDelegationTokenRenewer() {
     return new DelegationTokenRenewer();
   }
@@ -399,6 +404,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
       AMLivelinessMonitor amFinishingMonitor = createAMLivelinessMonitor();
       addService(amFinishingMonitor);
       rmContext.setAMFinishingMonitor(amFinishingMonitor);
+      
+      RMNodeLabelsManager nlm = createNodeLabelManager();
+      addService(nlm);
+      rmContext.setNodeLabelManager(nlm);
 
       boolean isRecoveryEnabled = conf.getBoolean(
           YarnConfiguration.RECOVERY_ENABLED,
@@ -962,7 +971,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
    * instance of {@link RMActiveServices} and initializes it.
    * @throws Exception
    */
-  void createAndInitActiveServices() throws Exception {
+  protected void createAndInitActiveServices() throws Exception {
     activeServices = new RMActiveServices();
     activeServices.init(conf);
   }

+ 13 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -126,14 +127,18 @@ public class CapacitySchedulerPlanFollower implements PlanFollower {
     // create the default reservation queue if it doesnt exist
     String defReservationQueue = planQueueName + PlanQueue.DEFAULT_QUEUE_SUFFIX;
     if (scheduler.getQueue(defReservationQueue) == null) {
-      ReservationQueue defQueue =
-          new ReservationQueue(scheduler, defReservationQueue, planQueue);
       try {
+        ReservationQueue defQueue =
+            new ReservationQueue(scheduler, defReservationQueue, planQueue);
         scheduler.addQueue(defQueue);
       } catch (SchedulerDynamicEditException e) {
         LOG.warn(
             "Exception while trying to create default reservation queue for plan: {}",
             planQueueName, e);
+      } catch (IOException e) {
+        LOG.warn(
+            "Exception while trying to create default reservation queue for plan: {}",
+            planQueueName, e);
       }
     }
     curReservationNames.add(defReservationQueue);
@@ -186,14 +191,18 @@ public class CapacitySchedulerPlanFollower implements PlanFollower {
       for (ReservationAllocation res : sortedAllocations) {
         String currResId = res.getReservationId().toString();
         if (curReservationNames.contains(currResId)) {
-          ReservationQueue resQueue =
-              new ReservationQueue(scheduler, currResId, planQueue);
           try {
+            ReservationQueue resQueue =
+                new ReservationQueue(scheduler, currResId, planQueue);
             scheduler.addQueue(resQueue);
           } catch (SchedulerDynamicEditException e) {
             LOG.warn(
                 "Exception while trying to activate reservation: {} for plan: {}",
                 currResId, planQueueName, e);
+          } catch (IOException e) {
+            LOG.warn(
+                "Exception while trying to activate reservation: {} for plan: {}",
+                currResId, planQueueName, e);
           }
         }
         Resource capToAssign = res.getResourcesAtTime(now);

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -143,6 +144,7 @@ public class RMAppImpl implements RMApp, Recoverable {
   private RMAppEvent eventCausingFinalSaving;
   private RMAppState targetedFinalState;
   private RMAppState recoveredFinalState;
+  private ResourceRequest amReq;
 
   Object transitionTodo;
 
@@ -342,7 +344,8 @@ public class RMAppImpl implements RMApp, Recoverable {
       Configuration config, String name, String user, String queue,
       ApplicationSubmissionContext submissionContext, YarnScheduler scheduler,
       ApplicationMasterService masterService, long submitTime,
-      String applicationType, Set<String> applicationTags) {
+      String applicationType, Set<String> applicationTags, 
+      ResourceRequest amReq) {
 
     this.systemClock = new SystemClock();
 
@@ -361,6 +364,7 @@ public class RMAppImpl implements RMApp, Recoverable {
     this.startTime = this.systemClock.getTime();
     this.applicationType = applicationType;
     this.applicationTags = applicationTags;
+    this.amReq = amReq;
 
     int globalMaxAppAttempts = conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
@@ -732,7 +736,7 @@ public class RMAppImpl implements RMApp, Recoverable {
           // previously failed attempts(which should not include Preempted,
           // hardware error and NM resync) + 1) equal to the max-attempt
           // limit.
-          maxAppAttempts == (getNumFailedAppAttempts() + 1));
+          maxAppAttempts == (getNumFailedAppAttempts() + 1), amReq);
     attempts.put(appAttemptId, attempt);
     currentAttempt = attempt;
   }

+ 26 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -93,7 +93,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -177,6 +176,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   private Object transitionTodo;
   
   private RMAppAttemptMetrics attemptMetrics = null;
+  private ResourceRequest amReq = null;
 
   private static final StateMachineFactory<RMAppAttemptImpl,
                                            RMAppAttemptState,
@@ -426,7 +426,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       RMContext rmContext, YarnScheduler scheduler,
       ApplicationMasterService masterService,
       ApplicationSubmissionContext submissionContext,
-      Configuration conf, boolean maybeLastAttempt) {
+      Configuration conf, boolean maybeLastAttempt, ResourceRequest amReq) {
     this.conf = conf;
     this.applicationAttemptId = appAttemptId;
     this.rmContext = rmContext;
@@ -442,8 +442,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     this.proxiedTrackingUrl = generateProxyUriWithScheme(null);
     this.maybeLastAttempt = maybeLastAttempt;
     this.stateMachine = stateMachineFactory.make(this);
+
     this.attemptMetrics =
         new RMAppAttemptMetrics(applicationAttemptId, rmContext);
+    
+    this.amReq = amReq;
   }
 
   @Override
@@ -885,24 +888,34 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   private static final List<ResourceRequest> EMPTY_CONTAINER_REQUEST_LIST =
       new ArrayList<ResourceRequest>();
 
-  private static final class ScheduleTransition
+  @VisibleForTesting
+  public static final class ScheduleTransition
       implements
       MultipleArcTransition<RMAppAttemptImpl, RMAppAttemptEvent, RMAppAttemptState> {
     @Override
     public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
         RMAppAttemptEvent event) {
-      if (!appAttempt.submissionContext.getUnmanagedAM()) {
-        // Request a container for the AM.
-        ResourceRequest request =
-            BuilderUtils.newResourceRequest(
-                AM_CONTAINER_PRIORITY, ResourceRequest.ANY, appAttempt
-                    .getSubmissionContext().getResource(), 1);
-
+      ApplicationSubmissionContext subCtx = appAttempt.submissionContext;
+      if (!subCtx.getUnmanagedAM()) {
+        // Need reset #containers before create new attempt, because this request
+        // will be passed to scheduler, and scheduler will deduct the number after
+        // AM container allocated
+        
+        // Currently, following fields are all hard code,
+        // TODO: change these fields when we want to support
+        // priority/resource-name/relax-locality specification for AM containers
+        // allocation.
+        appAttempt.amReq.setNumContainers(1);
+        appAttempt.amReq.setPriority(AM_CONTAINER_PRIORITY);
+        appAttempt.amReq.setResourceName(ResourceRequest.ANY);
+        appAttempt.amReq.setRelaxLocality(true);
+        
         // SchedulerUtils.validateResourceRequests is not necessary because
         // AM resource has been checked when submission
-        Allocation amContainerAllocation = appAttempt.scheduler.allocate(
-            appAttempt.applicationAttemptId,
-            Collections.singletonList(request), EMPTY_CONTAINER_RELEASE_LIST, null, null);
+        Allocation amContainerAllocation =
+            appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
+                Collections.singletonList(appAttempt.amReq),
+                EMPTY_CONTAINER_RELEASE_LIST, null, null);
         if (amContainerAllocation != null
             && amContainerAllocation.getContainers() != null) {
           assert (amContainerAllocation.getContainers().size() == 0);

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmnode;
 
 
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -135,4 +136,11 @@ public interface RMNode {
    * @return containerUpdates accumulated across NM heartbeats.
    */
   public List<UpdatedContainerInfo> pullContainerUpdates();
+  
+  /**
+   * Get set of labels in this node
+   * 
+   * @return labels in this node
+   */
+  public Set<String> getNodeLabels();
 }

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java

@@ -855,4 +855,12 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   public Set<ContainerId> getLaunchedContainers() {
     return this.launchedContainers;
   }
+
+  @Override
+  public Set<String> getNodeLabels() {
+    if (context.getNodeLabelManager() == null) {
+      return null;
+    }
+    return context.getNodeLabelManager().getLabelsOnNode(nodeId);
+  }
  }

+ 19 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
@@ -71,4 +72,22 @@ public interface Queue {
    */
   public void recoverContainer(Resource clusterResource,
       SchedulerApplicationAttempt schedulerAttempt, RMContainer rmContainer);
+  
+  /**
+   * Get labels can be accessed of this queue
+   * labels={*}, means this queue can access any label
+   * labels={ }, means this queue cannot access any label except node without label
+   * labels={a, b, c} means this queue can access a or b or c  
+   * @return labels
+   */
+  public Set<String> getAccessibleNodeLabels();
+  
+  /**
+   * Get default label expression of this queue. If label expression of
+   * ApplicationSubmissionContext and label expression of Resource Request not
+   * set, this will be used.
+   * 
+   * @return default label expression
+   */
+  public String getDefaultNodeLabelExpression();
 }

+ 120 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java

@@ -17,23 +17,29 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import java.io.IOException;
 import java.util.List;
+import java.util.Set;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import com.google.common.collect.Sets;
+
 /**
  * Utilities shared by schedulers. 
  */
@@ -190,7 +196,8 @@ public class SchedulerUtils {
    *         request
    */
   public static void validateResourceRequest(ResourceRequest resReq,
-      Resource maximumResource) throws InvalidResourceRequestException {
+      Resource maximumResource, String queueName, YarnScheduler scheduler)
+      throws InvalidResourceRequestException {
     if (resReq.getCapability().getMemory() < 0 ||
         resReq.getCapability().getMemory() > maximumResource.getMemory()) {
       throw new InvalidResourceRequestException("Invalid resource request"
@@ -209,5 +216,116 @@ public class SchedulerUtils {
           + resReq.getCapability().getVirtualCores()
           + ", maxVirtualCores=" + maximumResource.getVirtualCores());
     }
+    
+    // Get queue from scheduler
+    QueueInfo queueInfo = null;
+    try {
+      queueInfo = scheduler.getQueueInfo(queueName, false, false);
+    } catch (IOException e) {
+      // it is possible queue cannot get when queue mapping is set, just ignore
+      // the queueInfo here, and move forward
+    }
+
+    // check labels in the resource request.
+    String labelExp = resReq.getNodeLabelExpression();
+    
+    // if queue has default label expression, and RR doesn't have, use the
+    // default label expression of queue
+    if (labelExp == null && queueInfo != null) {
+      labelExp = queueInfo.getDefaultNodeLabelExpression();
+      resReq.setNodeLabelExpression(labelExp);
+    }
+    
+    if (labelExp != null && !labelExp.trim().isEmpty() && queueInfo != null) {
+      if (!checkQueueLabelExpression(queueInfo.getAccessibleNodeLabels(),
+          labelExp)) {
+        throw new InvalidResourceRequestException("Invalid resource request"
+            + ", queue="
+            + queueInfo.getQueueName()
+            + " doesn't have permission to access all labels "
+            + "in resource request. labelExpression of resource request="
+            + labelExp
+            + ". Queue labels="
+            + (queueInfo.getAccessibleNodeLabels() == null ? "" : StringUtils.join(queueInfo
+                .getAccessibleNodeLabels().iterator(), ',')));
+      }
+    }
+  }
+  
+  public static boolean checkQueueAccessToNode(Set<String> queueLabels,
+      Set<String> nodeLabels) {
+    // if queue's label is *, it can access any node
+    if (queueLabels != null && queueLabels.contains(RMNodeLabelsManager.ANY)) {
+      return true;
+    }
+    // any queue can access to a node without label
+    if (nodeLabels == null || nodeLabels.isEmpty()) {
+      return true;
+    }
+    // a queue can access to a node only if it contains any label of the node
+    if (queueLabels != null
+        && Sets.intersection(queueLabels, nodeLabels).size() > 0) {
+      return true;
+    }
+    // sorry, you cannot access
+    return false;
+  }
+  
+  public static void checkIfLabelInClusterNodeLabels(RMNodeLabelsManager mgr,
+      Set<String> labels) throws IOException {
+    if (mgr == null) {
+      if (labels != null && !labels.isEmpty()) {
+        throw new IOException("NodeLabelManager is null, please check");
+      }
+      return;
+    }
+
+    if (labels != null) {
+      for (String label : labels) {
+        if (!label.equals(RMNodeLabelsManager.ANY)
+            && !mgr.containsNodeLabel(label)) {
+          throw new IOException("NodeLabelManager doesn't include label = "
+              + label + ", please check.");
+        }
+      }
+    }
+  }
+  
+  public static boolean checkNodeLabelExpression(Set<String> nodeLabels,
+      String labelExpression) {
+    // empty label expression can only allocate on node with empty labels
+    if (labelExpression == null || labelExpression.trim().isEmpty()) {
+      if (!nodeLabels.isEmpty()) {
+        return false;
+      }
+    }
+
+    if (labelExpression != null) {
+      for (String str : labelExpression.split("&&")) {
+        if (!str.trim().isEmpty()
+            && (nodeLabels == null || !nodeLabels.contains(str.trim()))) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  public static boolean checkQueueLabelExpression(Set<String> queueLabels,
+      String labelExpression) {
+    if (queueLabels != null && queueLabels.contains(RMNodeLabelsManager.ANY)) {
+      return true;
+    }
+    // if label expression is empty, we can allocate container on any node
+    if (labelExpression == null) {
+      return true;
+    }
+    for (String str : labelExpression.split("&&")) {
+      if (!str.trim().isEmpty()
+          && (queueLabels == null || !queueLabels.contains(str.trim()))) {
+        return false;
+      }
+    }
+    return true;
   }
 }

+ 448 - 0
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

@@ -0,0 +1,448 @@
+/**
+ * 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.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import com.google.common.collect.Sets;
+
+public abstract class AbstractCSQueue implements CSQueue {
+  
+  CSQueue parent;
+  final String queueName;
+  
+  float capacity;
+  float maximumCapacity;
+  float absoluteCapacity;
+  float absoluteMaxCapacity;
+  float absoluteUsedCapacity = 0.0f;
+
+  float usedCapacity = 0.0f;
+  volatile int numContainers;
+  
+  final Resource minimumAllocation;
+  final Resource maximumAllocation;
+  QueueState state;
+  final QueueMetrics metrics;
+  
+  final ResourceCalculator resourceCalculator;
+  Set<String> accessibleLabels;
+  RMNodeLabelsManager labelManager;
+  String defaultLabelExpression;
+  Resource usedResources = Resources.createResource(0, 0);
+  QueueInfo queueInfo;
+  Map<String, Float> absoluteCapacityByNodeLabels;
+  Map<String, Float> capacitiyByNodeLabels;
+  Map<String, Resource> usedResourcesByNodeLabels = new HashMap<String, Resource>();
+  Map<String, Float> absoluteMaxCapacityByNodeLabels;
+  Map<String, Float> maxCapacityByNodeLabels;
+  
+  Map<QueueACL, AccessControlList> acls = 
+      new HashMap<QueueACL, AccessControlList>();
+  boolean reservationsContinueLooking;
+  
+  private final RecordFactory recordFactory = 
+      RecordFactoryProvider.getRecordFactory(null);
+  
+  public AbstractCSQueue(CapacitySchedulerContext cs, 
+      String queueName, CSQueue parent, CSQueue old) throws IOException {
+    this.minimumAllocation = cs.getMinimumResourceCapability();
+    this.maximumAllocation = cs.getMaximumResourceCapability();
+    this.labelManager = cs.getRMContext().getNodeLabelManager();
+    this.parent = parent;
+    this.queueName = queueName;
+    this.resourceCalculator = cs.getResourceCalculator();
+    this.queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
+    
+    // must be called after parent and queueName is set
+    this.metrics = old != null ? old.getMetrics() :
+        QueueMetrics.forQueue(getQueuePath(), parent,
+            cs.getConfiguration().getEnableUserMetrics(),
+            cs.getConf());
+    
+    // get labels
+    this.accessibleLabels = cs.getConfiguration().getAccessibleNodeLabels(getQueuePath());
+    this.defaultLabelExpression = cs.getConfiguration()
+        .getDefaultNodeLabelExpression(getQueuePath());
+    
+    this.queueInfo.setQueueName(queueName);
+    
+    // inherit from parent if labels not set
+    if (this.accessibleLabels == null && parent != null) {
+      this.accessibleLabels = parent.getAccessibleNodeLabels();
+      SchedulerUtils.checkIfLabelInClusterNodeLabels(labelManager,
+          this.accessibleLabels);
+    }
+    
+    // inherit from parent if labels not set
+    if (this.defaultLabelExpression == null && parent != null
+        && this.accessibleLabels.containsAll(parent.getAccessibleNodeLabels())) {
+      this.defaultLabelExpression = parent.getDefaultNodeLabelExpression();
+    }
+    
+    // set capacity by labels
+    capacitiyByNodeLabels =
+        cs.getConfiguration().getNodeLabelCapacities(getQueuePath(), accessibleLabels,
+            labelManager);
+
+    // set maximum capacity by labels
+    maxCapacityByNodeLabels =
+        cs.getConfiguration().getMaximumNodeLabelCapacities(getQueuePath(),
+            accessibleLabels, labelManager);
+  }
+  
+  @Override
+  public synchronized float getCapacity() {
+    return capacity;
+  }
+
+  @Override
+  public synchronized float getAbsoluteCapacity() {
+    return absoluteCapacity;
+  }
+
+  @Override
+  public float getAbsoluteMaximumCapacity() {
+    return absoluteMaxCapacity;
+  }
+
+  @Override
+  public synchronized float getAbsoluteUsedCapacity() {
+    return absoluteUsedCapacity;
+  }
+
+  @Override
+  public float getMaximumCapacity() {
+    return maximumCapacity;
+  }
+
+  @Override
+  public synchronized float getUsedCapacity() {
+    return usedCapacity;
+  }
+
+  @Override
+  public synchronized Resource getUsedResources() {
+    return usedResources;
+  }
+
+  public synchronized int getNumContainers() {
+    return numContainers;
+  }
+
+  @Override
+  public synchronized QueueState getState() {
+    return state;
+  }
+  
+  @Override
+  public QueueMetrics getMetrics() {
+    return metrics;
+  }
+  
+  @Override
+  public String getQueueName() {
+    return queueName;
+  }
+  
+  @Override
+  public synchronized CSQueue getParent() {
+    return parent;
+  }
+
+  @Override
+  public synchronized void setParent(CSQueue newParentQueue) {
+    this.parent = (ParentQueue)newParentQueue;
+  }
+  
+  public Set<String> getAccessibleNodeLabels() {
+    return accessibleLabels;
+  }
+  
+  @Override
+  public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
+    synchronized (this) {
+      if (acls.get(acl).isUserAllowed(user)) {
+        return true;
+      }
+    }
+    
+    if (parent != null) {
+      return parent.hasAccess(acl, user);
+    }
+    
+    return false;
+  }
+  
+  @Override
+  public synchronized void setUsedCapacity(float usedCapacity) {
+    this.usedCapacity = usedCapacity;
+  }
+  
+  @Override
+  public synchronized void setAbsoluteUsedCapacity(float absUsedCapacity) {
+    this.absoluteUsedCapacity = absUsedCapacity;
+  }
+
+  /**
+   * Set maximum capacity - used only for testing.
+   * @param maximumCapacity new max capacity
+   */
+  synchronized void setMaxCapacity(float maximumCapacity) {
+    // Sanity check
+    CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
+    float absMaxCapacity =
+        CSQueueUtils.computeAbsoluteMaximumCapacity(maximumCapacity, parent);
+    CSQueueUtils.checkAbsoluteCapacity(getQueueName(), absoluteCapacity,
+        absMaxCapacity);
+    
+    this.maximumCapacity = maximumCapacity;
+    this.absoluteMaxCapacity = absMaxCapacity;
+  }
+
+  @Override
+  public float getAbsActualCapacity() {
+    // for now, simply return actual capacity = guaranteed capacity for parent
+    // queue
+    return absoluteCapacity;
+  }
+
+  @Override
+  public String getDefaultNodeLabelExpression() {
+    return defaultLabelExpression;
+  }
+  
+  synchronized void setupQueueConfigs(Resource clusterResource, float capacity,
+      float absoluteCapacity, float maximumCapacity, float absoluteMaxCapacity,
+      QueueState state, Map<QueueACL, AccessControlList> acls,
+      Set<String> labels, String defaultLabelExpression,
+      Map<String, Float> nodeLabelCapacities,
+      Map<String, Float> maximumNodeLabelCapacities,
+      boolean reservationContinueLooking)
+      throws IOException {
+    // Sanity check
+    CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
+    CSQueueUtils.checkAbsoluteCapacity(getQueueName(), absoluteCapacity,
+        absoluteMaxCapacity);
+
+    this.capacity = capacity;
+    this.absoluteCapacity = absoluteCapacity;
+
+    this.maximumCapacity = maximumCapacity;
+    this.absoluteMaxCapacity = absoluteMaxCapacity;
+
+    this.state = state;
+
+    this.acls = acls;
+    
+    // set labels
+    this.accessibleLabels = labels;
+    
+    // set label expression
+    this.defaultLabelExpression = defaultLabelExpression;
+    
+    // copy node label capacity
+    this.capacitiyByNodeLabels = new HashMap<String, Float>(nodeLabelCapacities);
+    this.maxCapacityByNodeLabels =
+        new HashMap<String, Float>(maximumNodeLabelCapacities);
+    
+    this.queueInfo.setAccessibleNodeLabels(this.accessibleLabels);
+    this.queueInfo.setCapacity(this.capacity);
+    this.queueInfo.setMaximumCapacity(this.maximumCapacity);
+    this.queueInfo.setQueueState(this.state);
+    this.queueInfo.setDefaultNodeLabelExpression(this.defaultLabelExpression);
+
+    // Update metrics
+    CSQueueUtils.updateQueueStatistics(
+        resourceCalculator, this, parent, clusterResource, minimumAllocation);
+    
+    // Check if labels of this queue is a subset of parent queue, only do this
+    // when we not root
+    if (parent != null && parent.getParent() != null) {
+      if (parent.getAccessibleNodeLabels() != null
+          && !parent.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
+        // if parent isn't "*", child shouldn't be "*" too
+        if (this.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
+          throw new IOException("Parent's accessible queue is not ANY(*), "
+              + "but child's accessible queue is *");
+        } else {
+          Set<String> diff =
+              Sets.difference(this.getAccessibleNodeLabels(),
+                  parent.getAccessibleNodeLabels());
+          if (!diff.isEmpty()) {
+            throw new IOException("Some labels of child queue is not a subset "
+                + "of parent queue, these labels=["
+                + StringUtils.join(diff, ",") + "]");
+          }
+        }
+      }
+    }
+    
+    // calculate absolute capacity by each node label
+    this.absoluteCapacityByNodeLabels =
+        CSQueueUtils.computeAbsoluteCapacityByNodeLabels(
+            this.capacitiyByNodeLabels, parent);
+    
+    // calculate maximum capacity by each node label
+    this.absoluteMaxCapacityByNodeLabels =
+        CSQueueUtils.computeAbsoluteMaxCapacityByNodeLabels(
+            maximumNodeLabelCapacities, parent);
+    
+    // check absoluteMaximumNodeLabelCapacities is valid
+    CSQueueUtils.checkAbsoluteCapacitiesByLabel(getQueueName(),
+        absoluteCapacityByNodeLabels, absoluteCapacityByNodeLabels);
+    
+    this.reservationsContinueLooking = reservationContinueLooking;
+  }
+  
+  @Private
+  public Resource getMaximumAllocation() {
+    return maximumAllocation;
+  }
+  
+  @Private
+  public Resource getMinimumAllocation() {
+    return minimumAllocation;
+  }
+  
+  synchronized void allocateResource(Resource clusterResource, 
+      Resource resource, Set<String> nodeLabels) {
+    Resources.addTo(usedResources, resource);
+    
+    // Update usedResources by labels
+    if (nodeLabels == null || nodeLabels.isEmpty()) {
+      if (!usedResourcesByNodeLabels.containsKey(RMNodeLabelsManager.NO_LABEL)) {
+        usedResourcesByNodeLabels.put(RMNodeLabelsManager.NO_LABEL,
+            Resources.createResource(0));
+      }
+      Resources.addTo(usedResourcesByNodeLabels.get(RMNodeLabelsManager.NO_LABEL),
+          resource);
+    } else {
+      for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
+        if (!usedResourcesByNodeLabels.containsKey(label)) {
+          usedResourcesByNodeLabels.put(label, Resources.createResource(0));
+        }
+        Resources.addTo(usedResourcesByNodeLabels.get(label), resource);
+      }
+    }
+
+    ++numContainers;
+    CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
+        clusterResource, minimumAllocation);
+  }
+  
+  protected synchronized void releaseResource(Resource clusterResource,
+      Resource resource, Set<String> nodeLabels) {
+    // Update queue metrics
+    Resources.subtractFrom(usedResources, resource);
+
+    // Update usedResources by labels
+    if (null == nodeLabels || nodeLabels.isEmpty()) {
+      if (!usedResourcesByNodeLabels.containsKey(RMNodeLabelsManager.NO_LABEL)) {
+        usedResourcesByNodeLabels.put(RMNodeLabelsManager.NO_LABEL,
+            Resources.createResource(0));
+      }
+      Resources.subtractFrom(
+          usedResourcesByNodeLabels.get(RMNodeLabelsManager.NO_LABEL), resource);
+    } else {
+      for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
+        if (!usedResourcesByNodeLabels.containsKey(label)) {
+          usedResourcesByNodeLabels.put(label, Resources.createResource(0));
+        }
+        Resources.subtractFrom(usedResourcesByNodeLabels.get(label), resource);
+      }
+    }
+
+    CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
+        clusterResource, minimumAllocation);
+    --numContainers;
+  }
+  
+  @Private
+  public float getCapacityByNodeLabel(String label) {
+    if (null == parent) {
+      return 1f;
+    }
+    
+    if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
+      return getCapacity();
+    }
+    
+    if (!capacitiyByNodeLabels.containsKey(label)) {
+      return 0;
+    } else {
+      return capacitiyByNodeLabels.get(label);
+    }
+  }
+  
+  @Private
+  public float getAbsoluteCapacityByNodeLabel(String label) {
+    if (null == parent) {
+      return 1; 
+    }
+    
+    if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
+      return getAbsoluteCapacity();
+    }
+    
+    if (!absoluteMaxCapacityByNodeLabels.containsKey(label)) {
+      return 0;
+    } else {
+      return absoluteMaxCapacityByNodeLabels.get(label);
+    }
+  }
+  
+  @Private
+  public float getAbsoluteMaximumCapacityByNodeLabel(String label) {
+    if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
+      return getAbsoluteMaximumCapacity();
+    }
+    
+    return getAbsoluteCapacityByNodeLabel(label);
+  }
+  
+  @Private
+  public boolean getReservationContinueLooking() {
+    return reservationsContinueLooking;
+  }
+  
+  @Private
+  public Map<QueueACL, AccessControlList> getACLs() {
+    return acls;
+  }
+}

+ 47 - 14
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

@@ -72,9 +72,18 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
   
   /**
    * Get the configured <em>capacity</em> of the queue.
-   * @return queue capacity
+   * @return configured queue capacity
    */
   public float getCapacity();
+  
+  /**
+   * Get actual <em>capacity</em> of the queue, this may be different from
+   * configured capacity when mis-config take place, like add labels to the
+   * cluster
+   * 
+   * @return actual queue capacity
+   */
+  public float getAbsActualCapacity();
 
   /**
    * Get capacity of the parent of the queue as a function of the 
@@ -105,29 +114,32 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
    */
   public float getAbsoluteUsedCapacity();
 
-  /**
-   * Get the current used capacity of the queue
-   * and it's children (if any).
-   * @return queue used capacity
-   */
-  public float getUsedCapacity();
-  
   /**
    * Set used capacity of the queue.
-   * @param usedCapacity used capacity of the queue
+   * @param usedCapacity
+   *          used capacity of the queue
    */
   public void setUsedCapacity(float usedCapacity);
-  
+
   /**
    * Set absolute used capacity of the queue.
-   * @param absUsedCapacity absolute used capacity of the queue
+   * @param absUsedCapacity
+   *          absolute used capacity of the queue
    */
   public void setAbsoluteUsedCapacity(float absUsedCapacity);
 
   /**
-   * Get the currently utilized resources in the cluster 
-   * by the queue and children (if any).
-   * @return used resources by the queue and it's children 
+   * Get the current used capacity of nodes without label(s) of the queue
+   * and it's children (if any).
+   * @return queue used capacity
+   */
+  public float getUsedCapacity();
+
+  /**
+   * Get the currently utilized resources which allocated at nodes without any
+   * labels in the cluster by the queue and children (if any).
+   * 
+   * @return used resources by the queue and it's children
    */
   public Resource getUsedResources();
   
@@ -259,4 +271,25 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
    */
   public void attachContainer(Resource clusterResource,
                FiCaSchedulerApp application, RMContainer container);
+  
+  /**
+   * Get absolute capacity by label of this queue can use 
+   * @param nodeLabel
+   * @return absolute capacity by label of this queue can use
+   */
+  public float getAbsoluteCapacityByNodeLabel(String nodeLabel);
+  
+  /**
+   * Get absolute max capacity by label of this queue can use 
+   * @param nodeLabel
+   * @return absolute capacity by label of this queue can use
+   */
+  public float getAbsoluteMaximumCapacityByNodeLabel(String nodeLabel);
+
+  /**
+   * Get capacity by node label
+   * @param nodeLabel
+   * @return capacity by node label
+   */
+  public float getCapacityByNodeLabel(String nodeLabel);
 }

+ 55 - 2
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

@@ -17,9 +17,12 @@
 */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -40,7 +43,7 @@ class CSQueueUtils {
     }
     }
 
-  public static void checkAbsoluteCapacities(String queueName,
+  public static void checkAbsoluteCapacity(String queueName,
       float absCapacity, float absMaxCapacity) {
     if (absMaxCapacity < (absCapacity - EPSILON)) {
       throw new IllegalArgumentException("Illegal call to setMaxCapacity. "
@@ -49,6 +52,23 @@ class CSQueueUtils {
           + ")");
   }
   }
+  
+  public static void checkAbsoluteCapacitiesByLabel(String queueName,
+          Map<String, Float> absCapacities,
+          Map<String, Float> absMaximumCapacities) {
+    for (Entry<String, Float> entry : absCapacities.entrySet()) {
+      String label = entry.getKey();
+      float absCapacity = entry.getValue();
+      float absMaxCapacity = absMaximumCapacities.get(label);
+      if (absMaxCapacity < (absCapacity - EPSILON)) {
+        throw new IllegalArgumentException("Illegal call to setMaxCapacity. "
+            + "Queue '" + queueName + "' has " + "an absolute capacity ("
+            + absCapacity + ") greater than "
+            + "its absolute maximumCapacity (" + absMaxCapacity + ") of label="
+            + label);
+      }
+    }
+  }
 
   public static float computeAbsoluteMaximumCapacity(
       float maximumCapacity, CSQueue parent) {
@@ -56,6 +76,39 @@ class CSQueueUtils {
         (parent == null) ? 1.0f : parent.getAbsoluteMaximumCapacity();
     return (parentAbsMaxCapacity * maximumCapacity);
   }
+  
+  public static Map<String, Float> computeAbsoluteCapacityByNodeLabels(
+      Map<String, Float> nodeLabelToCapacities, CSQueue parent) {
+    if (parent == null) {
+      return nodeLabelToCapacities;
+    }
+    
+    Map<String, Float> absoluteCapacityByNodeLabels =
+        new HashMap<String, Float>();
+    for (Entry<String, Float> entry : nodeLabelToCapacities.entrySet()) {
+      String label = entry.getKey();
+      float capacity = entry.getValue();
+      absoluteCapacityByNodeLabels.put(label,
+          capacity * parent.getAbsoluteCapacityByNodeLabel(label));
+    }
+    return absoluteCapacityByNodeLabels;
+  }
+  
+  public static Map<String, Float> computeAbsoluteMaxCapacityByNodeLabels(
+      Map<String, Float> maximumNodeLabelToCapacities, CSQueue parent) {
+    if (parent == null) {
+      return maximumNodeLabelToCapacities;
+    }
+    Map<String, Float> absoluteMaxCapacityByNodeLabels =
+        new HashMap<String, Float>();
+    for (Entry<String, Float> entry : maximumNodeLabelToCapacities.entrySet()) {
+      String label = entry.getKey();
+      float maxCapacity = entry.getValue();
+      absoluteMaxCapacityByNodeLabels.put(label,
+          maxCapacity * parent.getAbsoluteMaximumCapacityByNodeLabel(label));
+    }
+    return absoluteMaxCapacityByNodeLabels;
+  }
 
   public static int computeMaxActiveApplications(
       ResourceCalculator calculator,

+ 43 - 5
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

@@ -20,7 +20,15 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -53,8 +61,13 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.*;
+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.RMAppRejectedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@@ -191,6 +204,7 @@ public class CapacityScheduler extends
 
   private boolean scheduleAsynchronously;
   private AsyncScheduleThread asyncSchedulerThread;
+  private RMNodeLabelsManager labelManager;
   
   /**
    * EXPERT
@@ -275,6 +289,8 @@ public class CapacityScheduler extends
     this.applications =
         new ConcurrentHashMap<ApplicationId,
             SchedulerApplication<FiCaSchedulerApp>>();
+    this.labelManager = rmContext.getNodeLabelManager();
+
     initializeQueues(this.conf);
 
     scheduleAsynchronously = this.conf.getScheduleAynschronously();
@@ -446,7 +462,7 @@ public class CapacityScheduler extends
     root = 
         parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT, 
             queues, queues, noop);
-
+    labelManager.reinitializeQueueLabels(getQueueToLabels());
     LOG.info("Initialized root queue " + root);
     initializeQueueMappings();
   }
@@ -469,10 +485,19 @@ public class CapacityScheduler extends
     // Re-configure queues
     root.reinitialize(newRoot, clusterResource);
     initializeQueueMappings();
-    
+
     // Re-calculate headroom for active applications
     root.updateClusterResource(clusterResource);
 
+    labelManager.reinitializeQueueLabels(getQueueToLabels());
+  }
+  
+  private Map<String, Set<String>> getQueueToLabels() {
+    Map<String, Set<String>> queueToLabels = new HashMap<String, Set<String>>();
+    for (CSQueue queue : queues.values()) {
+      queueToLabels.put(queue.getQueueName(), queue.getAccessibleNodeLabels());
+    }
+    return queueToLabels;
   }
 
   /**
@@ -515,7 +540,7 @@ public class CapacityScheduler extends
   
   @Lock(CapacityScheduler.class)
   static CSQueue parseQueue(
-      CapacitySchedulerContext csContext, 
+      CapacitySchedulerContext csContext,
       CapacitySchedulerConfiguration conf, 
       CSQueue parent, String queueName, Map<String, CSQueue> queues,
       Map<String, CSQueue> oldQueues, 
@@ -1094,11 +1119,18 @@ public class CapacityScheduler extends
   }
 
   private synchronized void addNode(RMNode nodeManager) {
+    // update this node to node label manager
+    if (labelManager != null) {
+      labelManager.activateNode(nodeManager.getNodeID(),
+          nodeManager.getTotalCapability());
+    }
+    
     this.nodes.put(nodeManager.getNodeID(), new FiCaSchedulerNode(nodeManager,
         usePortForNodeName));
     Resources.addTo(clusterResource, nodeManager.getTotalCapability());
     root.updateClusterResource(clusterResource);
     int numNodes = numNodeManagers.incrementAndGet();
+    
     LOG.info("Added node " + nodeManager.getNodeAddress() + 
         " clusterResource: " + clusterResource);
 
@@ -1108,6 +1140,11 @@ public class CapacityScheduler extends
   }
 
   private synchronized void removeNode(RMNode nodeInfo) {
+    // update this node to node label manager
+    if (labelManager != null) {
+      labelManager.deactivateNode(nodeInfo.getNodeID());
+    }
+    
     FiCaSchedulerNode node = nodes.get(nodeInfo.getNodeID());
     if (node == null) {
       return;
@@ -1141,6 +1178,7 @@ public class CapacityScheduler extends
     }
 
     this.nodes.remove(nodeInfo.getNodeID());
+
     LOG.info("Removed node " + nodeInfo.getNodeAddress() + 
         " clusterResource: " + clusterResource);
   }

+ 147 - 1
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

@@ -18,7 +18,15 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringTokenizer;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -31,10 +39,14 @@ import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 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 com.google.common.collect.ImmutableSet;
+
 public class CapacitySchedulerConfiguration extends Configuration {
 
   private static final Log LOG = 
@@ -83,6 +95,12 @@ public class CapacitySchedulerConfiguration extends Configuration {
   public static final String STATE = "state";
   
   @Private
+  public static final String ACCESSIBLE_NODE_LABELS = "accessible-node-labels";
+  
+  @Private
+  public static final String DEFAULT_NODE_LABEL_EXPRESSION =
+      "default-node-label-expression";
+
   public static final String RESERVE_CONT_LOOK_ALL_NODES = PREFIX
       + "reservations-continue-look-all-nodes";
   
@@ -268,6 +286,10 @@ public class CapacitySchedulerConfiguration extends Configuration {
     return queueName;
   }
   
+  private String getNodeLabelPrefix(String queue, String label) {
+    return getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS + DOT + label + DOT;
+  }
+  
   public int getMaximumSystemApplications() {
     int maxApplications = 
       getInt(MAXIMUM_SYSTEM_APPLICATIONS, DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS);
@@ -343,6 +365,15 @@ public class CapacitySchedulerConfiguration extends Configuration {
         ", maxCapacity=" + maxCapacity);
   }
   
+  public void setCapacityByLabel(String queue, String label, float capacity) {
+    setFloat(getNodeLabelPrefix(queue, label) + CAPACITY, capacity);
+  }
+  
+  public void setMaximumCapacityByLabel(String queue, String label,
+      float capacity) {
+    setFloat(getNodeLabelPrefix(queue, label) + MAXIMUM_CAPACITY, capacity);
+  }
+  
   public int getUserLimit(String queue) {
     int userLimit = getInt(getQueuePrefix(queue) + USER_LIMIT,
         DEFAULT_USER_LIMIT);
@@ -372,6 +403,121 @@ public class CapacitySchedulerConfiguration extends Configuration {
         QueueState.valueOf(state.toUpperCase()) : QueueState.RUNNING;
   }
   
+  public void setAccessibleNodeLabels(String queue, Set<String> labels) {
+    if (labels == null) {
+      return;
+    }
+    String str = StringUtils.join(",", labels);
+    set(getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS, str);
+  }
+  
+  public Set<String> getAccessibleNodeLabels(String queue) {
+    String accessibleLabelStr =
+        get(getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS);
+
+    // When accessible-label is null, 
+    if (accessibleLabelStr == null) {
+      // Only return null when queue is not ROOT
+      if (!queue.equals(ROOT)) {
+        return null;
+      }
+    } else {
+      // print a warning when accessibleNodeLabel specified in config and queue
+      // is ROOT
+      if (queue.equals(ROOT)) {
+        LOG.warn("Accessible node labels for root queue will be ignored,"
+            + " it will be automatically set to \"*\".");
+      }
+    }
+
+    // always return ANY for queue root
+    if (queue.equals(ROOT)) {
+      return ImmutableSet.of(RMNodeLabelsManager.ANY);
+    }
+
+    // In other cases, split the accessibleLabelStr by ","
+    Set<String> set = new HashSet<String>();
+    for (String str : accessibleLabelStr.split(",")) {
+      if (!str.trim().isEmpty()) {
+        set.add(str.trim());
+      }
+    }
+    
+    // if labels contains "*", only keep ANY behind
+    if (set.contains(RMNodeLabelsManager.ANY)) {
+      set.clear();
+      set.add(RMNodeLabelsManager.ANY);
+    }
+    return Collections.unmodifiableSet(set);
+  }
+  
+  public Map<String, Float> getNodeLabelCapacities(String queue,
+      Set<String> labels, RMNodeLabelsManager mgr) {
+    Map<String, Float> nodeLabelCapacities = new HashMap<String, Float>();
+    
+    if (labels == null) {
+      return nodeLabelCapacities;
+    }
+
+    for (String label : labels.contains(CommonNodeLabelsManager.ANY) ? mgr
+        .getClusterNodeLabels() : labels) {
+      // capacity of all labels in each queue should be 1
+      if (org.apache.commons.lang.StringUtils.equals(ROOT, queue)) {
+        nodeLabelCapacities.put(label, 1.0f);
+        continue;
+      }
+      float capacity =
+          getFloat(getNodeLabelPrefix(queue, label) + CAPACITY, UNDEFINED);
+      if (capacity < MINIMUM_CAPACITY_VALUE
+          || capacity > MAXIMUM_CAPACITY_VALUE) {
+        throw new IllegalArgumentException("Illegal " + "capacity of "
+            + capacity + " for label=" + label + " in queue=" + queue);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("CSConf - getCapacityOfLabel: prefix="
+            + getNodeLabelPrefix(queue, label) + ", capacity=" + capacity);
+      }
+      
+      nodeLabelCapacities.put(label, capacity / 100f);
+    }
+    return nodeLabelCapacities;
+  }
+  
+  public Map<String, Float> getMaximumNodeLabelCapacities(String queue,
+      Set<String> labels, RMNodeLabelsManager mgr) {
+    Map<String, Float> maximumNodeLabelCapacities = new HashMap<String, Float>();
+    if (labels == null) {
+      return maximumNodeLabelCapacities;
+    }
+
+    for (String label : labels.contains(CommonNodeLabelsManager.ANY) ? mgr
+        .getClusterNodeLabels() : labels) {
+      float maxCapacity =
+          getFloat(getNodeLabelPrefix(queue, label) + MAXIMUM_CAPACITY,
+              UNDEFINED);
+      maxCapacity = (maxCapacity == DEFAULT_MAXIMUM_CAPACITY_VALUE) ? 
+          MAXIMUM_CAPACITY_VALUE : maxCapacity;
+      if (maxCapacity < MINIMUM_CAPACITY_VALUE
+          || maxCapacity > MAXIMUM_CAPACITY_VALUE) {
+        throw new IllegalArgumentException("Illegal " + "capacity of "
+            + maxCapacity + " for label=" + label + " in queue=" + queue);
+      }
+      LOG.debug("CSConf - getCapacityOfLabel: prefix="
+          + getNodeLabelPrefix(queue, label) + ", capacity=" + maxCapacity);
+      
+      maximumNodeLabelCapacities.put(label, maxCapacity / 100f);
+    }
+    return maximumNodeLabelCapacities;
+  }
+  
+  public String getDefaultNodeLabelExpression(String queue) {
+    return get(getQueuePrefix(queue) + DEFAULT_NODE_LABEL_EXPRESSION);
+  }
+  
+  public void setDefaultNodeLabelExpression(String queue, String exp) {
+    set(getQueuePrefix(queue) + DEFAULT_NODE_LABEL_EXPRESSION, exp);
+  }
+
   /*
    * Returns whether we should continue to look at all heart beating nodes even
    * after the reservation limit was hit. The node heart beating in could

Những thai đổi đã bị hủy bỏ vì nó quá lớn
+ 289 - 280
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


+ 139 - 252
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -23,12 +23,14 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -46,77 +48,42 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 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.rmcontainer.RMContainerState;
 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.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 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.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import com.google.common.collect.Sets;
+
 @Private
 @Evolving
-public class ParentQueue implements CSQueue {
+public class ParentQueue extends AbstractCSQueue {
 
   private static final Log LOG = LogFactory.getLog(ParentQueue.class);
 
-  private CSQueue parent;
-  private final String queueName;
-  
-  private float capacity;
-  private float maximumCapacity;
-  private float absoluteCapacity;
-  private float absoluteMaxCapacity;
-  private float absoluteUsedCapacity = 0.0f;
-
-  private float usedCapacity = 0.0f;
-
-  protected final Set<CSQueue> childQueues;
-  private final Comparator<CSQueue> queueComparator;
-  
-  private Resource usedResources = Resources.createResource(0, 0);
-  
+  protected final Set<CSQueue> childQueues;  
   private final boolean rootQueue;
-  
-  private final Resource minimumAllocation;
-
-  private volatile int numApplications;
-  private volatile int numContainers;
-
-  private QueueState state;
-
-  private final QueueMetrics metrics;
-
-  private QueueInfo queueInfo; 
-
-  private Map<QueueACL, AccessControlList> acls = 
-    new HashMap<QueueACL, AccessControlList>();
+  final Comparator<CSQueue> queueComparator;
+  volatile int numApplications;
 
   private final RecordFactory recordFactory = 
     RecordFactoryProvider.getRecordFactory(null);
 
-  private final ResourceCalculator resourceCalculator;
-
-  private boolean reservationsContinueLooking;
-  
   public ParentQueue(CapacitySchedulerContext cs, 
-      String queueName, CSQueue parent, CSQueue old) {
-    minimumAllocation = cs.getMinimumResourceCapability();
+      String queueName, CSQueue parent, CSQueue old) throws IOException {
+    super(cs, queueName, parent, old);
     
-    this.parent = parent;
-    this.queueName = queueName;
-    this.rootQueue = (parent == null);
-    this.resourceCalculator = cs.getResourceCalculator();
+    this.queueComparator = cs.getQueueComparator();
 
-    // must be called after parent and queueName is set
-    this.metrics = old != null ? old.getMetrics() :
-        QueueMetrics.forQueue(getQueuePath(), parent,
-			      cs.getConfiguration().getEnableUserMetrics(),
-			      cs.getConf());
+    this.rootQueue = (parent == null);
 
     float rawCapacity = cs.getConfiguration().getCapacity(getQueuePath());
 
@@ -141,17 +108,14 @@ public class ParentQueue implements CSQueue {
 
     Map<QueueACL, AccessControlList> acls = 
       cs.getConfiguration().getAcls(getQueuePath());
-    
-    this.queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
-    this.queueInfo.setQueueName(queueName);
+
     this.queueInfo.setChildQueues(new ArrayList<QueueInfo>());
 
-    setupQueueConfigs(cs.getClusterResource(),
-        capacity, absoluteCapacity, 
-        maximumCapacity, absoluteMaxCapacity, state, acls,
+    setupQueueConfigs(cs.getClusterResource(), capacity, absoluteCapacity,
+        maximumCapacity, absoluteMaxCapacity, state, acls, accessibleLabels,
+        defaultLabelExpression, capacitiyByNodeLabels, maxCapacityByNodeLabels, 
         cs.getConfiguration().getReservationContinueLook());
     
-    this.queueComparator = cs.getQueueComparator();
     this.childQueues = new TreeSet<CSQueue>(queueComparator);
 
     LOG.info("Initialized parent-queue " + queueName + 
@@ -159,41 +123,29 @@ public class ParentQueue implements CSQueue {
         ", fullname=" + getQueuePath()); 
   }
 
-  protected synchronized void setupQueueConfigs(
-      Resource clusterResource,
-      float capacity, float absoluteCapacity, 
-      float maximumCapacity, float absoluteMaxCapacity,
+  synchronized void setupQueueConfigs(Resource clusterResource, float capacity,
+      float absoluteCapacity, float maximumCapacity, float absoluteMaxCapacity,
       QueueState state, Map<QueueACL, AccessControlList> acls,
-      boolean continueLooking
-  ) {
-    // Sanity check
-    CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
-    CSQueueUtils.checkAbsoluteCapacities(getQueueName(), absoluteCapacity, absoluteMaxCapacity);
-
-    this.capacity = capacity;
-    this.absoluteCapacity = absoluteCapacity;
-
-    this.maximumCapacity = maximumCapacity;
-    this.absoluteMaxCapacity = absoluteMaxCapacity;
-
-    this.state = state;
-
-    this.acls = acls;
-    
-    this.queueInfo.setCapacity(this.capacity);
-    this.queueInfo.setMaximumCapacity(this.maximumCapacity);
-    this.queueInfo.setQueueState(this.state);
-
-    this.reservationsContinueLooking = continueLooking;
-
-    StringBuilder aclsString = new StringBuilder();
+      Set<String> accessibleLabels, String defaultLabelExpression,
+      Map<String, Float> nodeLabelCapacities,
+      Map<String, Float> maximumCapacitiesByLabel, 
+      boolean reservationContinueLooking) throws IOException {
+    super.setupQueueConfigs(clusterResource, capacity, absoluteCapacity,
+        maximumCapacity, absoluteMaxCapacity, state, acls, accessibleLabels,
+        defaultLabelExpression, nodeLabelCapacities, maximumCapacitiesByLabel,
+        reservationContinueLooking);
+   StringBuilder aclsString = new StringBuilder();
     for (Map.Entry<QueueACL, AccessControlList> e : acls.entrySet()) {
       aclsString.append(e.getKey() + ":" + e.getValue().getAclString());
     }
 
-    // Update metrics
-    CSQueueUtils.updateQueueStatistics(
-        resourceCalculator, this, parent, clusterResource, minimumAllocation);
+    StringBuilder labelStrBuilder = new StringBuilder(); 
+    if (accessibleLabels != null) {
+      for (String s : accessibleLabels) {
+        labelStrBuilder.append(s);
+        labelStrBuilder.append(",");
+      }
+    }
 
     LOG.info(queueName +
         ", capacity=" + capacity +
@@ -201,13 +153,13 @@ public class ParentQueue implements CSQueue {
         ", maxCapacity=" + maximumCapacity +
         ", asboluteMaxCapacity=" + absoluteMaxCapacity + 
         ", state=" + state +
-        ", acls=" + aclsString +
+        ", acls=" + aclsString + 
+        ", labels=" + labelStrBuilder.toString() + "\n" +
         ", reservationsContinueLooking=" + reservationsContinueLooking);
   }
 
   private static float PRECISION = 0.0005f; // 0.05% precision
   void setChildQueues(Collection<CSQueue> childQueues) {
-    
     // Validate
     float childCapacities = 0;
     for (CSQueue queue : childQueues) {
@@ -221,6 +173,21 @@ public class ParentQueue implements CSQueue {
       		" capacity of " + childCapacities + 
       		" for children of queue " + queueName);
     }
+    // check label capacities
+    for (String nodeLabel : labelManager.getClusterNodeLabels()) {
+      float capacityByLabel = getCapacityByNodeLabel(nodeLabel);
+      // check children's labels
+      float sum = 0;
+      for (CSQueue queue : childQueues) {
+        sum += queue.getCapacityByNodeLabel(nodeLabel);
+      }
+      if ((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);
+      }
+    }
     
     this.childQueues.clear();
     this.childQueues.addAll(childQueues);
@@ -228,21 +195,6 @@ public class ParentQueue implements CSQueue {
       LOG.debug("setChildQueues: " + getChildQueuesToPrint());
     }
   }
-  
-  @Override
-  public synchronized CSQueue getParent() {
-    return parent;
-  }
-
-  @Override
-  public synchronized void setParent(CSQueue newParentQueue) {
-    this.parent = (ParentQueue)newParentQueue;
-  }
-  
-  @Override
-  public String getQueueName() {
-    return queueName;
-  }
 
   @Override
   public String getQueuePath() {
@@ -250,65 +202,6 @@ public class ParentQueue implements CSQueue {
     return parentPath + getQueueName();
   }
 
-  @Override
-  public synchronized float getCapacity() {
-    return capacity;
-  }
-
-  @Override
-  public synchronized float getAbsoluteCapacity() {
-    return absoluteCapacity;
-  }
-
-  @Override
-  public float getAbsoluteMaximumCapacity() {
-    return absoluteMaxCapacity;
-  }
-
-  @Override
-  public synchronized float getAbsoluteUsedCapacity() {
-    return absoluteUsedCapacity;
-  }
-
-  @Override
-  public float getMaximumCapacity() {
-    return maximumCapacity;
-  }
-
-  @Override
-  public ActiveUsersManager getActiveUsersManager() {
-    // Should never be called since all applications are submitted to LeafQueues
-    return null;
-  }
-
-  @Override
-  public synchronized float getUsedCapacity() {
-    return usedCapacity;
-  }
-
-  @Override
-  public synchronized Resource getUsedResources() {
-    return usedResources;
-  }
-  
-  @Override
-  public synchronized List<CSQueue> getChildQueues() {
-    return new ArrayList<CSQueue>(childQueues);
-  }
-
-  public synchronized int getNumContainers() {
-    return numContainers;
-  }
-  
-  public synchronized int getNumApplications() {
-    return numApplications;
-  }
-
-  @Override
-  public synchronized QueueState getState() {
-    return state;
-  }
-
   @Override
   public synchronized QueueInfo getQueueInfo( 
       boolean includeChildQueues, boolean recursive) {
@@ -391,6 +284,10 @@ public class ParentQueue implements CSQueue {
         newlyParsedParentQueue.absoluteMaxCapacity,
         newlyParsedParentQueue.state, 
         newlyParsedParentQueue.acls,
+        newlyParsedParentQueue.accessibleLabels,
+        newlyParsedParentQueue.defaultLabelExpression,
+        newlyParsedParentQueue.capacitiyByNodeLabels,
+        newlyParsedParentQueue.maxCapacityByNodeLabels,
         newlyParsedParentQueue.reservationsContinueLooking);
 
     // Re-configure existing child queues and add new ones
@@ -434,21 +331,6 @@ public class ParentQueue implements CSQueue {
     }
     return queuesMap;
   }
-  
-  @Override
-  public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
-    synchronized (this) {
-      if (acls.get(acl).isUserAllowed(user)) {
-        return true;
-      }
-    }
-    
-    if (parent != null) {
-      return parent.hasAccess(acl, user);
-    }
-    
-    return false;
-  }
 
   @Override
   public void submitApplication(ApplicationId applicationId, String user,
@@ -521,7 +403,7 @@ public class ParentQueue implements CSQueue {
     }
   }
 
-  public synchronized void removeApplication(ApplicationId applicationId, 
+  private synchronized void removeApplication(ApplicationId applicationId, 
       String user) {
     
     --numApplications;
@@ -532,30 +414,6 @@ public class ParentQueue implements CSQueue {
         " leaf-queue of parent: " + getQueueName() + 
         " #applications: " + getNumApplications());
   }
-  
-  @Override
-  public synchronized void setUsedCapacity(float usedCapacity) {
-    this.usedCapacity = usedCapacity;
-  }
-  
-  @Override
-  public synchronized void setAbsoluteUsedCapacity(float absUsedCapacity) {
-    this.absoluteUsedCapacity = absUsedCapacity;
-  }
-
-  /**
-   * Set maximum capacity - used only for testing.
-   * @param maximumCapacity new max capacity
-   */
-  synchronized void setMaxCapacity(float maximumCapacity) {
-    // Sanity check
-    CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
-    float absMaxCapacity = CSQueueUtils.computeAbsoluteMaximumCapacity(maximumCapacity, parent);
-    CSQueueUtils.checkAbsoluteCapacities(getQueueName(), absoluteCapacity, absMaxCapacity);
-    
-    this.maximumCapacity = maximumCapacity;
-    this.absoluteMaxCapacity = absMaxCapacity;
-  }
 
   @Override
   public synchronized CSAssignment assignContainers(
@@ -563,6 +421,12 @@ public class ParentQueue implements CSQueue {
     CSAssignment assignment = 
         new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
     
+    // if our queue cannot access this node, just return
+    if (!SchedulerUtils.checkQueueAccessToNode(accessibleLabels,
+        labelManager.getLabelsOnNode(node.getNodeID()))) {
+      return assignment;
+    }
+    
     while (canAssign(clusterResource, node)) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Trying to assign containers to child-queue of "
@@ -570,8 +434,10 @@ public class ParentQueue implements CSQueue {
       }
       
       boolean localNeedToUnreserve = false;
+      Set<String> nodeLabels = labelManager.getLabelsOnNode(node.getNodeID()); 
+      
       // Are we over maximum-capacity for this queue?
-      if (!assignToQueue(clusterResource)) {
+      if (!canAssignToThisQueue(clusterResource, nodeLabels)) {
         // check to see if we could if we unreserve first
         localNeedToUnreserve = assignToQueueIfUnreserve(clusterResource);
         if (!localNeedToUnreserve) {
@@ -589,7 +455,8 @@ public class ParentQueue implements CSQueue {
               resourceCalculator, clusterResource, 
               assignedToChild.getResource(), Resources.none())) {
         // Track resource utilization for the parent-queue
-        allocateResource(clusterResource, assignedToChild.getResource());
+        super.allocateResource(clusterResource, assignedToChild.getResource(),
+            nodeLabels);
         
         // Track resource utilization in this pass of the scheduler
         Resources.addTo(assignment.getResource(), assignedToChild.getResource());
@@ -628,22 +495,41 @@ public class ParentQueue implements CSQueue {
     return assignment;
   }
 
-  private synchronized boolean assignToQueue(Resource clusterResource) {
-    // Check how of the cluster's absolute capacity we are currently using...
-    float currentCapacity =
-        Resources.divide(
-            resourceCalculator, clusterResource, 
-            usedResources, clusterResource);
+  private synchronized boolean canAssignToThisQueue(Resource clusterResource,
+      Set<String> nodeLabels) {
+    Set<String> labelCanAccess =
+        new HashSet<String>(
+            accessibleLabels.contains(CommonNodeLabelsManager.ANY) ? nodeLabels
+                : Sets.intersection(accessibleLabels, nodeLabels));
+    if (nodeLabels.isEmpty()) {
+      // Any queue can always access any node without label
+      labelCanAccess.add(RMNodeLabelsManager.NO_LABEL);
+    }
     
-    if (currentCapacity >= absoluteMaxCapacity) {
-      LOG.info(getQueueName() + 
-          " used=" + usedResources + 
-          " current-capacity (" + currentCapacity + ") " +
-          " >= max-capacity (" + absoluteMaxCapacity + ")");
-      return false;
+    boolean canAssign = true;
+    for (String label : labelCanAccess) {
+      if (!usedResourcesByNodeLabels.containsKey(label)) {
+        usedResourcesByNodeLabels.put(label, Resources.createResource(0));
+      }
+      float currentAbsoluteLabelUsedCapacity =
+          Resources.divide(resourceCalculator, clusterResource,
+              usedResourcesByNodeLabels.get(label),
+              labelManager.getResourceByLabel(label, clusterResource));
+      // if any of the label doesn't beyond limit, we can allocate on this node
+      if (currentAbsoluteLabelUsedCapacity >= 
+            getAbsoluteMaximumCapacityByNodeLabel(label)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(getQueueName() + " used=" + usedResources
+              + " current-capacity (" + usedResourcesByNodeLabels.get(label) + ") "
+              + " >= max-capacity ("
+              + labelManager.getResourceByLabel(label, clusterResource) + ")");
+        }
+        canAssign = false;
+        break;
+      }
     }
-    return true;
-
+    
+    return canAssign;
   }
 
   
@@ -685,7 +571,7 @@ public class ParentQueue implements CSQueue {
             node.getAvailableResource(), minimumAllocation);
   }
   
-  synchronized CSAssignment assignContainersToChildQueues(Resource cluster, 
+  private synchronized CSAssignment assignContainersToChildQueues(Resource cluster, 
       FiCaSchedulerNode node, boolean needToUnreserve) {
     CSAssignment assignment = 
         new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
@@ -728,11 +614,16 @@ public class ParentQueue implements CSQueue {
   String getChildQueuesToPrint() {
     StringBuilder sb = new StringBuilder();
     for (CSQueue q : childQueues) {
-      sb.append(q.getQueuePath() + "(" + q.getUsedCapacity() + "), ");
+      sb.append(q.getQueuePath() + 
+          "usedCapacity=(" + q.getUsedCapacity() + "), " + 
+          " label=("
+          + StringUtils.join(q.getAccessibleNodeLabels().iterator(), ",") 
+          + ")");
     }
     return sb.toString();
   }
-  void printChildQueues() {
+
+  private void printChildQueues() {
     if (LOG.isDebugEnabled()) {
       LOG.debug("printChildQueues - queue: " + getQueuePath()
         + " child-queues: " + getChildQueuesToPrint());
@@ -749,8 +640,8 @@ public class ParentQueue implements CSQueue {
       // Careful! Locking order is important!
       // Book keeping
       synchronized (this) {
-        releaseResource(clusterResource, 
-            rmContainer.getContainer().getResource());
+        super.releaseResource(clusterResource, rmContainer.getContainer()
+            .getResource(), labelManager.getLabelsOnNode(node.getNodeID()));
 
         LOG.info("completedContainer" +
             " queue=" + getQueueName() + 
@@ -787,27 +678,6 @@ public class ParentQueue implements CSQueue {
     }
   }
 
-  @Private
-  boolean getReservationContinueLooking() {
-    return reservationsContinueLooking;
-  }
-  
-  synchronized void allocateResource(Resource clusterResource, 
-      Resource resource) {
-    Resources.addTo(usedResources, resource);
-    CSQueueUtils.updateQueueStatistics(
-        resourceCalculator, this, parent, clusterResource, minimumAllocation);
-    ++numContainers;
-  }
-  
-  synchronized void releaseResource(Resource clusterResource, 
-      Resource resource) {
-    Resources.subtractFrom(usedResources, resource);
-    CSQueueUtils.updateQueueStatistics(
-        resourceCalculator, this, parent, clusterResource, minimumAllocation);
-    --numContainers;
-  }
-
   @Override
   public synchronized void updateClusterResource(Resource clusterResource) {
     // Update all children
@@ -821,10 +691,9 @@ public class ParentQueue implements CSQueue {
   }
   
   @Override
-  public QueueMetrics getMetrics() {
-    return metrics;
+  public synchronized List<CSQueue> getChildQueues() {
+    return new ArrayList<CSQueue>(childQueues);
   }
-
   
   @Override
   public void recoverContainer(Resource clusterResource,
@@ -834,12 +703,20 @@ public class ParentQueue implements CSQueue {
     }
     // Careful! Locking order is important! 
     synchronized (this) {
-      allocateResource(clusterResource,rmContainer.getContainer().getResource());
+      super.allocateResource(clusterResource, rmContainer.getContainer()
+          .getResource(), labelManager.getLabelsOnNode(rmContainer
+          .getContainer().getNodeId()));
     }
     if (parent != null) {
       parent.recoverContainer(clusterResource, attempt, rmContainer);
     }
   }
+  
+  @Override
+  public ActiveUsersManager getActiveUsersManager() {
+    // Should never be called since all applications are submitted to LeafQueues
+    return null;
+  }
 
   @Override
   public void collectSchedulerApplications(
@@ -853,8 +730,9 @@ public class ParentQueue implements CSQueue {
   public void attachContainer(Resource clusterResource,
       FiCaSchedulerApp application, RMContainer rmContainer) {
     if (application != null) {
-      allocateResource(clusterResource, rmContainer.getContainer()
-          .getResource());
+      super.allocateResource(clusterResource, rmContainer.getContainer()
+          .getResource(), labelManager.getLabelsOnNode(rmContainer
+          .getContainer().getNodeId()));
       LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
           + getAbsoluteUsedCapacity() + " used=" + usedResources + " cluster="
@@ -870,7 +748,9 @@ public class ParentQueue implements CSQueue {
   public void detachContainer(Resource clusterResource,
       FiCaSchedulerApp application, RMContainer rmContainer) {
     if (application != null) {
-      releaseResource(clusterResource, rmContainer.getContainer().getResource());
+      super.releaseResource(clusterResource,
+          rmContainer.getContainer().getResource(),
+          labelManager.getLabelsOnNode(rmContainer.getContainer().getNodeId()));
       LOG.info("movedContainer" + " queueMoveOut=" + getQueueName()
           + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
           + getAbsoluteUsedCapacity() + " used=" + usedResources + " cluster="
@@ -882,7 +762,14 @@ public class ParentQueue implements CSQueue {
     }
   }
 
-  public Map<QueueACL, AccessControlList> getACLs() {
-    return acls;
+  @Override
+  public float getAbsActualCapacity() {
+    // for now, simply return actual capacity = guaranteed capacity for parent
+    // queue
+    return absoluteCapacity;
+  }
+  
+  public synchronized int getNumApplications() {
+    return numApplications;
   }
 }

+ 7 - 1
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

@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -47,7 +49,7 @@ public class PlanQueue extends ParentQueue {
   private boolean showReservationsAsQueues;
 
   public PlanQueue(CapacitySchedulerContext cs, String queueName,
-      CSQueue parent, CSQueue old) {
+      CSQueue parent, CSQueue old) throws IOException {
     super(cs, queueName, parent, old);
 
     this.schedulerContext = cs;
@@ -104,6 +106,10 @@ public class PlanQueue extends ParentQueue {
         newlyParsedParentQueue.getMaximumCapacity(),
         newlyParsedParentQueue.getAbsoluteMaximumCapacity(),
         newlyParsedParentQueue.getState(), newlyParsedParentQueue.getACLs(),
+        newlyParsedParentQueue.accessibleLabels,
+        newlyParsedParentQueue.defaultLabelExpression,
+        newlyParsedParentQueue.capacitiyByNodeLabels,
+        newlyParsedParentQueue.maxCapacityByNodeLabels,
         newlyParsedParentQueue.getReservationContinueLooking());
 
     updateQuotas(newlyParsedParentQueue.userLimit,

+ 1 - 1
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

@@ -42,7 +42,7 @@ public class ReservationQueue extends LeafQueue {
   private int maxSystemApps;
 
   public ReservationQueue(CapacitySchedulerContext cs, String queueName,
-      PlanQueue parent) {
+      PlanQueue parent) throws IOException {
     super(cs, queueName, parent, null);
     maxSystemApps = cs.getConfiguration().getMaximumSystemApplications();
     // the following parameters are common to all reservation in the plan

+ 13 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -270,4 +271,16 @@ public abstract class FSQueue implements Queue, Schedulable {
     return String.format("[%s, demand=%s, running=%s, share=%s, w=%s]",
         getName(), getDemand(), getResourceUsage(), fairShare, getWeights());
   }
+  
+  @Override
+  public Set<String> getAccessibleNodeLabels() {
+    // TODO, add implementation for FS
+    return null;
+  }
+  
+  @Override
+  public String getDefaultNodeLabelExpression() {
+    // TODO, add implementation for FS
+    return null;
+  }
 }

+ 13 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -25,6 +25,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentSkipListMap;
 
 import org.apache.commons.logging.Log;
@@ -187,6 +188,18 @@ public class FifoScheduler extends
       updateAppHeadRoom(schedulerAttempt);
       updateAvailableResourcesMetrics();
     }
+
+    @Override
+    public Set<String> getAccessibleNodeLabels() {
+      // TODO add implementation for FIFO scheduler
+      return null;
+    }
+
+    @Override
+    public String getDefaultNodeLabelExpression() {
+      // TODO add implementation for FIFO scheduler
+      return null;
+    }
   };
 
   public FifoScheduler() {

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java

@@ -147,6 +147,7 @@ public class Application {
     return used;
   }
   
+  @SuppressWarnings("deprecation")
   public synchronized void submit() throws IOException, YarnException {
     ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     context.setApplicationId(this.applicationId);

+ 28 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java

@@ -135,34 +135,52 @@ public class MockAM {
   public void addContainerToBeReleased(ContainerId containerId) {
     releases.add(containerId);
   }
+  
   public AllocateResponse allocate(
       String host, int memory, int numContainers,
       List<ContainerId> releases) throws Exception {
-    List<ResourceRequest> reqs = createReq(new String[]{host}, memory, 1, numContainers);
+    return allocate(host, memory, numContainers, releases, null);
+  }
+  
+  public AllocateResponse allocate(
+      String host, int memory, int numContainers,
+      List<ContainerId> releases, String labelExpression) throws Exception {
+    List<ResourceRequest> reqs =
+        createReq(new String[] { host }, memory, 1, numContainers,
+            labelExpression);
     return allocate(reqs, releases);
   }
-
+  
   public List<ResourceRequest> createReq(String[] hosts, int memory, int priority,
       int containers) throws Exception {
+    return createReq(hosts, memory, priority, containers, null);
+  }
+
+  public List<ResourceRequest> createReq(String[] hosts, int memory, int priority,
+      int containers, String labelExpression) throws Exception {
     List<ResourceRequest> reqs = new ArrayList<ResourceRequest>();
     for (String host : hosts) {
       ResourceRequest hostReq = createResourceReq(host, memory, priority,
-          containers);
+          containers, labelExpression);
       reqs.add(hostReq);
       ResourceRequest rackReq = createResourceReq("/default-rack", memory,
-          priority, containers);
+          priority, containers, labelExpression);
       reqs.add(rackReq);
     }
 
     ResourceRequest offRackReq = createResourceReq(ResourceRequest.ANY, memory,
-        priority, containers);
+        priority, containers, labelExpression);
     reqs.add(offRackReq);
     return reqs;
-
   }
-
+  
   public ResourceRequest createResourceReq(String resource, int memory, int priority,
       int containers) throws Exception {
+    return createResourceReq(resource, memory, priority, containers, null);
+  }
+
+  public ResourceRequest createResourceReq(String resource, int memory, int priority,
+      int containers, String labelExpression) throws Exception {
     ResourceRequest req = Records.newRecord(ResourceRequest.class);
     req.setResourceName(resource);
     req.setNumContainers(containers);
@@ -172,6 +190,9 @@ public class MockAM {
     Resource capability = Records.newRecord(Resource.class);
     capability.setMemory(memory);
     req.setCapability(capability);
+    if (labelExpression != null) {
+     req.setNodeLabelExpression(labelExpression); 
+    }
     return req;
   }
 

+ 6 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -202,7 +203,11 @@ public class MockNodes {
     public long getLastHealthReportTime() {
       return lastHealthReportTime;
     }
-    
+
+    @Override
+    public Set<String> getNodeLabels() {
+      return null;
+    }
   };
 
   private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) {

+ 32 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -59,6 +59,8 @@ import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.DummyRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
@@ -110,6 +112,13 @@ public class MockRM extends ResourceManager {
     Logger rootLogger = LogManager.getRootLogger();
     rootLogger.setLevel(Level.DEBUG);    
   }
+  
+  @Override
+  protected RMNodeLabelsManager createNodeLabelManager() {
+    RMNodeLabelsManager mgr = new DummyRMNodeLabelsManager();
+    mgr.init(getConfig());
+    return mgr;
+  }
 
   public void waitForState(ApplicationId appId, RMAppState finalState)
       throws Exception {
@@ -183,27 +192,43 @@ public class MockRM extends ResourceManager {
     return launchAndRegisterAM(app, this, nm);
   }
 
-  public void waitForState(MockNM nm, ContainerId containerId,
+  public boolean waitForState(MockNM nm, ContainerId containerId,
       RMContainerState containerState) throws Exception {
+    // default is wait for 30,000 ms
+    return waitForState(nm, containerId, containerState, 30 * 1000);
+  }
+  
+  public boolean waitForState(MockNM nm, ContainerId containerId,
+      RMContainerState containerState, int timeoutMillisecs) throws Exception {
     RMContainer container = getResourceScheduler().getRMContainer(containerId);
     int timeoutSecs = 0;
-    while(container == null && timeoutSecs++ < 100) {
+    while(container == null && timeoutSecs++ < timeoutMillisecs / 100) {
       nm.nodeHeartbeat(true);
       container = getResourceScheduler().getRMContainer(containerId);
       System.out.println("Waiting for container " + containerId + " to be allocated.");
       Thread.sleep(100);
+      
+      if (timeoutMillisecs <= timeoutSecs * 100) {
+        return false;
+      }
     }
     Assert.assertNotNull("Container shouldn't be null", container);
-    timeoutSecs = 0;
-    while (!containerState.equals(container.getState()) && timeoutSecs++ < 40) {
+    while (!containerState.equals(container.getState())
+        && timeoutSecs++ < timeoutMillisecs / 100) {
       System.out.println("Container : " + containerId + " State is : "
           + container.getState() + " Waiting for state : " + containerState);
       nm.nodeHeartbeat(true);
-      Thread.sleep(300);
+      Thread.sleep(100);
+      
+      if (timeoutMillisecs <= timeoutSecs * 100) {
+        return false;
+      }
     }
+    
     System.out.println("Container State is : " + container.getState());
     Assert.assertEquals("Container state is not correct (timedout)",
       containerState, container.getState());
+    return true;
   }
 
   // get new application id
@@ -310,6 +335,7 @@ public class MockRM extends ResourceManager {
       isAppIdProvided, applicationId, 0, null);
   }
 
+  @SuppressWarnings("deprecation")
   public RMApp submitApp(int masterMemory,
       LogAggregationContext logAggregationContext) throws Exception {
     return submitApp(masterMemory, "", UserGroupInformation.getCurrentUser()
@@ -319,6 +345,7 @@ public class MockRM extends ResourceManager {
       false, null, 0, logAggregationContext);
    }
 
+  @SuppressWarnings("deprecation")
   public RMApp submitApp(int masterMemory, String name, String user,
       Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
       int maxAppAttempts, Credentials ts, String appType,

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -34,9 +35,11 @@ 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.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -150,7 +153,7 @@ public class RMHATestBase extends ClientBaseWithFixes{
               this.rmContext.getScheduler(),
               this.rmContext.getApplicationMasterService(),
               submitTime, submissionContext.getApplicationType(),
-              submissionContext.getApplicationTags());
+              submissionContext.getApplicationTags(), null);
       this.rmContext.getRMApps().put(submissionContext.getApplicationId(),
           application);
       //Do not send RMAppEventType.START event

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java

@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
+
 import static org.mockito.Matchers.isA;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
@@ -37,7 +38,6 @@ import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 
 import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.MockApps;
@@ -207,6 +207,7 @@ public class TestAppManager{
   private ApplicationSubmissionContext asContext;
   private ApplicationId appId;
 
+  @SuppressWarnings("deprecation")
   @Before
   public void setUp() {
     long now = System.currentTimeMillis();
@@ -540,6 +541,7 @@ public class TestAppManager{
     Assert.assertEquals("app state doesn't match", RMAppState.FINISHED, app.getState());
   }
 
+  @SuppressWarnings("deprecation")
   @Test (timeout = 30000)
   public void testRMAppSubmitInvalidResourceRequest() throws Exception {
     asContext.setResource(Resources.createResource(

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java

@@ -22,6 +22,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
@@ -30,7 +31,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -179,6 +179,7 @@ public class TestApplicationACLs {
     verifyAdministerQueueUserAccess();
   }
 
+  @SuppressWarnings("deprecation")
   private ApplicationId submitAppAndGetAppId(AccessControlList viewACL,
       AccessControlList modifyACL) throws Exception {
     SubmitApplicationRequest submitRequest = recordFactory

+ 30 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -44,13 +44,12 @@ import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CyclicBarrier;
 
-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.io.Text;
-import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.MockApps;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
@@ -87,7 +86,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
-import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -100,10 +98,11 @@ import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
 import org.apache.hadoop.yarn.api.records.ReservationRequests;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -138,10 +137,10 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -333,7 +332,7 @@ public class TestClientRMService {
         mock(ApplicationSubmissionContext.class);
     YarnConfiguration config = new YarnConfiguration();
     RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId,
-        rmContext, yarnScheduler, null, asContext, config, false);
+        rmContext, yarnScheduler, null, asContext, config, false, null);
     ApplicationResourceUsageReport report = rmAppAttemptImpl
         .getApplicationResourceUsageReport();
     assertEquals(report, RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT);
@@ -1061,6 +1060,7 @@ public class TestClientRMService {
     return mockSubmitAppRequest(appId, name, queue, tags, false);
   }
 
+  @SuppressWarnings("deprecation")
   private SubmitApplicationRequest mockSubmitAppRequest(ApplicationId appId,
         String name, String queue, Set<String> tags, boolean unmanaged) {
 
@@ -1150,26 +1150,32 @@ public class TestClientRMService {
       final long memorySeconds, final long vcoreSeconds) {
     ApplicationSubmissionContext asContext = mock(ApplicationSubmissionContext.class);
     when(asContext.getMaxAppAttempts()).thenReturn(1);
-    RMAppImpl app = spy(new RMAppImpl(applicationId3, rmContext, config, null,
-        null, queueName, asContext, yarnScheduler, null,
-        System.currentTimeMillis(), "YARN", null) {
-              @Override
-              public ApplicationReport createAndGetApplicationReport(
-                  String clientUserName, boolean allowAccess) {
-                ApplicationReport report = super.createAndGetApplicationReport(
-                    clientUserName, allowAccess);
-                ApplicationResourceUsageReport usageReport = 
-                    report.getApplicationResourceUsageReport();
-                usageReport.setMemorySeconds(memorySeconds);
-                usageReport.setVcoreSeconds(vcoreSeconds);
-                report.setApplicationResourceUsageReport(usageReport);
-                return report;
-              }
-          });
+
+    RMAppImpl app =
+        spy(new RMAppImpl(applicationId3, rmContext, config, null, null,
+            queueName, asContext, yarnScheduler, null,
+            System.currentTimeMillis(), "YARN", null,
+            BuilderUtils.newResourceRequest(
+                RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
+                Resource.newInstance(1024, 1), 1)){
+                  @Override
+                  public ApplicationReport createAndGetApplicationReport(
+                      String clientUserName, boolean allowAccess) {
+                    ApplicationReport report = super.createAndGetApplicationReport(
+                        clientUserName, allowAccess);
+                    ApplicationResourceUsageReport usageReport = 
+                        report.getApplicationResourceUsageReport();
+                    usageReport.setMemorySeconds(memorySeconds);
+                    usageReport.setVcoreSeconds(vcoreSeconds);
+                    report.setApplicationResourceUsageReport(usageReport);
+                    return report;
+                  }
+              });
+
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
         ApplicationId.newInstance(123456, 1), 1);
     RMAppAttemptImpl rmAppAttemptImpl = spy(new RMAppAttemptImpl(attemptId,
-        rmContext, yarnScheduler, null, asContext, config, false));
+        rmContext, yarnScheduler, null, asContext, config, false, null));
     Container container = Container.newInstance(
         ContainerId.newInstance(attemptId, 1), null, "", null, null, null);
     RMContainerImpl containerimpl = spy(new RMContainerImpl(container,
@@ -1230,7 +1236,7 @@ public class TestClientRMService {
     rm.start();
     MockNM nm;
     try {
-      nm = rm.registerNode("127.0.0.1:0", 102400, 100);
+      nm = rm.registerNode("127.0.0.1:1", 102400, 100);
       // allow plan follower to synchronize
       Thread.sleep(1050);
     } catch (Exception e) {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java

@@ -127,7 +127,7 @@ public class TestRMHA {
 
     try {
       rm.getNewAppId();
-      rm.registerNode("127.0.0.1:0", 2048);
+      rm.registerNode("127.0.0.1:1", 2048);
       app = rm.submitApp(1024);
       attempt = app.getCurrentAppAttempt();
       rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.SCHEDULED);

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java

@@ -292,7 +292,7 @@ public class TestWorkPreservingRMRestart {
       1e-8);
     // assert user consumed resources.
     assertEquals(usedResource, leafQueue.getUser(app.getUser())
-      .getConsumedResources());
+      .getTotalConsumedResources());
   }
 
   private void checkFifoQueue(SchedulerApplication schedulerApp,

+ 30 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java

@@ -17,13 +17,16 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
@@ -37,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ReservationRequestsPBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 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.security.ClientToAMTokenSecretManagerInRM;
@@ -44,6 +48,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManag
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.junit.Assert;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class ReservationSystemTestUtil {
 
@@ -55,6 +61,7 @@ public class ReservationSystemTestUtil {
     return ReservationId.newInstance(rand.nextLong(), rand.nextLong());
   }
 
+  @SuppressWarnings("unchecked")
   public CapacityScheduler mockCapacityScheduler(int numContainers)
       throws IOException {
     // stolen from TestCapacityScheduler
@@ -68,6 +75,29 @@ public class ReservationSystemTestUtil {
             new RMContainerTokenSecretManager(conf),
             new NMTokenSecretManagerInRM(conf),
             new ClientToAMTokenSecretManagerInRM(), null));
+    
+    RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
+    when(
+        nlm.getQueueResource(any(String.class), any(Set.class),
+            any(Resource.class))).thenAnswer(new Answer<Resource>() {
+      @Override
+      public Resource answer(InvocationOnMock invocation) throws Throwable {
+        Object[] args = invocation.getArguments();
+        return (Resource) args[2];
+      }
+    });
+    
+    when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
+        .thenAnswer(new Answer<Resource>() {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
+            Object[] args = invocation.getArguments();
+            return (Resource) args[1];
+          }
+        });
+    
+    mockRmContext.setNodeLabelManager(nlm);
+    
     cs.setRMContext(mockRmContext);
     try {
       cs.serviceInit(conf);

+ 9 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -63,6 +64,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessM
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@@ -73,6 +75,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretMan
 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.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -254,7 +257,7 @@ public class TestRMAppTransitions {
     RMApp application =
         new RMAppImpl(applicationId, rmContext, conf, name, user, queue,
           submissionContext, scheduler, masterService,
-          System.currentTimeMillis(), "YARN", null);
+          System.currentTimeMillis(), "YARN", null, null);
 
     testAppStartState(applicationId, user, name, queue, application);
     this.rmContext.getRMApps().putIfAbsent(application.getApplicationId(),
@@ -914,6 +917,7 @@ public class TestRMAppTransitions {
     }
   }
   
+  @SuppressWarnings("deprecation")
   public void testRecoverApplication(ApplicationState appState, RMState rmState)
       throws Exception {
     ApplicationSubmissionContext submissionContext =
@@ -923,7 +927,10 @@ public class TestRMAppTransitions {
             submissionContext.getApplicationName(), null,
             submissionContext.getQueue(), submissionContext, null, null,
             appState.getSubmitTime(), submissionContext.getApplicationType(),
-            submissionContext.getApplicationTags());
+            submissionContext.getApplicationTags(),
+            BuilderUtils.newResourceRequest(
+                RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
+                submissionContext.getResource(), 1));
     Assert.assertEquals(RMAppState.NEW, application.getState());
     application.recover(rmState);
 

+ 62 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -40,7 +40,9 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
@@ -62,7 +64,9 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -83,8 +87,8 @@ 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.RMAppFailedAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
@@ -96,7 +100,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
-
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -112,6 +115,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSe
 import org.apache.hadoop.yarn.server.security.MasterKeyData;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -122,6 +126,8 @@ import org.junit.runners.Parameterized;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Matchers;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 @RunWith(value = Parameterized.class)
 public class TestRMAppAttemptTransitions {
@@ -229,6 +235,7 @@ public class TestRMAppAttemptTransitions {
     this.isSecurityEnabled = isSecurityEnabled;
   }
 
+  @SuppressWarnings("deprecation")
   @Before
   public void setUp() throws Exception {
     AuthenticationMethod authMethod = AuthenticationMethod.SIMPLE;
@@ -300,6 +307,7 @@ public class TestRMAppAttemptTransitions {
     Mockito.doReturn(resourceScheduler).when(spyRMContext).getScheduler();
 
 
+    final String user = MockApps.newUserName();
     final String queue = MockApps.newQueue();
     submissionContext = mock(ApplicationSubmissionContext.class);
     when(submissionContext.getQueue()).thenReturn(queue);
@@ -315,7 +323,11 @@ public class TestRMAppAttemptTransitions {
     application = mock(RMAppImpl.class);
     applicationAttempt =
         new RMAppAttemptImpl(applicationAttemptId, spyRMContext, scheduler,
-          masterService, submissionContext, new Configuration(), false);
+            masterService, submissionContext, new Configuration(), false,
+            BuilderUtils.newResourceRequest(
+                RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
+                submissionContext.getResource(), 1));
+
     when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt);
     when(application.getApplicationId()).thenReturn(applicationId);
     spyRMContext.getRMApps().put(application.getApplicationId(), application);
@@ -1399,13 +1411,16 @@ public class TestRMAppAttemptTransitions {
   }
 
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testContainersCleanupForLastAttempt() {
     // create a failed attempt.
     applicationAttempt =
         new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), spyRMContext,
           scheduler, masterService, submissionContext, new Configuration(),
-          true);
+          true, BuilderUtils.newResourceRequest(
+              RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
+              submissionContext.getResource(), 1));
     when(submissionContext.getKeepContainersAcrossApplicationAttempts())
       .thenReturn(true);
     when(submissionContext.getMaxAppAttempts()).thenReturn(1);
@@ -1427,6 +1442,49 @@ public class TestRMAppAttemptTransitions {
     assertFalse(transferStateFromPreviousAttempt);
     verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
+  
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testScheduleTransitionReplaceAMContainerRequestWithDefaults() {
+    YarnScheduler mockScheduler = mock(YarnScheduler.class);
+    when(
+        mockScheduler.allocate(any(ApplicationAttemptId.class),
+            any(List.class), any(List.class), any(List.class), any(List.class)))
+        .thenAnswer(new Answer<Allocation>() {
+
+          @SuppressWarnings("rawtypes")
+          @Override
+          public Allocation answer(InvocationOnMock invocation)
+              throws Throwable {
+            ResourceRequest rr =
+                (ResourceRequest) ((List) invocation.getArguments()[1]).get(0);
+            
+            // capacity shouldn't changed
+            assertEquals(Resource.newInstance(3333, 1), rr.getCapability());
+            assertEquals("label-expression", rr.getNodeLabelExpression());
+            
+            // priority, #container, relax-locality will be changed
+            assertEquals(RMAppAttemptImpl.AM_CONTAINER_PRIORITY, rr.getPriority());
+            assertEquals(1, rr.getNumContainers());
+            assertEquals(ResourceRequest.ANY, rr.getResourceName());
+
+            // just return an empty allocation
+            List l = new ArrayList();
+            Set s = new HashSet();
+            return new Allocation(l, Resources.none(), s, s, l);
+          }
+        });
+    
+    // create an attempt.
+    applicationAttempt =
+        new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(),
+            spyRMContext, scheduler, masterService, submissionContext,
+            new Configuration(), true, ResourceRequest.newInstance(
+                Priority.UNDEFINED, "host1", Resource.newInstance(3333, 1), 3,
+                false, "label-expression"));
+    new RMAppAttemptImpl.ScheduleTransition().transition(
+        (RMAppAttemptImpl) applicationAttempt, null);
+  }
 
   private void verifyAMCrashAtAllocatedDiagnosticInfo(String diagnostics,
         int exitCode, boolean shouldCheckURL) {

+ 234 - 45
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java

@@ -21,13 +21,19 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
+import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -46,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -58,6 +65,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 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;
@@ -74,6 +82,8 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.Sets;
+
 public class TestSchedulerUtils {
 
   private static final Log LOG = LogFactory.getLog(TestSchedulerUtils.class);
@@ -173,69 +183,240 @@ public class TestSchedulerUtils {
     assertEquals(1, ask.getCapability().getVirtualCores());
     assertEquals(2048, ask.getCapability().getMemory());
   }
-
+  
   @Test (timeout = 30000)
-  public void testValidateResourceRequest() {
+  public void testValidateResourceRequestWithErrorLabelsPermission()
+      throws IOException {
+    // mock queue and scheduler
+    YarnScheduler scheduler = mock(YarnScheduler.class);
+    Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
+    QueueInfo queueInfo = mock(QueueInfo.class);
+    when(queueInfo.getQueueName()).thenReturn("queue");
+    when(queueInfo.getAccessibleNodeLabels()).thenReturn(queueAccessibleNodeLabels);
+    when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean()))
+        .thenReturn(queueInfo);
+    
     Resource maxResource = Resources.createResource(
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
 
-    // zero memory
+    // queue has labels, success cases
     try {
+      // set queue accessible node labesl to [x, y]
+      queueAccessibleNodeLabels.clear();
+      queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
       Resource resource = Resources.createResource(
           0,
           YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
           mock(Priority.class), ResourceRequest.ANY, resource, 1);
-      SchedulerUtils.validateResourceRequest(resReq, maxResource);
+      resReq.setNodeLabelExpression("x");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      
+      resReq.setNodeLabelExpression("x && y");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      
+      resReq.setNodeLabelExpression("y");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      
+      resReq.setNodeLabelExpression("");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      
+      resReq.setNodeLabelExpression(" ");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
     } catch (InvalidResourceRequestException e) {
-      fail("Zero memory should be accepted");
+      e.printStackTrace();
+      fail("Should be valid when request labels is a subset of queue labels");
     }
-
-    // zero vcores
+    
+    // queue has labels, failed cases (when ask a label not included by queue)
     try {
+      // set queue accessible node labesl to [x, y]
+      queueAccessibleNodeLabels.clear();
+      queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
+      
       Resource resource = Resources.createResource(
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-          0);
+          0,
+          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
           mock(Priority.class), ResourceRequest.ANY, resource, 1);
-      SchedulerUtils.validateResourceRequest(resReq, maxResource);
+      resReq.setNodeLabelExpression("z");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      fail("Should fail");
     } catch (InvalidResourceRequestException e) {
-      fail("Zero vcores should be accepted");
     }
-
-    // max memory
+    
     try {
+      // set queue accessible node labesl to [x, y]
+      queueAccessibleNodeLabels.clear();
+      queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
+      
       Resource resource = Resources.createResource(
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+          0,
           YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
           mock(Priority.class), ResourceRequest.ANY, resource, 1);
-      SchedulerUtils.validateResourceRequest(resReq, maxResource);
+      resReq.setNodeLabelExpression("x && y && z");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      fail("Should fail");
     } catch (InvalidResourceRequestException e) {
-      fail("Max memory should be accepted");
     }
-
-    // max vcores
+    
+    // queue doesn't have label, succeed (when request no label)
+    queueAccessibleNodeLabels.clear();
     try {
+      // set queue accessible node labels to empty
+      queueAccessibleNodeLabels.clear();
+      
       Resource resource = Resources.createResource(
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+          0,
+          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
           mock(Priority.class), ResourceRequest.ANY, resource, 1);
-      SchedulerUtils.validateResourceRequest(resReq, maxResource);
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      
+      resReq.setNodeLabelExpression("");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      
+      resReq.setNodeLabelExpression("  ");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
     } catch (InvalidResourceRequestException e) {
-      fail("Max vcores should not be accepted");
+      e.printStackTrace();
+      fail("Should be valid when request labels is empty");
     }
-
-    // negative memory
+    
+    // queue doesn't have label, failed (when request any label)
+    try {
+      // set queue accessible node labels to empty
+      queueAccessibleNodeLabels.clear();
+      
+      Resource resource = Resources.createResource(
+          0,
+          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+      ResourceRequest resReq = BuilderUtils.newResourceRequest(
+          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+      resReq.setNodeLabelExpression("x");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      fail("Should fail");
+    } catch (InvalidResourceRequestException e) {
+    }
+    
+    // queue is "*", always succeeded
     try {
+      // set queue accessible node labels to empty
+      queueAccessibleNodeLabels.clear();
+      queueAccessibleNodeLabels.add(RMNodeLabelsManager.ANY);
+      
       Resource resource = Resources.createResource(
-          -1,
+          0,
           YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
           mock(Priority.class), ResourceRequest.ANY, resource, 1);
-      SchedulerUtils.validateResourceRequest(resReq, maxResource);
+      resReq.setNodeLabelExpression("x");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      
+      resReq.setNodeLabelExpression("x && y && z");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+      
+      resReq.setNodeLabelExpression("z");
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
+          scheduler);
+    } catch (InvalidResourceRequestException e) {
+      e.printStackTrace();
+      fail("Should be valid when request labels is empty");
+    }
+  }
+
+  @Test (timeout = 30000)
+  public void testValidateResourceRequest() {
+    YarnScheduler mockScheduler = mock(YarnScheduler.class);
+
+    Resource maxResource =
+        Resources.createResource(
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+
+    // zero memory
+    try {
+      Resource resource =
+          Resources.createResource(0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+      ResourceRequest resReq =
+          BuilderUtils.newResourceRequest(mock(Priority.class),
+              ResourceRequest.ANY, resource, 1);
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
+          mockScheduler);
+    } catch (InvalidResourceRequestException e) {
+      fail("Zero memory should be accepted");
+    }
+
+    // zero vcores
+    try {
+      Resource resource =
+          Resources.createResource(
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0);
+      ResourceRequest resReq =
+          BuilderUtils.newResourceRequest(mock(Priority.class),
+              ResourceRequest.ANY, resource, 1);
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
+          mockScheduler);
+    } catch (InvalidResourceRequestException e) {
+      fail("Zero vcores should be accepted");
+    }
+
+    // max memory
+    try {
+      Resource resource =
+          Resources.createResource(
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+      ResourceRequest resReq =
+          BuilderUtils.newResourceRequest(mock(Priority.class),
+              ResourceRequest.ANY, resource, 1);
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
+          mockScheduler);
+    } catch (InvalidResourceRequestException e) {
+      fail("Max memory should be accepted");
+    }
+
+    // max vcores
+    try {
+      Resource resource =
+          Resources.createResource(
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+      ResourceRequest resReq =
+          BuilderUtils.newResourceRequest(mock(Priority.class),
+              ResourceRequest.ANY, resource, 1);
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
+          mockScheduler);
+    } catch (InvalidResourceRequestException e) {
+      fail("Max vcores should not be accepted");
+    }
+
+    // negative memory
+    try {
+      Resource resource =
+          Resources.createResource(-1,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+      ResourceRequest resReq =
+          BuilderUtils.newResourceRequest(mock(Priority.class),
+              ResourceRequest.ANY, resource, 1);
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
+          mockScheduler);
       fail("Negative memory should not be accepted");
     } catch (InvalidResourceRequestException e) {
       // expected
@@ -243,12 +424,14 @@ public class TestSchedulerUtils {
 
     // negative vcores
     try {
-      Resource resource = Resources.createResource(
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-          -1);
-      ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
-      SchedulerUtils.validateResourceRequest(resReq, maxResource);
+      Resource resource =
+          Resources.createResource(
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1);
+      ResourceRequest resReq =
+          BuilderUtils.newResourceRequest(mock(Priority.class),
+              ResourceRequest.ANY, resource, 1);
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
+          mockScheduler);
       fail("Negative vcores should not be accepted");
     } catch (InvalidResourceRequestException e) {
       // expected
@@ -256,12 +439,15 @@ public class TestSchedulerUtils {
 
     // more than max memory
     try {
-      Resource resource = Resources.createResource(
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
-      ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
-      SchedulerUtils.validateResourceRequest(resReq, maxResource);
+      Resource resource =
+          Resources.createResource(
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+      ResourceRequest resReq =
+          BuilderUtils.newResourceRequest(mock(Priority.class),
+              ResourceRequest.ANY, resource, 1);
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
+          mockScheduler);
       fail("More than max memory should not be accepted");
     } catch (InvalidResourceRequestException e) {
       // expected
@@ -269,13 +455,16 @@ public class TestSchedulerUtils {
 
     // more than max vcores
     try {
-      Resource resource = Resources.createResource(
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES
-          + 1);
-      ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
-      SchedulerUtils.validateResourceRequest(resReq, maxResource);
+      Resource resource =
+          Resources
+              .createResource(
+                  YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+                  YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
+      ResourceRequest resReq =
+          BuilderUtils.newResourceRequest(mock(Priority.class),
+              ResourceRequest.ANY, resource, 1);
+      SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
+          mockScheduler);
       fail("More than max vcores should not be accepted");
     } catch (InvalidResourceRequestException e) {
       // expected

+ 10 - 1
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

@@ -65,6 +65,9 @@ public class TestApplicationLimits {
   LeafQueue queue;
   
   private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
+
+  RMContext rmContext = null;
+
   
   @Before
   public void setUp() throws IOException {
@@ -73,7 +76,9 @@ public class TestApplicationLimits {
     YarnConfiguration conf = new YarnConfiguration();
     setupQueueConfiguration(csConf);
     
-    
+    rmContext = TestUtils.getMockRMContext();
+
+
     CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);
     when(csContext.getConfiguration()).thenReturn(csConf);
     when(csContext.getConf()).thenReturn(conf);
@@ -89,6 +94,8 @@ public class TestApplicationLimits {
         thenReturn(CapacityScheduler.queueComparator);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
+    when(csContext.getRMContext()).thenReturn(rmContext);
+    
     RMContainerTokenSecretManager containerTokenSecretManager =
         new RMContainerTokenSecretManager(conf);
     containerTokenSecretManager.rollMasterKey();
@@ -162,6 +169,7 @@ public class TestApplicationLimits {
     when(csContext.getQueueComparator()).
         thenReturn(CapacityScheduler.queueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getRMContext()).thenReturn(rmContext);
     
     // Say cluster has 100 nodes of 16G each
     Resource clusterResource = Resources.createResource(100 * 16 * GB, 100 * 16);
@@ -475,6 +483,7 @@ public class TestApplicationLimits {
     when(csContext.getQueueComparator()).
         thenReturn(CapacityScheduler.queueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getRMContext()).thenReturn(rmContext);
     
     // Say cluster has 100 nodes of 16G each
     Resource clusterResource = Resources.createResource(100 * 16 * GB);

+ 8 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCSQueueUtils.java

@@ -19,38 +19,19 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.when;
 
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 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.Resources;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
-import org.mockito.InOrder;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 
 public class TestCSQueueUtils {
 
@@ -88,6 +69,8 @@ public class TestCSQueueUtils {
         thenReturn(Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).
         thenReturn(Resources.createResource(0, 0));
+    RMContext rmContext = TestUtils.getMockRMContext();
+    when(csContext.getRMContext()).thenReturn(rmContext);
   
     final String L1Q1 = "L1Q1";
     csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1});
@@ -129,6 +112,8 @@ public class TestCSQueueUtils {
         thenReturn(Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).
         thenReturn(Resources.createResource(16*GB, 32));
+    RMContext rmContext = TestUtils.getMockRMContext();
+    when(csContext.getRMContext()).thenReturn(rmContext);
     
     final String L1Q1 = "L1Q1";
     csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1});
@@ -174,6 +159,9 @@ public class TestCSQueueUtils {
     when(csContext.getMaximumResourceCapability()).
         thenReturn(Resources.createResource(16*GB, 32));
     
+    RMContext rmContext = TestUtils.getMockRMContext();
+    when(csContext.getRMContext()).thenReturn(rmContext);
+    
     final String L1Q1 = "L1Q1";
     final String L1Q2 = "L1Q2";
     final String L2Q1 = "L2Q1";

+ 20 - 8
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

@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -64,7 +65,6 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -84,6 +84,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.Task;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.DummyRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
@@ -149,7 +151,14 @@ public class TestCapacityScheduler {
   
   @Before
   public void setUp() throws Exception {
-    resourceManager = new ResourceManager();
+    resourceManager = new ResourceManager() {
+      @Override
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        RMNodeLabelsManager mgr = new DummyRMNodeLabelsManager();
+        mgr.init(getConfig());
+        return mgr;
+      }
+    };
     CapacitySchedulerConfiguration csConf 
        = new CapacitySchedulerConfiguration();
     setupQueueConfiguration(csConf);
@@ -962,10 +971,7 @@ public class TestCapacityScheduler {
     YarnConfiguration conf = new YarnConfiguration();
     CapacityScheduler cs = new CapacityScheduler();
     cs.setConf(conf);
-    RMContextImpl rmContext =  new RMContextImpl(null, null, null, null, null,
-        null, new RMContainerTokenSecretManager(conf),
-        new NMTokenSecretManagerInRM(conf),
-        new ClientToAMTokenSecretManagerInRM(), null);
+    RMContext rmContext = TestUtils.getMockRMContext();
     cs.setRMContext(rmContext);
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration();
@@ -1476,8 +1482,14 @@ public class TestCapacityScheduler {
 
   @Test(expected = YarnException.class)
   public void testMoveAppViolateQueueState() throws Exception {
-
-    resourceManager = new ResourceManager();
+    resourceManager = new ResourceManager() {
+       @Override
+        protected RMNodeLabelsManager createNodeLabelManager() {
+          RMNodeLabelsManager mgr = new DummyRMNodeLabelsManager();
+          mgr.init(getConfig());
+          return mgr;
+        }
+    };
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration();
     setupQueueConfiguration(csConf);

+ 3 - 2
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

@@ -99,6 +99,7 @@ public class TestChildQueueOrder {
     thenReturn(CapacityScheduler.queueComparator);
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
+    when(csContext.getRMContext()).thenReturn(rmContext);
   }
 
   private FiCaSchedulerApp getMockApplication(int appId, String user) {
@@ -132,11 +133,11 @@ public class TestChildQueueOrder {
         final Resource allocatedResource = Resources.createResource(allocation);
         if (queue instanceof ParentQueue) {
           ((ParentQueue)queue).allocateResource(clusterResource, 
-              allocatedResource);
+              allocatedResource, null);
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
-              allocatedResource);
+              allocatedResource, null);
         }
 
         // Next call - nothing

+ 460 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java

@@ -20,12 +20,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.util.ArrayList;
 import java.util.List;
+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.security.SecurityUtilTestHelper;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
@@ -41,19 +43,28 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.RMSecretManagerService;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.TestFifoScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.DummyRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
 
 public class TestContainerAllocation {
 
@@ -62,12 +73,16 @@ public class TestContainerAllocation {
   private final int GB = 1024;
 
   private YarnConfiguration conf;
+  
+  RMNodeLabelsManager mgr;
 
   @Before
   public void setUp() throws Exception {
     conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
       ResourceScheduler.class);
+    mgr = new DummyRMNodeLabelsManager();
+    mgr.init(conf);
   }
 
   @Test(timeout = 3000000)
@@ -305,4 +320,449 @@ public class TestContainerAllocation {
     rm1.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.ALLOCATED);
     MockRM.launchAndRegisterAM(app1, rm1, nm1);
   }
+  
+  private Configuration getConfigurationWithDefaultQueueLabels(
+      Configuration config) {
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    
+    CapacitySchedulerConfiguration conf =
+        (CapacitySchedulerConfiguration) getConfigurationWithQueueLabels(config);
+        new CapacitySchedulerConfiguration(config);
+    conf.setDefaultNodeLabelExpression(A, "x");
+    conf.setDefaultNodeLabelExpression(B, "y");
+    return conf;
+  }
+  
+  private Configuration getConfigurationWithQueueLabels(Configuration config) {
+    CapacitySchedulerConfiguration conf =
+        new CapacitySchedulerConfiguration(config);
+    
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 15);
+    conf.setAccessibleNodeLabels(A, toSet("x"));
+    conf.setCapacityByLabel(A, "x", 100);
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 20);
+    conf.setAccessibleNodeLabels(B, toSet("y"));
+    conf.setCapacityByLabel(B, "y", 100);
+    
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    conf.setCapacity(C, 70);
+    conf.setMaximumCapacity(C, 70);
+    conf.setAccessibleNodeLabels(C, RMNodeLabelsManager.EMPTY_STRING_SET);
+    
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    conf.setQueues(A, new String[] {"a1"});
+    conf.setCapacity(A1, 100);
+    conf.setMaximumCapacity(A1, 100);
+    conf.setCapacityByLabel(A1, "x", 100);
+    
+    final String B1 = B + ".b1";
+    conf.setQueues(B, new String[] {"b1"});
+    conf.setCapacity(B1, 100);
+    conf.setMaximumCapacity(B1, 100);
+    conf.setCapacityByLabel(B1, "y", 100);
+
+    final String C1 = C + ".c1";
+    conf.setQueues(C, new String[] {"c1"});
+    conf.setCapacity(C1, 100);
+    conf.setMaximumCapacity(C1, 100);
+    
+    return conf;
+  }
+  
+  private void checkTaskContainersHost(ApplicationAttemptId attemptId,
+      ContainerId containerId, ResourceManager rm, String host) {
+    YarnScheduler scheduler = rm.getRMContext().getScheduler();
+    SchedulerAppReport appReport = scheduler.getSchedulerAppInfo(attemptId);
+
+    Assert.assertTrue(appReport.getLiveContainers().size() > 0);
+    for (RMContainer c : appReport.getLiveContainers()) {
+      if (c.getContainerId().equals(containerId)) {
+        Assert.assertEquals(host, c.getAllocatedNode().getHost());
+      }
+    }
+  }
+  
+  private <E> Set<E> toSet(E... elements) {
+    Set<E> set = Sets.newHashSet(elements);
+    return set;
+  }
+  
+  private Configuration getComplexConfigurationWithQueueLabels(
+      Configuration config) {
+    CapacitySchedulerConfiguration conf =
+        new CapacitySchedulerConfiguration(config);
+    
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 10);
+    conf.setAccessibleNodeLabels(A, toSet("x", "y"));
+    conf.setCapacityByLabel(A, "x", 100);
+    conf.setCapacityByLabel(A, "y", 50);
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 90);
+    conf.setMaximumCapacity(B, 100);
+    conf.setAccessibleNodeLabels(B, toSet("y", "z"));
+    conf.setCapacityByLabel(B, "y", 50);
+    conf.setCapacityByLabel(B, "z", 100);
+    
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    conf.setQueues(A, new String[] {"a1"});
+    conf.setCapacity(A1, 100);
+    conf.setMaximumCapacity(A1, 100);
+    conf.setAccessibleNodeLabels(A1, toSet("x", "y"));
+    conf.setDefaultNodeLabelExpression(A1, "x");
+    conf.setCapacityByLabel(A1, "x", 100);
+    conf.setCapacityByLabel(A1, "y", 100);
+    
+    conf.setQueues(B, new String[] {"b1", "b2"});
+    final String B1 = B + ".b1";
+    conf.setCapacity(B1, 50);
+    conf.setMaximumCapacity(B1, 50);
+    conf.setAccessibleNodeLabels(B1, RMNodeLabelsManager.EMPTY_STRING_SET);
+
+    final String B2 = B + ".b2";
+    conf.setCapacity(B2, 50);
+    conf.setMaximumCapacity(B2, 50);
+    conf.setAccessibleNodeLabels(B2, toSet("y", "z"));
+    conf.setCapacityByLabel(B2, "y", 100);
+    conf.setCapacityByLabel(B2, "z", 100);
+
+    return conf;
+  }
+  
+  @Test(timeout = 300000)
+  public void testContainerAllocationWithSingleUserLimits() throws Exception {
+    final RMNodeLabelsManager mgr = new DummyRMNodeLabelsManager();
+    mgr.init(conf);
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
+        NodeId.newInstance("h2", 0), toSet("y")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(getConfigurationWithDefaultQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
+    MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
+    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
+
+    // launch an app to queue a1 (label = x), and check all container will
+    // be allocated in h1
+    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    
+    // A has only 10% of x, so it can only allocate one container in label=empty
+    ContainerId containerId =
+        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
+    Assert.assertTrue(rm1.waitForState(nm3, containerId,
+          RMContainerState.ALLOCATED, 10 * 1000));
+    // Cannot allocate 2nd label=empty container
+    containerId =
+        ContainerId.newInstance(am1.getApplicationAttemptId(), 3);
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
+    Assert.assertFalse(rm1.waitForState(nm3, containerId,
+          RMContainerState.ALLOCATED, 10 * 1000));
+
+    // A has default user limit = 100, so it can use all resource in label = x
+    // We can allocate floor(8000 / 1024) = 7 containers
+    for (int id = 3; id <= 8; id++) {
+      containerId =
+          ContainerId.newInstance(am1.getApplicationAttemptId(), id);
+      am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
+      Assert.assertTrue(rm1.waitForState(nm1, containerId,
+          RMContainerState.ALLOCATED, 10 * 1000));
+    }
+    rm1.close();
+  }
+  
+  @Test(timeout = 300000)
+  public void testContainerAllocateWithComplexLabels() throws Exception {
+    /*
+     * Queue structure:
+     *                      root (*)
+     *                  ________________
+     *                 /                \
+     *               a x(100%), y(50%)   b y(50%), z(100%)
+     *               ________________    ______________
+     *              /                   /              \
+     *             a1 (x,y)         b1(no)              b2(y,z)
+     *               100%                          y = 100%, z = 100%
+     *                           
+     * Node structure:
+     * h1 : x
+     * h2 : x, y
+     * h3 : y
+     * h4 : y, z
+     * h5 : NO
+     * 
+     * Total resource:
+     * x: 4G
+     * y: 6G
+     * z: 2G
+     * *: 2G
+     * 
+     * Resource of
+     * a1: x=4G, y=3G, NO=0.2G
+     * b1: NO=0.9G (max=1G)
+     * b2: y=3, z=2G, NO=0.9G (max=1G)
+     * 
+     * Each node can only allocate two containers
+     */
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y", "z"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0),
+        toSet("x"), NodeId.newInstance("h2", 0), toSet("x", "y"),
+        NodeId.newInstance("h3", 0), toSet("y"), NodeId.newInstance("h4", 0),
+        toSet("y", "z"), NodeId.newInstance("h5", 0),
+        RMNodeLabelsManager.EMPTY_STRING_SET));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(getComplexConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 2048);
+    MockNM nm2 = rm1.registerNode("h2:1234", 2048);
+    MockNM nm3 = rm1.registerNode("h3:1234", 2048);
+    MockNM nm4 = rm1.registerNode("h4:1234", 2048);
+    MockNM nm5 = rm1.registerNode("h5:1234", 2048);
+    
+    ContainerId containerId;
+
+    // launch an app to queue a1 (label = x), and check all container will
+    // be allocated in h1
+    RMApp app1 = rm1.submitApp(1024, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // request a container (label = x && y). can only allocate on nm2 
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x && y");
+    containerId =
+        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
+        "h2");
+
+    // launch an app to queue b1 (label = y), and check all container will
+    // be allocated in h5
+    RMApp app2 = rm1.submitApp(1024, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm5);
+
+    // request a container for AM, will succeed
+    // and now b1's queue capacity will be used, cannot allocate more containers
+    // (Maximum capacity reached)
+    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm4, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertFalse(rm1.waitForState(nm5, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    
+    // launch an app to queue b2
+    RMApp app3 = rm1.submitApp(1024, "app", "user", null, "b2");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm5);
+
+    // request a container. try to allocate on nm1 (label = x) and nm3 (label =
+    // y,z). Will successfully allocate on nm3
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
+    containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
+        "h3");
+    
+    // try to allocate container (request label = y && z) on nm3 (label = y) and
+    // nm4 (label = y,z). Will sucessfully allocate on nm4 only.
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y && z");
+    containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 3);
+    Assert.assertFalse(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm4, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
+        "h4");
+
+    rm1.close();
+  }
+
+  @Test (timeout = 120000)
+  public void testContainerAllocateWithLabels() throws Exception {
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
+        NodeId.newInstance("h2", 0), toSet("y")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
+    MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
+    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
+    
+    ContainerId containerId;
+
+    // launch an app to queue a1 (label = x), and check all container will
+    // be allocated in h1
+    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm3);
+
+    // request a container.
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
+    containerId =
+        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
+        "h1");
+
+    // launch an app to queue b1 (label = y), and check all container will
+    // be allocated in h2
+    RMApp app2 = rm1.submitApp(200, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm3);
+
+    // request a container.
+    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
+    containerId = ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am2.getApplicationAttemptId(), containerId, rm1,
+        "h2");
+    
+    // launch an app to queue c1 (label = ""), and check all container will
+    // be allocated in h3
+    RMApp app3 = rm1.submitApp(200, "app", "user", null, "c1");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm3);
+
+    // request a container.
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
+        "h3");
+
+    rm1.close();
+  }
+  
+  @Test (timeout = 120000)
+  public void testContainerAllocateWithDefaultQueueLabels() throws Exception {
+    // This test is pretty much similar to testContainerAllocateWithLabel.
+    // Difference is, this test doesn't specify label expression in ResourceRequest,
+    // instead, it uses default queue label expression
+
+    // set node -> label
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
+        NodeId.newInstance("h2", 0), toSet("y")));
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(getConfigurationWithDefaultQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
+    MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
+    MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
+    
+    ContainerId containerId;
+
+    // launch an app to queue a1 (label = x), and check all container will
+    // be allocated in h1
+    RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // request a container.
+    am1.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId =
+        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
+        "h1");
+
+    // launch an app to queue b1 (label = y), and check all container will
+    // be allocated in h2
+    RMApp app2 = rm1.submitApp(200, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // request a container.
+    am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am2.getApplicationAttemptId(), containerId, rm1,
+        "h2");
+    
+    // launch an app to queue c1 (label = ""), and check all container will
+    // be allocated in h3
+    RMApp app3 = rm1.submitApp(200, "app", "user", null, "c1");
+    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm3);
+
+    // request a container.
+    am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
+    containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 2);
+    Assert.assertFalse(rm1.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    Assert.assertTrue(rm1.waitForState(nm3, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
+        "h3");
+
+    rm1.close();
+  }
 }

+ 24 - 13
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

@@ -40,10 +40,10 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.junit.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 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.RMAppAttemptState;
@@ -82,6 +83,7 @@ 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.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Matchers;
@@ -147,6 +149,7 @@ public class TestLeafQueue {
         thenReturn(CapacityScheduler.queueComparator);
     when(csContext.getResourceCalculator()).
         thenReturn(resourceCalculator);
+    when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager =
         new RMContainerTokenSecretManager(conf);
     containerTokenSecretManager.rollMasterKey();
@@ -686,8 +689,9 @@ public class TestLeafQueue {
         1, qb.getActiveUsersManager().getNumActiveUsers());
     //get headroom
     qb.assignContainers(clusterResource, node_0, false);
-    qb.computeUserLimitAndSetHeadroom(app_0, clusterResource,
-        app_0.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability());
+    qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
+        .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
+        null);
 
     //maxqueue 16G, userlimit 13G, - 4G used = 9G
     assertEquals(9*GB,app_0.getHeadroom().getMemory());
@@ -704,8 +708,9 @@ public class TestLeafQueue {
             u1Priority, recordFactory)));
     qb.submitApplicationAttempt(app_2, user_1);
     qb.assignContainers(clusterResource, node_1, false);
-    qb.computeUserLimitAndSetHeadroom(app_0, clusterResource,
-         app_0.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability());
+    qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
+        .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
+        null);
 
     assertEquals(8*GB, qb.getUsedResources().getMemory());
     assertEquals(4*GB, app_0.getCurrentConsumption().getMemory());
@@ -718,8 +723,10 @@ public class TestLeafQueue {
     //test case 3
     qb.finishApplication(app_0.getApplicationId(), user_0);
     qb.finishApplication(app_2.getApplicationId(), user_1);
-    qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority));
-    qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority));
+    qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority),
+        null);
+    qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority),
+        null);
 
     qb.setUserLimit(50);
     qb.setUserLimitFactor(1);
@@ -744,8 +751,9 @@ public class TestLeafQueue {
     qb.submitApplicationAttempt(app_3, user_1);
     qb.assignContainers(clusterResource, node_0, false);
     qb.assignContainers(clusterResource, node_0, false);
-    qb.computeUserLimitAndSetHeadroom(app_3, clusterResource,
-        app_3.getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability());
+    qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3
+        .getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(),
+        null);
     assertEquals(4*GB, qb.getUsedResources().getMemory());
     //maxqueue 16G, userlimit 7G, used (by each user) 2G, headroom 5G (both)
     assertEquals(5*GB, app_3.getHeadroom().getMemory());
@@ -761,10 +769,12 @@ public class TestLeafQueue {
               TestUtils.createResourceRequest(ResourceRequest.ANY, 6*GB, 1, true,
                       u0Priority, recordFactory)));
     qb.assignContainers(clusterResource, node_1, false);
-    qb.computeUserLimitAndSetHeadroom(app_4, clusterResource,
-              app_4.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability());
-    qb.computeUserLimitAndSetHeadroom(app_3, clusterResource,
-        app_3.getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability());
+    qb.computeUserLimitAndSetHeadroom(app_4, clusterResource, app_4
+        .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
+        null);
+    qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3
+        .getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(),
+        null);
     
     
     //app3 is user1, active from last test case
@@ -2272,6 +2282,7 @@ public class TestLeafQueue {
     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, 

+ 3 - 2
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

@@ -95,6 +95,7 @@ public class TestParentQueue {
     thenReturn(CapacityScheduler.queueComparator);
     when(csContext.getResourceCalculator()).
     thenReturn(resourceComparator);
+    when(csContext.getRMContext()).thenReturn(rmContext);
   }
   
   private static final String A = "a";
@@ -144,11 +145,11 @@ public class TestParentQueue {
         final Resource allocatedResource = Resources.createResource(allocation);
         if (queue instanceof ParentQueue) {
           ((ParentQueue)queue).allocateResource(clusterResource, 
-              allocatedResource);
+              allocatedResource, null);
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
-              allocatedResource);
+              allocatedResource, null);
         }
         
         // Next call - nothing

+ 2 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueMappings.java

@@ -27,14 +27,11 @@ import org.apache.hadoop.security.GroupMappingServiceProvider;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 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.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SimpleGroupsMapping;
-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.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
@@ -79,10 +76,7 @@ public class TestQueueMappings {
     YarnConfiguration conf = new YarnConfiguration(csConf);
     CapacityScheduler cs = new CapacityScheduler();
 
-    RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
-        null, new RMContainerTokenSecretManager(conf),
-        new NMTokenSecretManagerInRM(conf),
-        new ClientToAMTokenSecretManagerInRM(), null);
+    RMContext rmContext = TestUtils.getMockRMContext();
     cs.setConf(conf);
     cs.setRMContext(rmContext);
     cs.init(conf);

+ 259 - 8
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

@@ -18,23 +18,41 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
-import org.junit.Assert;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
-import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableSet;
+
 public class TestQueueParsing {
 
   private static final Log LOG = LogFactory.getLog(TestQueueParsing.class);
   
   private static final double DELTA = 0.000001;
   
+  private RMNodeLabelsManager nodeLabelManager;
+  
+  @Before
+  public void setup() {
+    nodeLabelManager = mock(RMNodeLabelsManager.class);
+    when(nodeLabelManager.containsNodeLabel(any(String.class))).thenReturn(true);
+  }
+  
   @Test
   public void testQueueParsing() throws Exception {
     CapacitySchedulerConfiguration csConf = 
@@ -43,15 +61,11 @@ public class TestQueueParsing {
     YarnConfiguration conf = new YarnConfiguration(csConf);
 
     CapacityScheduler capacityScheduler = new CapacityScheduler();
-    RMContextImpl rmContext = new RMContextImpl(null, null,
-        null, null, null, null, new RMContainerTokenSecretManager(conf),
-        new NMTokenSecretManagerInRM(conf),
-        new ClientToAMTokenSecretManagerInRM(), null);
     capacityScheduler.setConf(conf);
-    capacityScheduler.setRMContext(rmContext);
+    capacityScheduler.setRMContext(TestUtils.getMockRMContext());
     capacityScheduler.init(conf);
     capacityScheduler.start();
-    capacityScheduler.reinitialize(conf, rmContext);
+    capacityScheduler.reinitialize(conf, TestUtils.getMockRMContext());
     
     CSQueue a = capacityScheduler.getQueue("a");
     Assert.assertEquals(0.10, a.getAbsoluteCapacity(), DELTA);
@@ -202,4 +216,241 @@ public class TestQueueParsing {
     capacityScheduler.stop();
   }
   
+  private void setupQueueConfigurationWithoutLabels(CapacitySchedulerConfiguration conf) {
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 15);
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 90);
+
+    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.setCapacity(A1, 30);
+    conf.setMaximumCapacity(A1, 45);
+    conf.setCapacity(A2, 70);
+    conf.setMaximumCapacity(A2, 85);
+    
+    final String B1 = B + ".b1";
+    final String B2 = B + ".b2";
+    final String B3 = B + ".b3";
+    conf.setQueues(B, new String[] {"b1", "b2", "b3"});
+    conf.setCapacity(B1, 50);
+    conf.setMaximumCapacity(B1, 85);
+    conf.setCapacity(B2, 30);
+    conf.setMaximumCapacity(B2, 35);
+    conf.setCapacity(B3, 20);
+    conf.setMaximumCapacity(B3, 35);
+  }
+  
+  private void setupQueueConfigurationWithLabels(CapacitySchedulerConfiguration conf) {
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 15);
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 90);
+
+    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.setCapacityByLabel(A, "blue", 50);
+    
+    conf.setCapacity(A1, 30);
+    conf.setMaximumCapacity(A1, 45);
+    conf.setCapacityByLabel(A1, "red", 50);
+    conf.setCapacityByLabel(A1, "blue", 100);
+    
+    conf.setCapacity(A2, 70);
+    conf.setMaximumCapacity(A2, 85);
+    conf.setAccessibleNodeLabels(A2, ImmutableSet.of("red"));
+    conf.setCapacityByLabel(A2, "red", 50);
+    
+    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.setCapacityByLabel(B, "blue", 50);
+    
+    conf.setCapacity(B1, 50);
+    conf.setMaximumCapacity(B1, 85);
+    conf.setCapacityByLabel(B1, "red", 50);
+    conf.setCapacityByLabel(B1, "blue", 50);
+    
+    conf.setCapacity(B2, 30);
+    conf.setMaximumCapacity(B2, 35);
+    conf.setCapacityByLabel(B2, "red", 25);
+    conf.setCapacityByLabel(B2, "blue", 25);
+    
+    conf.setCapacity(B3, 20);
+    conf.setMaximumCapacity(B3, 35);
+    conf.setCapacityByLabel(B3, "red", 25);
+    conf.setCapacityByLabel(B3, "blue", 25);
+  }
+  
+  private void setupQueueConfigurationWithLabelsInherit(
+      CapacitySchedulerConfiguration conf) {
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+
+    // Set A configuration
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 15);
+    conf.setQueues(A, new String[] {"a1", "a2"});
+    conf.setAccessibleNodeLabels(A, ImmutableSet.of("red", "blue"));
+    conf.setCapacityByLabel(A, "red", 100);
+    conf.setCapacityByLabel(A, "blue", 100);
+    
+    // Set B configuraiton
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 90);
+    conf.setAccessibleNodeLabels(B, CommonNodeLabelsManager.EMPTY_STRING_SET);
+    
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    final String A2 = A + ".a2";
+    
+    conf.setCapacity(A1, 30);
+    conf.setMaximumCapacity(A1, 45);
+    conf.setCapacityByLabel(A1, "red", 50);
+    conf.setCapacityByLabel(A1, "blue", 100);
+    
+    conf.setCapacity(A2, 70);
+    conf.setMaximumCapacity(A2, 85);
+    conf.setAccessibleNodeLabels(A2, ImmutableSet.of("red"));
+    conf.setCapacityByLabel(A2, "red", 50);
+  }
+  
+  @Test
+  public void testQueueParsingReinitializeWithLabels() throws IOException {
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    setupQueueConfigurationWithoutLabels(csConf);
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+
+    CapacityScheduler capacityScheduler = new CapacityScheduler();
+    RMContextImpl rmContext =
+        new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(conf),
+            new NMTokenSecretManagerInRM(conf),
+            new ClientToAMTokenSecretManagerInRM(), null);
+    rmContext.setNodeLabelManager(nodeLabelManager);
+    capacityScheduler.setConf(conf);
+    capacityScheduler.setRMContext(rmContext);
+    capacityScheduler.init(conf);
+    capacityScheduler.start();
+    csConf = new CapacitySchedulerConfiguration();
+    setupQueueConfigurationWithLabels(csConf);
+    conf = new YarnConfiguration(csConf);
+    capacityScheduler.reinitialize(conf, rmContext);
+    checkQueueLabels(capacityScheduler);
+    capacityScheduler.stop();
+  }
+  
+  private void checkQueueLabels(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")));
+  }
+  
+  private void
+      checkQueueLabelsInheritConfig(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()
+        .isEmpty());
+  }
+  
+  @Test
+  public void testQueueParsingWithLabels() throws IOException {
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(conf);
+    setupQueueConfigurationWithLabels(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();
+    checkQueueLabels(capacityScheduler);
+    capacityScheduler.stop();
+  }
+  
+  @Test
+  public void testQueueParsingWithLabelsInherit() throws IOException {
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(conf);
+    setupQueueConfigurationWithLabelsInherit(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();
+    checkQueueLabelsInheritConfig(capacityScheduler);
+    capacityScheduler.stop();
+  }
 }

+ 7 - 2
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

@@ -23,7 +23,10 @@ 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;
@@ -42,8 +45,7 @@ public class TestReservationQueue {
   ReservationQueue reservationQueue;
 
   @Before
-  public void setup() {
-
+  public void setup() throws IOException {
     // setup a context / conf
     csConf = new CapacitySchedulerConfiguration();
     YarnConfiguration conf = new YarnConfiguration();
@@ -57,6 +59,9 @@ public class TestReservationQueue {
     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);

+ 20 - 10
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

@@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
@@ -121,6 +122,7 @@ public class TestReservations {
     when(csContext.getQueueComparator()).thenReturn(
         CapacityScheduler.queueComparator);
     when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getRMContext()).thenReturn(rmContext);
     RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager(
         conf);
     containerTokenSecretManager.rollMasterKey();
@@ -819,7 +821,9 @@ public class TestReservations {
     // allocate to queue so that the potential new capacity is greater then
     // absoluteMaxCapacity
     Resource capability = Resources.createResource(32 * GB, 0);
-    boolean res = a.assignToQueue(clusterResource, capability, app_0, true);
+    boolean res =
+        a.canAssignToThisQueue(clusterResource, capability,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, true);
     assertFalse(res);
 
     // now add in reservations and make sure it continues if config set
@@ -836,23 +840,29 @@ public class TestReservations {
     assertEquals(3 * GB, node_1.getUsedResource().getMemory());
 
     capability = Resources.createResource(5 * GB, 0);
-    res = a
-        .assignToQueue(clusterResource, capability, app_0, true);
+    res =
+        a.canAssignToThisQueue(clusterResource, capability,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, true);
     assertTrue(res);
 
     // tell to not check reservations
-    res = a.assignToQueue(clusterResource, capability, app_0, false);
+    res =
+        a.canAssignToThisQueue(clusterResource, capability,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, false);
     assertFalse(res);
 
     refreshQueuesTurnOffReservationsContLook(a, csConf);
 
     // should return false no matter what checkReservations is passed
     // in since feature is off
-    res = a.assignToQueue(clusterResource, capability, app_0, false);
+    res =
+        a.canAssignToThisQueue(clusterResource, capability,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, false);
     assertFalse(res);
 
-    res = a
-        .assignToQueue(clusterResource, capability, app_0, true);
+    res =
+        a.canAssignToThisQueue(clusterResource, capability,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, true);
     assertFalse(res);
   }
 
@@ -1000,18 +1010,18 @@ public class TestReservations {
     // set limit so subtrace reservations it can continue
     Resource limit = Resources.createResource(12 * GB, 0);
     boolean res = a.assignToUser(clusterResource, user_0, limit, app_0,
-        true);
+        true, null);
     assertTrue(res);
 
     // tell it not to check for reservations and should fail as already over
     // limit
-    res = a.assignToUser(clusterResource, user_0, limit, app_0, false);
+    res = a.assignToUser(clusterResource, user_0, limit, app_0, false, null);
     assertFalse(res);
 
     refreshQueuesTurnOffReservationsContLook(a, csConf);
 
     // should now return false since feature off
-    res = a.assignToUser(clusterResource, user_0, limit, app_0, true);
+    res = a.assignToUser(clusterResource, user_0, limit, app_0, true, null);
     assertFalse(res);
   }
 

+ 29 - 2
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

@@ -18,11 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
+import java.util.Set;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -43,16 +46,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 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.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 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.AMRMTokenSecretManager;
 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.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class TestUtils {
   private static final Log LOG = LogFactory.getLog(TestUtils.class);
@@ -61,7 +67,7 @@ public class TestUtils {
    * Get a mock {@link RMContext} for use in test cases.
    * @return a mock {@link RMContext} for use in test cases
    */
-  @SuppressWarnings("rawtypes") 
+  @SuppressWarnings({ "rawtypes", "unchecked" }) 
   public static RMContext getMockRMContext() {
     // Null dispatcher
     Dispatcher nullDispatcher = new Dispatcher() {
@@ -93,6 +99,27 @@ public class TestUtils {
           new RMContainerTokenSecretManager(conf),
           new NMTokenSecretManagerInRM(conf),
           new ClientToAMTokenSecretManagerInRM(), writer);
+    RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
+    when(
+        nlm.getQueueResource(any(String.class), any(Set.class),
+            any(Resource.class))).thenAnswer(new Answer<Resource>() {
+      @Override
+      public Resource answer(InvocationOnMock invocation) throws Throwable {
+        Object[] args = invocation.getArguments();
+        return (Resource) args[2];
+      }
+    });
+    
+    when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
+        .thenAnswer(new Answer<Resource>() {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
+            Object[] args = invocation.getArguments();
+            return (Resource) args[1];
+          }
+        });
+    
+    rmContext.setNodeLabelManager(nlm);
     rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class));
     return rmContext;
   }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java

@@ -216,7 +216,7 @@ public class FairSchedulerTestBase {
     RMApp rmApp = new RMAppImpl(attId.getApplicationId(), rmContext, conf,
         null, null, null, ApplicationSubmissionContext.newInstance(null, null,
         null, null, null, false, false, 0, amResource, null), null, null,
-        0, null, null);
+        0, null, null, null);
     rmContext.getRMApps().put(attId.getApplicationId(), rmApp);
     AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent(
         attId.getApplicationId(), queue, user);

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -2420,7 +2420,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     RMApp application =
         new RMAppImpl(applicationId, resourceManager.getRMContext(), conf, name, user, 
           queue, submissionContext, scheduler, masterService,
-          System.currentTimeMillis(), "YARN", null);
+          System.currentTimeMillis(), "YARN", null, null);
     resourceManager.getRMContext().getRMApps().putIfAbsent(applicationId, application);
     application.handle(new RMAppEvent(applicationId, RMAppEventType.START));
 

+ 24 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.MockAsm;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.DummyRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -46,8 +47,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
-import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.StringHelper;
 import org.apache.hadoop.yarn.webapp.WebApps;
@@ -162,21 +163,24 @@ public class TestRMWebApp {
     for (RMNode node : deactivatedNodes) {
       deactivatedNodesMap.put(node.getHostName(), node);
     }
-   return new RMContextImpl(null, null, null, null,
-       null, null, null, null, null, null) {
-      @Override
-      public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
-        return applicationsMaps;
-      }
-      @Override
-      public ConcurrentMap<String, RMNode> getInactiveRMNodes() {
-        return deactivatedNodesMap;
-      }
-      @Override
-      public ConcurrentMap<NodeId, RMNode> getRMNodes() {
-        return nodesMap;
-      }
-    };
+
+    RMContextImpl rmContext = new RMContextImpl(null, null, null, null,
+        null, null, null, null, null, null) {
+       @Override
+       public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
+         return applicationsMaps;
+       }
+       @Override
+       public ConcurrentMap<String, RMNode> getInactiveRMNodes() {
+         return deactivatedNodesMap;
+       }
+       @Override
+       public ConcurrentMap<NodeId, RMNode> getRMNodes() {
+         return nodesMap;
+       }
+     }; 
+    rmContext.setNodeLabelManager(new DummyRMNodeLabelsManager());
+    return rmContext;
   }
 
   public static ResourceManager mockRm(int apps, int racks, int nodes,
@@ -203,10 +207,12 @@ public class TestRMWebApp {
 
     CapacityScheduler cs = new CapacityScheduler();
     cs.setConf(new YarnConfiguration());
-    cs.setRMContext(new RMContextImpl(null, null, null, null, null,
+    RMContext rmContext = new RMContextImpl(null, null, null, null, null,
         null, new RMContainerTokenSecretManager(conf),
         new NMTokenSecretManagerInRM(conf),
-        new ClientToAMTokenSecretManagerInRM(), null));
+        new ClientToAMTokenSecretManagerInRM(), null);
+    rmContext.setNodeLabelManager(new DummyRMNodeLabelsManager());
+    cs.setRMContext(rmContext);
     cs.init(conf);
     return cs;
   }

Một số tệp đã không được hiển thị bởi vì quá nhiều tập tin thay đổi trong này khác