Quellcode durchsuchen

YARN-6599. Support anti-affinity constraint via AppPlacementAllocator. (Wangda Tan via asuresh)

Arun Suresh vor 7 Jahren
Ursprung
Commit
38af237969
55 geänderte Dateien mit 2496 neuen und 431 gelöschten Zeilen
  1. 9 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
  2. 8 7
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
  3. 7 5
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java
  4. 8 0
      hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
  5. 42 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
  6. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  7. 47 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/SchedulerInvalidResoureRequestException.java
  8. 6 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
  9. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
  10. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/SchedulerRequestKey.java
  11. 9 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
  12. 2 3
      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
  13. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
  14. 161 44
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
  15. 7 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ApplicationPlacementAllocatorFactory.java
  16. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.java
  17. 19 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
  18. 8 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
  19. 45 9
      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. 5 0
      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
  21. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
  22. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ContainerRequest.java
  23. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/PendingAsk.java
  24. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
  25. 24 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsManager.java
  26. 0 31
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsNamespaces.java
  27. 118 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
  28. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/algorithm/DefaultPlacementAlgorithm.java
  29. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/processor/PlacementProcessor.java
  30. 8 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  31. 4 3
      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
  32. 46 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
  33. 18 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
  34. 531 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
  35. 4 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
  36. 51 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
  37. 5 5
      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
  38. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
  39. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
  40. 79 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestBase.java
  41. 16 74
      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
  42. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
  43. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoQueueCreation.java
  44. 260 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSchedulingRequestUpdate.java
  45. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestIncreaseAllocationExpirer.java
  46. 277 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestSchedulingRequestContainerAllocation.java
  47. 139 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestSchedulingRequestContainerAllocationAsync.java
  48. 2 0
      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
  49. 15 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsManager.java
  50. 18 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintsUtil.java
  51. 3 3
      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
  52. 4 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java
  53. 14 16
      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
  54. 14 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
  55. 403 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java

+ 9 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java

@@ -111,6 +111,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -1751,6 +1752,7 @@ public class TestRMContainerAllocator {
       super();
       try {
         Configuration conf = new Configuration();
+        init(conf);
         reinitialize(conf, rmContext);
       } catch (IOException ie) {
         LOG.info("add application failed with ", ie);
@@ -1769,8 +1771,8 @@ public class TestRMContainerAllocator {
     @Override
     public synchronized Allocation allocate(
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
-        List<ContainerId> release, List<String> blacklistAdditions,
-        List<String> blacklistRemovals,
+        List<SchedulingRequest> schedulingRequests, List<ContainerId> release,
+        List<String> blacklistAdditions, List<String> blacklistRemovals,
         ContainerUpdates updateRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
@@ -1785,7 +1787,7 @@ public class TestRMContainerAllocator {
       lastBlacklistAdditions = blacklistAdditions;
       lastBlacklistRemovals = blacklistRemovals;
       Allocation allocation = super.allocate(
-          applicationAttemptId, askCopy, release, blacklistAdditions,
+          applicationAttemptId, askCopy, schedulingRequests, release, blacklistAdditions,
           blacklistRemovals, updateRequests);
       if (forceResourceLimit != null) {
         // Test wants to force the non-default resource limit
@@ -1805,6 +1807,7 @@ public class TestRMContainerAllocator {
       super();
       try {
         Configuration conf = new Configuration();
+        init(conf);
         reinitialize(conf, rmContext);
       } catch (IOException ie) {
         LOG.info("add application failed with ", ie);
@@ -1815,8 +1818,8 @@ public class TestRMContainerAllocator {
     @Override
     public synchronized Allocation allocate(
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
-        List<ContainerId> release, List<String> blacklistAdditions,
-        List<String> blacklistRemovals,
+        List<SchedulingRequest> schedulingRequests, List<ContainerId> release,
+        List<String> blacklistAdditions, List<String> blacklistRemovals,
         ContainerUpdates updateRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
@@ -1827,7 +1830,7 @@ public class TestRMContainerAllocator {
       }
       SecurityUtil.setTokenServiceUseIp(false);
       Allocation normalAlloc = super.allocate(
-          applicationAttemptId, askCopy, release,
+          applicationAttemptId, askCopy, schedulingRequests, release,
           blacklistAdditions, blacklistRemovals, updateRequests);
       List<Container> containers = normalAlloc.getContainers();
       if(containers.size() > 0) {

+ 8 - 7
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
@@ -42,9 +43,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 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.scheduler.event.NodeUpdateSchedulerEvent;
@@ -100,16 +99,17 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
 
   @Override
   public Allocation allocate(ApplicationAttemptId attemptId,
-      List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
-      List<String> strings, List<String> strings2,
-      ContainerUpdates updateRequests) {
+      List<ResourceRequest> resourceRequests,
+      List<SchedulingRequest> schedulingRequests, List<ContainerId> containerIds,
+      List<String> strings, List<String> strings2, ContainerUpdates updateRequests) {
     if (metricsON) {
       final Timer.Context context = schedulerMetrics.getSchedulerAllocateTimer()
           .time();
       Allocation allocation = null;
       try {
         allocation = super
-            .allocate(attemptId, resourceRequests, containerIds, strings,
+            .allocate(attemptId, resourceRequests, schedulingRequests,
+                containerIds, strings,
                 strings2, updateRequests);
         return allocation;
       } finally {
@@ -123,7 +123,8 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
         }
       }
     } else {
-      return super.allocate(attemptId, resourceRequests, containerIds, strings,
+      return super.allocate(attemptId, resourceRequests, schedulingRequests,
+          containerIds, strings,
           strings2, updateRequests);
     }
   }

+ 7 - 5
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
@@ -39,8 +40,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptR
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.sls.SLSRunner;
 import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
@@ -94,7 +93,8 @@ public class SLSFairScheduler extends FairScheduler
 
   @Override
   public Allocation allocate(ApplicationAttemptId attemptId,
-      List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
+      List<ResourceRequest> resourceRequests,
+      List<SchedulingRequest> schedulingRequests, List<ContainerId> containerIds,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
       ContainerUpdates updateRequests) {
     if (metricsON) {
@@ -102,7 +102,8 @@ public class SLSFairScheduler extends FairScheduler
           .time();
       Allocation allocation = null;
       try {
-        allocation = super.allocate(attemptId, resourceRequests, containerIds,
+        allocation = super.allocate(attemptId, resourceRequests,
+            schedulingRequests, containerIds,
             blacklistAdditions, blacklistRemovals, updateRequests);
         return allocation;
       } finally {
@@ -116,7 +117,8 @@ public class SLSFairScheduler extends FairScheduler
         }
       }
     } else {
-      return super.allocate(attemptId, resourceRequests, containerIds,
+      return super.allocate(attemptId, resourceRequests, schedulingRequests,
+          containerIds,
           blacklistAdditions, blacklistRemovals, updateRequests);
     }
   }

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

@@ -650,4 +650,12 @@
     <Method name="equals" />
     <Bug pattern="EQ_OVERRIDING_EQUALS_NOT_SYMMETRIC" />
   </Match>
+
+  <!-- Null pointer exception needs to be ignored here as Findbugs doesn't properly detect code logic -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SingleConstraintAppPlacementAllocator" />
+    <Method name="validateAndSetSchedulingRequest" />
+    <Bug pattern="NP_NULL_ON_SOME_PATH" />
+  </Match>
+
 </FindBugsFilter>

+ 42 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java

@@ -20,8 +20,12 @@ package org.apache.hadoop.yarn.api.resource;
 
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
@@ -47,6 +51,14 @@ public final class PlacementConstraints {
 
   public static final String NODE = PlacementConstraint.NODE_SCOPE;
   public static final String RACK = PlacementConstraint.RACK_SCOPE;
+  public static final String NODE_PARTITION = "yarn_node_partition/";
+
+  private static final String APPLICATION_LABEL_PREFIX =
+      "yarn_application_label/";
+
+  @InterfaceAudience.Private
+  public static final String APPLICATION_LABEL_INTRA_APPLICATION =
+      APPLICATION_LABEL_PREFIX + "%intra_app%";
 
   /**
    * Creates a constraint that requires allocations to be placed on nodes that
@@ -186,6 +198,20 @@ public final class PlacementConstraints {
           attributeValues);
     }
 
+    /**
+     * Constructs a target expression on a node partition. It is satisfied if
+     * the specified node partition has one of the specified nodePartitions
+     *
+     * @param nodePartitions the set of values that the attribute should take
+     *          values from
+     * @return the resulting expression on the node attribute
+     */
+    public static TargetExpression nodePartition(
+        String... nodePartitions) {
+      return new TargetExpression(TargetType.NODE_ATTRIBUTE, NODE_PARTITION,
+          nodePartitions);
+    }
+
     /**
      * Constructs a target expression on an allocation tag. It is satisfied if
      * the there are allocations with one of the given tags.
@@ -198,6 +224,22 @@ public final class PlacementConstraints {
       return new TargetExpression(TargetType.ALLOCATION_TAG, null,
           allocationTags);
     }
+
+    /**
+     * Constructs a target expression on an allocation tag. It is satisfied if
+     * the there are allocations with one of the given tags. Comparing to
+     * {@link PlacementTargets#allocationTag(String...)}, this only check tags
+     * within the application.
+     *
+     * @param allocationTags the set of tags that the attribute should take
+     *          values from
+     * @return the resulting expression on the allocation tags
+     */
+    public static TargetExpression allocationTagToIntraApp(
+        String... allocationTags) {
+      return new TargetExpression(TargetType.ALLOCATION_TAG,
+          APPLICATION_LABEL_INTRA_APPLICATION, allocationTags);
+    }
   }
 
   // Creation of compound constraints.
@@ -277,5 +319,4 @@ public final class PlacementConstraints {
   public static PlacementConstraint build(AbstractConstraint constraintExpr) {
     return constraintExpr.build();
   }
-
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -543,7 +543,7 @@ public class YarnConfiguration extends Configuration {
   public static final String RM_PLACEMENT_CONSTRAINTS_ENABLED =
       RM_PREFIX + "placement-constraints.enabled";
 
-  public static final boolean DEFAULT_RM_PLACEMENT_CONSTRAINTS_ENABLED = true;
+  public static final boolean DEFAULT_RM_PLACEMENT_CONSTRAINTS_ENABLED = false;
 
   public static final String RM_PLACEMENT_CONSTRAINTS_RETRY_ATTEMPTS =
       RM_PREFIX + "placement-constraints.retry-attempts";

+ 47 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/SchedulerInvalidResoureRequestException.java

@@ -0,0 +1,47 @@
+/**
+ * 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.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This exception is thrown when any issue inside scheduler to handle a new or
+ * updated {@link org.apache.hadoop.yarn.api.records.SchedulingRequest}/
+ * {@link org.apache.hadoop.yarn.api.records.ResourceRequest} add to the
+ * scheduler.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SchedulerInvalidResoureRequestException extends YarnRuntimeException {
+  private static final long serialVersionUID = 10081123982L;
+
+  public SchedulerInvalidResoureRequestException(String message) {
+    super(message);
+  }
+
+  public SchedulerInvalidResoureRequestException(Throwable cause) {
+    super(cause);
+  }
+
+  public SchedulerInvalidResoureRequestException(String message,
+      Throwable cause) {
+    super(message, cause);
+  }
+}

+ 6 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
@@ -545,6 +546,7 @@ public class TestAMRMClientOnRMRestart {
       super();
       try {
         Configuration conf = new Configuration();
+        init(conf);
         reinitialize(conf, rmContext);
       } catch (IOException ie) {
         assert (false);
@@ -563,8 +565,8 @@ public class TestAMRMClientOnRMRestart {
     @Override
     public synchronized Allocation allocate(
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
-        List<ContainerId> release, List<String> blacklistAdditions,
-        List<String> blacklistRemovals,
+        List<SchedulingRequest> schedulingRequests, List<ContainerId> release,
+        List<String> blacklistAdditions, List<String> blacklistRemovals,
         ContainerUpdates updateRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
@@ -580,7 +582,8 @@ public class TestAMRMClientOnRMRestart {
       lastDecrease = updateRequests.getDecreaseRequests();
       lastBlacklistAdditions = blacklistAdditions;
       lastBlacklistRemovals = blacklistRemovals;
-      return super.allocate(applicationAttemptId, askCopy, release,
+      return super.allocate(applicationAttemptId, askCopy, schedulingRequests,
+          release,
           blacklistAdditions, blacklistRemovals, updateRequests);
     }
   }

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java

@@ -194,6 +194,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   public void setSchedulingRequests(
       List<SchedulingRequest> schedulingRequests) {
     if (schedulingRequests == null) {
+      builder.clearSchedulingRequests();
       return;
     }
     initSchedulingRequests();

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/SchedulerRequestKey.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 
 /**
@@ -45,6 +46,16 @@ public final class SchedulerRequestKey implements
         req.getAllocationRequestId(), null);
   }
 
+  /**
+   * Factory method to generate a SchedulerRequestKey from a SchedulingRequest.
+   * @param req SchedulingRequest
+   * @return SchedulerRequestKey
+   */
+  public static SchedulerRequestKey create(SchedulingRequest req) {
+    return new SchedulerRequestKey(req.getPriority(),
+        req.getAllocationRequestId(), null);
+  }
+
   public static SchedulerRequestKey create(UpdateContainerRequest req,
       SchedulerRequestKey schedulerRequestKey) {
     return new SchedulerRequestKey(schedulerRequestKey.getPriority(),

+ 9 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
+import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -273,10 +274,14 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
           " state, ignore container allocate request.");
       allocation = EMPTY_ALLOCATION;
     } else {
-      allocation =
-          getScheduler().allocate(appAttemptId, ask, release,
-              blacklistAdditions, blacklistRemovals,
-              containerUpdateRequests);
+      try {
+        allocation = getScheduler().allocate(appAttemptId, ask,
+            request.getSchedulingRequests(), release,
+            blacklistAdditions, blacklistRemovals, containerUpdateRequests);
+      } catch (SchedulerInvalidResoureRequestException e) {
+        LOG.warn("Exceptions caught when scheduler handling requests");
+        throw new YarnException(e);
+      }
     }
 
     if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {

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

@@ -1113,8 +1113,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         Allocation amContainerAllocation =
             appAttempt.scheduler.allocate(
                 appAttempt.applicationAttemptId,
-                appAttempt.amReqs,
-                EMPTY_CONTAINER_RELEASE_LIST,
+                appAttempt.amReqs, null, EMPTY_CONTAINER_RELEASE_LIST,
                 amBlacklist.getBlacklistAdditions(),
                 amBlacklist.getBlacklistRemovals(),
                 new ContainerUpdates());
@@ -1140,7 +1139,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       // Acquire the AM container from the scheduler.
       Allocation amContainerAllocation =
           appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
-            EMPTY_CONTAINER_REQUEST_LIST, EMPTY_CONTAINER_RELEASE_LIST, null,
+            EMPTY_CONTAINER_REQUEST_LIST, null, EMPTY_CONTAINER_RELEASE_LIST, null,
             null, new ContainerUpdates());
       // There must be at least one container allocated, because a
       // CONTAINER_ALLOCATED is emitted after an RMContainer is constructed,

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

@@ -53,6 +53,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 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.ResourceSizing;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerError;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
@@ -1155,7 +1156,7 @@ public abstract class AbstractYarnScheduler
    *
    * @param asks resource requests
    */
-  protected void normalizeRequests(List<ResourceRequest> asks) {
+  protected void normalizeResourceRequests(List<ResourceRequest> asks) {
     for (ResourceRequest ask: asks) {
       ask.setCapability(getNormalizedResource(ask.getCapability()));
     }

+ 161 - 44
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java

@@ -41,6 +41,8 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
@@ -49,7 +51,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.Applicatio
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.LocalityAppPlacementAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PendingAskUpdateResult;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SingleConstraintAppPlacementAllocator;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.util.resource.Resources;
 /**
@@ -91,11 +95,12 @@ public class AppSchedulingInfo {
 
   public final ContainerUpdateContext updateContext;
   public final Map<String, String> applicationSchedulingEnvs = new HashMap<>();
+  private final RMContext rmContext;
 
   public AppSchedulingInfo(ApplicationAttemptId appAttemptId, String user,
       Queue queue, AbstractUsersManager abstractUsersManager, long epoch,
       ResourceUsage appResourceUsage,
-      Map<String, String> applicationSchedulingEnvs) {
+      Map<String, String> applicationSchedulingEnvs, RMContext rmContext) {
     this.applicationAttemptId = appAttemptId;
     this.applicationId = appAttemptId.getApplicationId();
     this.queue = queue;
@@ -105,6 +110,7 @@ public class AppSchedulingInfo {
         epoch << ResourceManager.EPOCH_BIT_SHIFT);
     this.appResourceUsage = appResourceUsage;
     this.applicationSchedulingEnvs.putAll(applicationSchedulingEnvs);
+    this.rmContext = rmContext;
 
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     updateContext = new ContainerUpdateContext(this);
@@ -163,74 +169,153 @@ public class AppSchedulingInfo {
    * application, by asking for more resources and releasing resources acquired
    * by the application.
    *
-   * @param requests
-   *          resources to be acquired
+   * @param resourceRequests resource requests to be allocated
    * @param recoverPreemptedRequestForAContainer
-   *          recover ResourceRequest on preemption
+   *          recover ResourceRequest/SchedulingRequest on preemption
    * @return true if any resource was updated, false otherwise
    */
-  public boolean updateResourceRequests(List<ResourceRequest> requests,
+  public boolean updateResourceRequests(List<ResourceRequest> resourceRequests,
       boolean recoverPreemptedRequestForAContainer) {
-    if (null == requests || requests.isEmpty()) {
-      return false;
+    // Flag to track if any incoming requests update "ANY" requests
+    boolean offswitchResourcesUpdated;
+
+    writeLock.lock();
+    try {
+      // Update AppPlacementAllocator by requests
+      offswitchResourcesUpdated = internalAddResourceRequests(
+          recoverPreemptedRequestForAContainer, resourceRequests);
+    } finally {
+      writeLock.unlock();
     }
 
+    return offswitchResourcesUpdated;
+  }
+
+  /**
+   * The ApplicationMaster is updating resource requirements for the
+   * application, by asking for more resources and releasing resources acquired
+   * by the application.
+   *
+   * @param dedupRequests (dedup) resource requests to be allocated
+   * @param recoverPreemptedRequestForAContainer
+   *          recover ResourceRequest/SchedulingRequest on preemption
+   * @return true if any resource was updated, false otherwise
+   */
+  public boolean updateResourceRequests(
+      Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests,
+      boolean recoverPreemptedRequestForAContainer) {
     // Flag to track if any incoming requests update "ANY" requests
-    boolean offswitchResourcesUpdated = false;
+    boolean offswitchResourcesUpdated;
 
+    writeLock.lock();
     try {
-      this.writeLock.lock();
-
-      // A map to group resource requests and dedup
-      Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests =
-          new HashMap<>();
+      // Update AppPlacementAllocator by requests
+      offswitchResourcesUpdated = internalAddResourceRequests(
+          recoverPreemptedRequestForAContainer, dedupRequests);
+    } finally {
+      writeLock.unlock();
+    }
 
-      // Group resource request by schedulerRequestKey and resourceName
-      for (ResourceRequest request : requests) {
-        SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request);
-        if (!dedupRequests.containsKey(schedulerKey)) {
-          dedupRequests.put(schedulerKey, new HashMap<>());
-        }
-        dedupRequests.get(schedulerKey).put(request.getResourceName(), request);
-      }
+    return offswitchResourcesUpdated;
+  }
 
-      // Update AppPlacementAllocator by dedup requests.
-      offswitchResourcesUpdated =
-          addRequestToAppPlacement(
-              recoverPreemptedRequestForAContainer, dedupRequests);
+  /**
+   * The ApplicationMaster is updating resource requirements for the
+   * application, by asking for more resources and releasing resources acquired
+   * by the application.
+   *
+   * @param schedulingRequests resource requests to be allocated
+   * @param recoverPreemptedRequestForAContainer
+   *          recover ResourceRequest/SchedulingRequest on preemption
+   * @return true if any resource was updated, false otherwise
+   */
+  public boolean updateSchedulingRequests(
+      List<SchedulingRequest> schedulingRequests,
+      boolean recoverPreemptedRequestForAContainer) {
+    // Flag to track if any incoming requests update "ANY" requests
+    boolean offswitchResourcesUpdated;
 
-      return offswitchResourcesUpdated;
+    writeLock.lock();
+    try {
+      // Update AppPlacementAllocator by requests
+      offswitchResourcesUpdated = addSchedulingRequests(
+          recoverPreemptedRequestForAContainer, schedulingRequests);
     } finally {
-      this.writeLock.unlock();
+      writeLock.unlock();
     }
+
+    return offswitchResourcesUpdated;
   }
 
   public void removeAppPlacement(SchedulerRequestKey schedulerRequestKey) {
     schedulerKeyToAppPlacementAllocator.remove(schedulerRequestKey);
   }
 
-  boolean addRequestToAppPlacement(
+  private boolean addSchedulingRequests(
+      boolean recoverPreemptedRequestForAContainer,
+      List<SchedulingRequest> schedulingRequests) {
+    // Do we need to update pending resource for app/queue, etc.?
+    boolean requireUpdatePendingResource = false;
+
+    for (SchedulingRequest request : schedulingRequests) {
+      SchedulerRequestKey schedulerRequestKey = SchedulerRequestKey.create(
+          request);
+
+      AppPlacementAllocator appPlacementAllocator =
+          getAndAddAppPlacementAllocatorIfNotExist(schedulerRequestKey,
+              SingleConstraintAppPlacementAllocator.class.getCanonicalName());
+
+      // Update AppPlacementAllocator
+      PendingAskUpdateResult pendingAmountChanges =
+          appPlacementAllocator.updatePendingAsk(schedulerRequestKey,
+              request, recoverPreemptedRequestForAContainer);
+
+      if (null != pendingAmountChanges) {
+        updatePendingResources(pendingAmountChanges, schedulerRequestKey,
+            queue.getMetrics());
+        requireUpdatePendingResource = true;
+      }
+    }
+
+    return requireUpdatePendingResource;
+  }
+
+  /**
+   * Get and insert AppPlacementAllocator if it doesn't exist, this should be
+   * protected by write lock.
+   * @param schedulerRequestKey schedulerRequestKey
+   * @param placementTypeClass placementTypeClass
+   * @return AppPlacementAllocator
+   */
+  private AppPlacementAllocator<SchedulerNode> getAndAddAppPlacementAllocatorIfNotExist(
+      SchedulerRequestKey schedulerRequestKey, String placementTypeClass) {
+    AppPlacementAllocator<SchedulerNode> appPlacementAllocator;
+    if ((appPlacementAllocator = schedulerKeyToAppPlacementAllocator.get(
+        schedulerRequestKey)) == null) {
+      appPlacementAllocator =
+          ApplicationPlacementAllocatorFactory.getAppPlacementAllocator(
+              placementTypeClass, this, schedulerRequestKey, rmContext);
+      schedulerKeyToAppPlacementAllocator.put(schedulerRequestKey,
+          appPlacementAllocator);
+    }
+    return appPlacementAllocator;
+  }
+
+  private boolean internalAddResourceRequests(
       boolean recoverPreemptedRequestForAContainer,
       Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests) {
     boolean offswitchResourcesUpdated = false;
     for (Map.Entry<SchedulerRequestKey, Map<String, ResourceRequest>> entry :
     dedupRequests.entrySet()) {
       SchedulerRequestKey schedulerRequestKey = entry.getKey();
-
-      if (!schedulerKeyToAppPlacementAllocator
-          .containsKey(schedulerRequestKey)) {
-        AppPlacementAllocator<SchedulerNode> placementAllocatorInstance = ApplicationPlacementFactory
-            .getAppPlacementAllocator(applicationSchedulingEnvs
-                .get(ApplicationSchedulingConfig.ENV_APPLICATION_PLACEMENT_TYPE_CLASS));
-        placementAllocatorInstance.setAppSchedulingInfo(this);
-
-        schedulerKeyToAppPlacementAllocator.put(schedulerRequestKey,
-            placementAllocatorInstance);
-      }
+      AppPlacementAllocator<SchedulerNode> appPlacementAllocator =
+          getAndAddAppPlacementAllocatorIfNotExist(schedulerRequestKey,
+              applicationSchedulingEnvs.get(
+                  ApplicationSchedulingConfig.ENV_APPLICATION_PLACEMENT_TYPE_CLASS));
 
       // Update AppPlacementAllocator
-      PendingAskUpdateResult pendingAmountChanges = schedulerKeyToAppPlacementAllocator
-          .get(schedulerRequestKey).updatePendingAsk(entry.getValue().values(),
+      PendingAskUpdateResult pendingAmountChanges =
+          appPlacementAllocator.updatePendingAsk(entry.getValue().values(),
               recoverPreemptedRequestForAContainer);
 
       if (null != pendingAmountChanges) {
@@ -242,6 +327,29 @@ public class AppSchedulingInfo {
     return offswitchResourcesUpdated;
   }
 
+  private boolean internalAddResourceRequests(boolean recoverPreemptedRequestForAContainer,
+      List<ResourceRequest> resourceRequests) {
+    if (null == resourceRequests || resourceRequests.isEmpty()) {
+      return false;
+    }
+
+    // A map to group resource requests and dedup
+    Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests =
+        new HashMap<>();
+
+    // Group resource request by schedulerRequestKey and resourceName
+    for (ResourceRequest request : resourceRequests) {
+      SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request);
+      if (!dedupRequests.containsKey(schedulerKey)) {
+        dedupRequests.put(schedulerKey, new HashMap<>());
+      }
+      dedupRequests.get(schedulerKey).put(request.getResourceName(), request);
+    }
+
+    return internalAddResourceRequests(recoverPreemptedRequestForAContainer,
+        dedupRequests);
+  }
+
   private void updatePendingResources(PendingAskUpdateResult updateResult,
       SchedulerRequestKey schedulerKey, QueueMetrics metrics) {
 
@@ -629,13 +737,22 @@ public class AppSchedulingInfo {
     }
   }
 
-  public boolean acceptNodePartition(SchedulerRequestKey schedulerKey,
-      String nodePartition, SchedulingMode schedulingMode) {
+  /**
+   * Pre-check node to see if it satisfy the given schedulerKey and
+   * scheduler mode
+   *
+   * @param schedulerKey schedulerKey
+   * @param schedulerNode schedulerNode
+   * @param schedulingMode schedulingMode
+   * @return can use the node or not.
+   */
+  public boolean precheckNode(SchedulerRequestKey schedulerKey,
+      SchedulerNode schedulerNode, SchedulingMode schedulingMode) {
     try {
       this.readLock.lock();
       AppPlacementAllocator ap =
           schedulerKeyToAppPlacementAllocator.get(schedulerKey);
-      return (ap != null) && ap.acceptNodePartition(nodePartition,
+      return (ap != null) && ap.precheckNode(schedulerNode,
           schedulingMode);
     } finally {
       this.readLock.unlock();

+ 7 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ApplicationPlacementFactory.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ApplicationPlacementAllocatorFactory.java

@@ -21,15 +21,17 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ApplicationSchedulingConfig;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
 /**
  * Factory class to build various application placement policies.
  */
 @Public
 @Unstable
-public class ApplicationPlacementFactory {
+public class ApplicationPlacementAllocatorFactory {
 
   /**
    * Get AppPlacementAllocator related to the placement type requested.
@@ -39,7 +41,8 @@ public class ApplicationPlacementFactory {
    * @return Specific AppPlacementAllocator instance based on type
    */
   public static AppPlacementAllocator<SchedulerNode> getAppPlacementAllocator(
-      String appPlacementAllocatorName) {
+      String appPlacementAllocatorName, AppSchedulingInfo appSchedulingInfo,
+      SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {
     Class<?> policyClass;
     try {
       if (appPlacementAllocatorName == null) {
@@ -58,6 +61,8 @@ public class ApplicationPlacementFactory {
     @SuppressWarnings("unchecked")
     AppPlacementAllocator<SchedulerNode> placementAllocatorInstance = (AppPlacementAllocator<SchedulerNode>) ReflectionUtils
         .newInstance(policyClass, null);
+    placementAllocatorInstance.initialize(appSchedulingInfo,
+        schedulerRequestKey, rmContext);
     return placementAllocatorInstance;
   }
 }

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

@@ -146,7 +146,7 @@ public class ContainerUpdateContext {
           createResourceRequests(rmContainer, schedulerNode,
               schedulerKey, resToIncrease);
       updateResReqs.put(schedulerKey, resMap);
-      appSchedulingInfo.addRequestToAppPlacement(false, updateResReqs);
+      appSchedulingInfo.updateResourceRequests(updateResReqs, false);
     }
     return true;
   }
@@ -290,7 +290,7 @@ public class ContainerUpdateContext {
           (rmContainer, node, schedulerKey,
           rmContainer.getContainer().getResource());
       reqsToUpdate.put(schedulerKey, resMap);
-      appSchedulingInfo.addRequestToAppPlacement(true, reqsToUpdate);
+      appSchedulingInfo.updateResourceRequests(reqsToUpdate, true);
       return UNDEFINED;
     }
     return retVal;

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

@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerError;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.api.ContainerType;
@@ -231,7 +232,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
 
     this.appSchedulingInfo = new AppSchedulingInfo(applicationAttemptId, user,
         queue, abstractUsersManager, rmContext.getEpoch(), attemptResourceUsage,
-        applicationSchedulingEnvs);
+        applicationSchedulingEnvs, rmContext);
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     readLock = lock.readLock();
     writeLock = lock.writeLock();
@@ -451,6 +452,23 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       writeLock.unlock();
     }
   }
+
+  public boolean updateSchedulingRequests(
+      List<SchedulingRequest> requests) {
+    if (requests == null) {
+      return false;
+    }
+
+    try {
+      writeLock.lock();
+      if (!isStopped) {
+        return appSchedulingInfo.updateSchedulingRequests(requests, false);
+      }
+      return false;
+    } finally {
+      writeLock.unlock();
+    }
+  }
   
   public void recoverResourceRequestsForContainer(
       ContainerRequest containerRequest) {

+ 8 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -132,18 +133,18 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
    * 
    * @param appAttemptId
    * @param ask
+   * @param schedulingRequests
    * @param release
-   * @param blacklistAdditions 
-   * @param blacklistRemovals 
-   * @param updateRequests
-   * @return the {@link Allocation} for the application
+   * @param blacklistAdditions
+   * @param blacklistRemovals
+   * @param updateRequests     @return the {@link Allocation} for the application
    */
   @Public
   @Stable
   Allocation allocate(ApplicationAttemptId appAttemptId,
-      List<ResourceRequest> ask, List<ContainerId> release,
-      List<String> blacklistAdditions, List<String> blacklistRemovals,
-      ContainerUpdates updateRequests);
+      List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
+      List<ContainerId> release, List<String> blacklistAdditions,
+      List<String> blacklistRemovals, ContainerUpdates updateRequests);
 
   /**
    * Get node resource usage report.

+ 45 - 9
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

@@ -60,8 +60,11 @@ import org.apache.hadoop.yarn.api.records.ReservationId;
 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.ResourceSizing;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
+import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
@@ -1058,12 +1061,29 @@ public class CapacityScheduler extends
     }
   }
 
+  /**
+   * Normalize a list of SchedulingRequest
+   *
+   * @param asks scheduling request
+   */
+  private void normalizeSchedulingRequests(List<SchedulingRequest> asks) {
+    if (asks == null) {
+      return;
+    }
+    for (SchedulingRequest ask: asks) {
+      ResourceSizing sizing = ask.getResourceSizing();
+      if (sizing != null && sizing.getResources() != null) {
+        sizing.setResources(getNormalizedResource(sizing.getResources()));
+      }
+    }
+  }
+
   @Override
   @Lock(Lock.NoLock.class)
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
-      List<ResourceRequest> ask, List<ContainerId> release,
-      List<String> blacklistAdditions, List<String> blacklistRemovals,
-      ContainerUpdates updateRequests) {
+      List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
+      List<ContainerId> release, List<String> blacklistAdditions,
+      List<String> blacklistRemovals, ContainerUpdates updateRequests) {
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
       LOG.error("Calling allocate on removed or non existent application " +
@@ -1071,6 +1091,18 @@ public class CapacityScheduler extends
       return EMPTY_ALLOCATION;
     }
 
+    if ((!getConfiguration().getBoolean(
+        CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
+        CapacitySchedulerConfiguration.DEFAULT_SCHEDULING_REQUEST_ALLOWED))
+        && schedulingRequests != null && (!schedulingRequests.isEmpty())) {
+      throw new SchedulerInvalidResoureRequestException(
+          "Application attempt:" + applicationAttemptId
+              + " is using SchedulingRequest, which is disabled. Please update "
+              + CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED
+              + " to true in capacity-scheduler.xml in order to use this "
+              + "feature.");
+    }
+
     // The allocate may be the leftover from previous attempt, and it will
     // impact current attempt, such as confuse the request and allocation for
     // current attempt's AM container.
@@ -1091,7 +1123,10 @@ public class CapacityScheduler extends
     LeafQueue updateDemandForQueue = null;
 
     // Sanity check for new allocation requests
-    normalizeRequests(ask);
+    normalizeResourceRequests(ask);
+
+    // Normalize scheduling requests
+    normalizeSchedulingRequests(schedulingRequests);
 
     Allocation allocation;
 
@@ -1104,7 +1139,8 @@ public class CapacityScheduler extends
       }
 
       // Process resource requests
-      if (!ask.isEmpty()) {
+      if (!ask.isEmpty() || (schedulingRequests != null && !schedulingRequests
+          .isEmpty())) {
         if (LOG.isDebugEnabled()) {
           LOG.debug(
               "allocate: pre-update " + applicationAttemptId + " ask size ="
@@ -1113,7 +1149,8 @@ public class CapacityScheduler extends
         }
 
         // Update application requests
-        if (application.updateResourceRequests(ask)) {
+        if (application.updateResourceRequests(ask) || application
+            .updateSchedulingRequests(schedulingRequests)) {
           updateDemandForQueue = (LeafQueue) application.getQueue();
         }
 
@@ -2580,10 +2617,9 @@ public class CapacityScheduler extends
         // Validate placement constraint is satisfied before
         // committing the request.
         try {
-          if (!PlacementConstraintsUtil.canSatisfyConstraints(
+          if (!PlacementConstraintsUtil.canSatisfySingleConstraint(
               appAttempt.getApplicationId(),
-              schedulingRequest.getAllocationTags(),
-              schedulerNode,
+              schedulingRequest.getAllocationTags(), schedulerNode,
               rmContext.getPlacementConstraintManager(),
               rmContext.getAllocationTagsManager())) {
             LOG.debug("Failed to allocate container for application "

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

@@ -77,6 +77,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   
   @Private
   public static final String PREFIX = "yarn.scheduler.capacity.";
+
+  @Private
+  public static final String SCHEDULING_REQUEST_ALLOWED =
+      PREFIX + "scheduling-request.allowed";
+  public static final boolean DEFAULT_SCHEDULING_REQUEST_ALLOWED = false;
   
   @Private
   public static final String DOT = ".";

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java

@@ -143,8 +143,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
     // Is the nodePartition of pending request matches the node's partition
     // If not match, jump to next priority.
-    if (!appInfo.acceptNodePartition(schedulerKey, node.getPartition(),
-        schedulingMode)) {
+    if (!appInfo.precheckNode(schedulerKey, node, schedulingMode)) {
       ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
           activitiesManager, node, application, priority,
           ActivityDiagnosticConstant.

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

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
 
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 
 import java.util.List;
 
@@ -43,12 +44,23 @@ import java.util.List;
  */
 public class ContainerRequest {
   private List<ResourceRequest> requests;
+  private SchedulingRequest schedulingRequest;
 
   public ContainerRequest(List<ResourceRequest> requests) {
     this.requests = requests;
+    schedulingRequest = null;
+  }
+
+  public ContainerRequest(SchedulingRequest schedulingRequest) {
+    this.schedulingRequest = schedulingRequest;
+    this.requests = null;
   }
 
   public List<ResourceRequest> getResourceRequests() {
     return requests;
   }
+
+  public SchedulingRequest getSchedulingRequest() {
+    return schedulingRequest;
+  }
 }

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/PendingAsk.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
 
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
@@ -31,6 +32,11 @@ public class PendingAsk {
   private final int count;
   public final static PendingAsk ZERO = new PendingAsk(Resources.none(), 0);
 
+  public PendingAsk(ResourceSizing sizing) {
+    this.perAllocationResource = sizing.getResources();
+    this.count = sizing.getNumAllocations();
+  }
+
   public PendingAsk(Resource res, int num) {
     this.perAllocationResource = res;
     this.count = num;

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java

@@ -542,6 +542,12 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
                 schedulerContainer.getRmContainer().getContainer());
             ((RMContainerImpl) rmContainer).setContainerRequest(
                 containerRequest);
+
+            // If this is from a SchedulingRequest, set allocation tags.
+            if (containerRequest.getSchedulingRequest() != null) {
+              ((RMContainerImpl) rmContainer).setAllocationTags(
+                  containerRequest.getSchedulingRequest().getAllocationTags());
+            }
           }
 
           attemptResourceUsage.incUsed(schedulerContainer.getNodePartition(),

+ 24 - 47
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsManager.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.log4j.Logger;
 
@@ -287,21 +288,15 @@ public class AllocationTagsManager {
    *                       {@link SchedulingRequest#getAllocationTags()}
    *                       application_id will be added to allocationTags.
    */
+  @SuppressWarnings("unchecked")
   public void addContainer(NodeId nodeId, ContainerId containerId,
       Set<String> allocationTags) {
+    // Do nothing for empty allocation tags.
+    if (allocationTags == null || allocationTags.isEmpty()) {
+      return;
+    }
     ApplicationId applicationId =
         containerId.getApplicationAttemptId().getApplicationId();
-    String applicationIdTag =
-        AllocationTagsNamespaces.APP_ID + applicationId.toString();
-
-    boolean useSet = false;
-    if (allocationTags != null && !allocationTags.isEmpty()) {
-      // Copy before edit it.
-      allocationTags = new HashSet<>(allocationTags);
-      allocationTags.add(applicationIdTag);
-      useSet = true;
-    }
-
     writeLock.lock();
     try {
       TypeToCountedTags perAppTagsMapping = perAppNodeMappings
@@ -311,19 +306,12 @@ public class AllocationTagsManager {
       // Covering test-cases where context is mocked
       String nodeRack = (rmContext.getRMNodes() != null
           && rmContext.getRMNodes().get(nodeId) != null)
-              ? rmContext.getRMNodes().get(nodeId).getRackName()
-              : "default-rack";
-      if (useSet) {
-        perAppTagsMapping.addTags(nodeId, allocationTags);
-        perAppRackTagsMapping.addTags(nodeRack, allocationTags);
-        globalNodeMapping.addTags(nodeId, allocationTags);
-        globalRackMapping.addTags(nodeRack, allocationTags);
-      } else {
-        perAppTagsMapping.addTag(nodeId, applicationIdTag);
-        perAppRackTagsMapping.addTag(nodeRack, applicationIdTag);
-        globalNodeMapping.addTag(nodeId, applicationIdTag);
-        globalRackMapping.addTag(nodeRack, applicationIdTag);
-      }
+              ? rmContext.getRMNodes().get(nodeId).getRackName() :
+          "default-rack";
+      perAppTagsMapping.addTags(nodeId, allocationTags);
+      perAppRackTagsMapping.addTags(nodeRack, allocationTags);
+      globalNodeMapping.addTags(nodeId, allocationTags);
+      globalRackMapping.addTags(nodeRack, allocationTags);
 
       if (LOG.isDebugEnabled()) {
         LOG.debug("Added container=" + containerId + " with tags=["
@@ -341,20 +329,15 @@ public class AllocationTagsManager {
    * @param containerId    containerId.
    * @param allocationTags allocation tags for given container
    */
+  @SuppressWarnings("unchecked")
   public void removeContainer(NodeId nodeId,
       ContainerId containerId, Set<String> allocationTags) {
+    // Do nothing for empty allocation tags.
+    if (allocationTags == null || allocationTags.isEmpty()) {
+      return;
+    }
     ApplicationId applicationId =
         containerId.getApplicationAttemptId().getApplicationId();
-    String applicationIdTag =
-        AllocationTagsNamespaces.APP_ID + applicationId.toString();
-    boolean useSet = false;
-
-    if (allocationTags != null && !allocationTags.isEmpty()) {
-      // Copy before edit it.
-      allocationTags = new HashSet<>(allocationTags);
-      allocationTags.add(applicationIdTag);
-      useSet = true;
-    }
 
     writeLock.lock();
     try {
@@ -368,19 +351,12 @@ public class AllocationTagsManager {
       // Covering test-cases where context is mocked
       String nodeRack = (rmContext.getRMNodes() != null
           && rmContext.getRMNodes().get(nodeId) != null)
-              ? rmContext.getRMNodes().get(nodeId).getRackName()
-              : "default-rack";
-      if (useSet) {
-        perAppTagsMapping.removeTags(nodeId, allocationTags);
-        perAppRackTagsMapping.removeTags(nodeRack, allocationTags);
-        globalNodeMapping.removeTags(nodeId, allocationTags);
-        globalRackMapping.removeTags(nodeRack, allocationTags);
-      } else {
-        perAppTagsMapping.removeTag(nodeId, applicationIdTag);
-        perAppRackTagsMapping.removeTag(nodeRack, applicationIdTag);
-        globalNodeMapping.removeTag(nodeId, applicationIdTag);
-        globalRackMapping.removeTag(nodeRack, applicationIdTag);
-      }
+              ? rmContext.getRMNodes().get(nodeId).getRackName() :
+          "default-rack";
+      perAppTagsMapping.removeTags(nodeId, allocationTags);
+      perAppRackTagsMapping.removeTags(nodeRack, allocationTags);
+      globalNodeMapping.removeTags(nodeId, allocationTags);
+      globalRackMapping.removeTags(nodeRack, allocationTags);
 
       if (perAppTagsMapping.isEmpty()) {
         perAppNodeMappings.remove(applicationId);
@@ -602,6 +578,7 @@ public class AllocationTagsManager {
    * @throws InvalidAllocationTagsQueryException when illegal query
    *                                            parameter specified
    */
+  @SuppressWarnings("unchecked")
   public long getRackCardinalityByOp(String rack, ApplicationId applicationId,
       Set<String> tags, LongBinaryOperator op)
       throws InvalidAllocationTagsQueryException {

+ 0 - 31
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsNamespaces.java

@@ -1,31 +0,0 @@
-/*
- * *
- *  Licensed to the Apache Software Foundation (ASF) under one
- *  or more contributor license agreements.  See the NOTICE file
- *  distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- *  to you under the Apache License, Version 2.0 (the
- *  "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- * /
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint;
-
-/**
- * Predefined namespaces for tags
- *
- * Same as namespace  of resource types. Namespaces of placement tags are start
- * with alphabets and ended with "/"
- */
-public class AllocationTagsNamespaces {
-  public static final String APP_ID = "yarn_app_id/";
-}

+ 118 - 47
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint;
 import java.util.Iterator;
 import java.util.Set;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -30,9 +32,12 @@ import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraintTransformations.SingleConstraintTransformer;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm.DefaultPlacementAlgorithm;
 
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE_PARTITION;
+
 /**
  * This class contains various static methods used by the Placement Algorithms
  * to simplify constrained placement.
@@ -41,16 +46,20 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algori
 @Public
 @Unstable
 public final class PlacementConstraintsUtil {
+  private static final Log LOG =
+      LogFactory.getLog(PlacementConstraintsUtil.class);
 
   // Suppresses default constructor, ensuring non-instantiability.
   private PlacementConstraintsUtil() {
   }
 
   /**
-   * Returns true if **single** application constraint with associated
+   * Returns true if **single** placement constraint with associated
    * allocationTags and scope is satisfied by a specific scheduler Node.
    *
-   * @param appId the application id
+   * @param targetApplicationId the application id, which could be override by
+   *                           target application id specified inside allocation
+   *                           tags.
    * @param sc the placement constraint
    * @param te the target expression
    * @param node the scheduler node
@@ -59,32 +68,123 @@ public final class PlacementConstraintsUtil {
    * @throws InvalidAllocationTagsQueryException
    */
   private static boolean canSatisfySingleConstraintExpression(
-      ApplicationId appId, SingleConstraint sc, TargetExpression te,
-      SchedulerNode node, AllocationTagsManager tm)
+      ApplicationId targetApplicationId, SingleConstraint sc,
+      TargetExpression te, SchedulerNode node, AllocationTagsManager tm)
       throws InvalidAllocationTagsQueryException {
     long minScopeCardinality = 0;
     long maxScopeCardinality = 0;
+    
+    // Optimizations to only check cardinality if necessary.
+    int desiredMinCardinality = sc.getMinCardinality();
+    int desiredMaxCardinality = sc.getMaxCardinality();
+    boolean checkMinCardinality = desiredMinCardinality > 0;
+    boolean checkMaxCardinality = desiredMaxCardinality < Integer.MAX_VALUE;
+
     if (sc.getScope().equals(PlacementConstraints.NODE)) {
-      minScopeCardinality = tm.getNodeCardinalityByOp(node.getNodeID(), appId,
-          te.getTargetValues(), Long::max);
-      maxScopeCardinality = tm.getNodeCardinalityByOp(node.getNodeID(), appId,
-          te.getTargetValues(), Long::min);
+      if (checkMinCardinality) {
+        minScopeCardinality = tm.getNodeCardinalityByOp(node.getNodeID(),
+            targetApplicationId, te.getTargetValues(), Long::max);
+      }
+      if (checkMaxCardinality) {
+        maxScopeCardinality = tm.getNodeCardinalityByOp(node.getNodeID(),
+            targetApplicationId, te.getTargetValues(), Long::min);
+      }
     } else if (sc.getScope().equals(PlacementConstraints.RACK)) {
-      minScopeCardinality = tm.getRackCardinalityByOp(node.getRackName(), appId,
-          te.getTargetValues(), Long::max);
-      maxScopeCardinality = tm.getRackCardinalityByOp(node.getRackName(), appId,
-          te.getTargetValues(), Long::min);
+      if (checkMinCardinality) {
+        minScopeCardinality = tm.getRackCardinalityByOp(node.getRackName(),
+            targetApplicationId, te.getTargetValues(), Long::max);
+      }
+      if (checkMaxCardinality) {
+        maxScopeCardinality = tm.getRackCardinalityByOp(node.getRackName(),
+            targetApplicationId, te.getTargetValues(), Long::min);
+      }
     }
     // Make sure Anti-affinity satisfies hard upper limit
-    maxScopeCardinality = sc.getMaxCardinality() == 0 ? maxScopeCardinality - 1
+    maxScopeCardinality = desiredMaxCardinality == 0 ? maxScopeCardinality - 1
         : maxScopeCardinality;
 
-    return (minScopeCardinality >= sc.getMinCardinality()
-        && maxScopeCardinality < sc.getMaxCardinality());
+    return (desiredMinCardinality <= 0
+        || minScopeCardinality >= desiredMinCardinality) && (
+        desiredMaxCardinality == Integer.MAX_VALUE
+            || maxScopeCardinality < desiredMaxCardinality);
+  }
+
+  private static boolean canSatisfyNodePartitionConstraintExpresssion(
+      TargetExpression targetExpression, SchedulerNode schedulerNode) {
+    Set<String> values = targetExpression.getTargetValues();
+    if (values == null || values.isEmpty()) {
+      return schedulerNode.getPartition().equals(
+          RMNodeLabelsManager.NO_LABEL);
+    } else{
+      String nodePartition = values.iterator().next();
+      if (!nodePartition.equals(schedulerNode.getPartition())) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  private static boolean canSatisfySingleConstraint(ApplicationId applicationId,
+      SingleConstraint singleConstraint, SchedulerNode schedulerNode,
+      AllocationTagsManager tagsManager)
+      throws InvalidAllocationTagsQueryException {
+    // Iterate through TargetExpressions
+    Iterator<TargetExpression> expIt =
+        singleConstraint.getTargetExpressions().iterator();
+    while (expIt.hasNext()) {
+      TargetExpression currentExp = expIt.next();
+      // Supporting AllocationTag Expressions for now
+      if (currentExp.getTargetType().equals(TargetType.ALLOCATION_TAG)) {
+        // Check if conditions are met
+        if (!canSatisfySingleConstraintExpression(applicationId,
+            singleConstraint, currentExp, schedulerNode, tagsManager)) {
+          return false;
+        }
+      } else if (currentExp.getTargetType().equals(TargetType.NODE_ATTRIBUTE)
+          && currentExp.getTargetKey().equals(NODE_PARTITION)) {
+        // This is a node partition expression, check it.
+        canSatisfyNodePartitionConstraintExpresssion(currentExp, schedulerNode);
+      }
+    }
+    // return true if all targetExpressions are satisfied
+    return true;
+  }
+
+  /**
+   * Returns true if all placement constraints are **currently** satisfied by a
+   * specific scheduler Node..
+   *
+   * To do so the method retrieves and goes through all application constraint
+   * expressions and checks if the specific allocation is between the allowed
+   * min-max cardinality values under the constraint scope (Node/Rack/etc).
+   *
+   * @param applicationId applicationId,
+   * @param placementConstraint placement constraint.
+   * @param node the scheduler node
+   * @param tagsManager the allocation tags store
+   * @return true if all application constraints are satisfied by node
+   * @throws InvalidAllocationTagsQueryException
+   */
+  public static boolean canSatisfySingleConstraint(ApplicationId applicationId,
+      PlacementConstraint placementConstraint, SchedulerNode node,
+      AllocationTagsManager tagsManager)
+      throws InvalidAllocationTagsQueryException {
+    if (placementConstraint == null) {
+      return true;
+    }
+    // Transform to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(placementConstraint);
+    placementConstraint = singleTransformer.transform();
+    AbstractConstraint sConstraintExpr = placementConstraint.getConstraintExpr();
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+
+    return canSatisfySingleConstraint(applicationId, single, node, tagsManager);
   }
 
   /**
-   * Returns true if all application constraints with associated allocationTags
+   * Returns true if all placement constraints with associated allocationTags
    * are **currently** satisfied by a specific scheduler Node.
    * To do so the method retrieves and goes through all application constraint
    * expressions and checks if the specific allocation is between the allowed
@@ -98,41 +198,12 @@ public final class PlacementConstraintsUtil {
    * @return true if all application constraints are satisfied by node
    * @throws InvalidAllocationTagsQueryException
    */
-  public static boolean canSatisfyConstraints(ApplicationId appId,
+  public static boolean canSatisfySingleConstraint(ApplicationId appId,
       Set<String> allocationTags, SchedulerNode node,
       PlacementConstraintManager pcm, AllocationTagsManager tagsManager)
       throws InvalidAllocationTagsQueryException {
     PlacementConstraint constraint = pcm.getConstraint(appId, allocationTags);
-    if (constraint == null) {
-      return true;
-    }
-    // Transform to SimpleConstraint
-    SingleConstraintTransformer singleTransformer =
-        new SingleConstraintTransformer(constraint);
-    constraint = singleTransformer.transform();
-    AbstractConstraint sConstraintExpr = constraint.getConstraintExpr();
-    SingleConstraint single = (SingleConstraint) sConstraintExpr;
-    // Iterate through TargetExpressions
-    Iterator<TargetExpression> expIt = single.getTargetExpressions().iterator();
-    while (expIt.hasNext()) {
-      TargetExpression currentExp = expIt.next();
-      // Supporting AllocationTag Expressions for now
-      if (currentExp.getTargetType().equals(TargetType.ALLOCATION_TAG)) {
-        // If source and tag allocation tags are the same, we do not enforce
-        // constraints with minimum cardinality.
-        if (currentExp.getTargetValues().equals(allocationTags)
-            && single.getMinCardinality() > 0) {
-          return true;
-        }
-        // Check if conditions are met
-        if (!canSatisfySingleConstraintExpression(appId, single, currentExp,
-            node, tagsManager)) {
-          return false;
-        }
-      }
-    }
-    // return true if all targetExpressions are satisfied
-    return true;
+    return canSatisfySingleConstraint(appId, constraint, node, tagsManager);
   }
 
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/algorithm/DefaultPlacementAlgorithm.java

@@ -67,7 +67,7 @@ public class DefaultPlacementAlgorithm implements ConstraintPlacementAlgorithm {
       throws InvalidAllocationTagsQueryException {
     int numAllocs = schedulingRequest.getResourceSizing().getNumAllocations();
     if (numAllocs > 0) {
-      if (PlacementConstraintsUtil.canSatisfyConstraints(appId,
+      if (PlacementConstraintsUtil.canSatisfySingleConstraint(appId,
           schedulingRequest.getAllocationTags(), schedulerNode,
           constraintManager, tagsManager)) {
         return true;

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/processor/PlacementProcessor.java

@@ -188,12 +188,18 @@ public class PlacementProcessor implements ApplicationMasterServiceProcessor {
   @Override
   public void allocate(ApplicationAttemptId appAttemptId,
       AllocateRequest request, AllocateResponse response) throws YarnException {
+    // Copy the scheduling request since we will clear it later after sending
+    // to dispatcher
     List<SchedulingRequest> schedulingRequests =
-        request.getSchedulingRequests();
+        new ArrayList<>(request.getSchedulingRequests());
     dispatchRequestsForPlacement(appAttemptId, schedulingRequests);
     reDispatchRetryableRequests(appAttemptId);
     schedulePlacedRequests(appAttemptId);
 
+    // Remove SchedulingRequest from AllocateRequest to avoid SchedulingRequest
+    // added to scheduler.
+    request.setSchedulingRequests(Collections.emptyList());
+
     nextAMSProcessor.allocate(appAttemptId, request, response);
 
     handleRejectedRequests(appAttemptId, response);

+ 8 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.ReservationId;
 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.SchedulingRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -830,9 +831,9 @@ public class FairScheduler extends
 
   @Override
   public Allocation allocate(ApplicationAttemptId appAttemptId,
-      List<ResourceRequest> ask, List<ContainerId> release,
-      List<String> blacklistAdditions, List<String> blacklistRemovals,
-      ContainerUpdates updateRequests) {
+      List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
+      List<ContainerId> release, List<String> blacklistAdditions,
+      List<String> blacklistRemovals, ContainerUpdates updateRequests) {
 
     // Make sure this application exists
     FSAppAttempt application = getSchedulerApp(appAttemptId);
@@ -857,7 +858,9 @@ public class FairScheduler extends
     handleContainerUpdates(application, updateRequests);
 
     // Sanity check
-    normalizeRequests(ask);
+    normalizeResourceRequests(ask);
+
+    // TODO, normalize SchedulingRequest
 
     // Record container allocation start time
     application.recordContainerRequestTime(getClock().getTime());
@@ -879,6 +882,7 @@ public class FairScheduler extends
         // Update application requests
         application.updateResourceRequests(ask);
 
+        // TODO, handle SchedulingRequest
         application.showRequests();
       }
     } finally {

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

@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -320,8 +321,8 @@ public class FifoScheduler extends
 
   @Override
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
-      List<ResourceRequest> ask, List<ContainerId> release,
-      List<String> blacklistAdditions, List<String> blacklistRemovals,
+      List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
+      List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals,
       ContainerUpdates updateRequests) {
     FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
@@ -342,7 +343,7 @@ public class FifoScheduler extends
     }
 
     // Sanity check
-    normalizeRequests(ask);
+    normalizeResourceRequests(ask);
 
     // Release containers
     releaseContainers(release, application);

+ 46 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java

@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
 
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
@@ -29,7 +31,6 @@ import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
 import java.util.Collection;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 
 /**
@@ -50,13 +51,18 @@ import java.util.Map;
  * requests.
  * </p>
  */
-public interface AppPlacementAllocator<N extends SchedulerNode> {
+public abstract class AppPlacementAllocator<N extends SchedulerNode> {
+  protected AppSchedulingInfo appSchedulingInfo;
+  protected SchedulerRequestKey schedulerRequestKey;
+  protected RMContext rmContext;
+
   /**
    * Get iterator of preferred node depends on requirement and/or availability
    * @param candidateNodeSet input CandidateNodeSet
    * @return iterator of preferred node
    */
-  Iterator<N> getPreferredNodeIterator(CandidateNodeSet<N> candidateNodeSet);
+  public abstract Iterator<N> getPreferredNodeIterator(
+      CandidateNodeSet<N> candidateNodeSet);
 
   /**
    * Replace existing pending asks by the new requests
@@ -66,15 +72,29 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    * requests for preempted container
    * @return true if total pending resource changed
    */
-  PendingAskUpdateResult updatePendingAsk(
+  public abstract PendingAskUpdateResult updatePendingAsk(
       Collection<ResourceRequest> requests,
       boolean recoverPreemptedRequestForAContainer);
 
+  /**
+   * Replace existing pending asks by the new SchedulingRequest
+   *
+   * @param schedulerRequestKey                  scheduler request key
+   * @param schedulingRequest                    new asks
+   * @param recoverPreemptedRequestForAContainer if we're recovering resource
+   *                                             requests for preempted container
+   * @return true if total pending resource changed
+   */
+  public abstract PendingAskUpdateResult updatePendingAsk(
+      SchedulerRequestKey schedulerRequestKey,
+      SchedulingRequest schedulingRequest,
+      boolean recoverPreemptedRequestForAContainer);
+
   /**
    * Get pending ResourceRequests by given schedulerRequestKey
    * @return Map of resourceName to ResourceRequest
    */
-  Map<String, ResourceRequest> getResourceRequests();
+  public abstract Map<String, ResourceRequest> getResourceRequests();
 
   /**
    * Get pending ask for given resourceName. If there's no such pendingAsk,
@@ -83,7 +103,7 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    * @param resourceName resourceName
    * @return PendingAsk
    */
-  PendingAsk getPendingAsk(String resourceName);
+  public abstract PendingAsk getPendingAsk(String resourceName);
 
   /**
    * Get #pending-allocations for given resourceName. If there's no such
@@ -92,7 +112,7 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    * @param resourceName resourceName
    * @return #pending-allocations
    */
-  int getOutstandingAsksCount(String resourceName);
+  public abstract int getOutstandingAsksCount(String resourceName);
 
   /**
    * Notify container allocated.
@@ -103,7 +123,7 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    *         the container. This will be used by scheduler to recover requests.
    *         Please refer to {@link ContainerRequest} for more details.
    */
-  ContainerRequest allocate(SchedulerRequestKey schedulerKey,
+  public abstract ContainerRequest allocate(SchedulerRequestKey schedulerKey,
       NodeType type, SchedulerNode node);
 
   /**
@@ -112,7 +132,7 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    * @param node which node we will allocate on
    * @return true if we has pending requirement
    */
-  boolean canAllocate(NodeType type, SchedulerNode node);
+  public abstract boolean canAllocate(NodeType type, SchedulerNode node);
 
   /**
    * Can delay to give locality?
@@ -123,16 +143,16 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    * @param resourceName resourceName
    * @return can/cannot
    */
-  boolean canDelayTo(String resourceName);
+  public abstract boolean canDelayTo(String resourceName);
 
   /**
-   * Does this {@link AppPlacementAllocator} accept resources on nodePartition?
+   * Does this {@link AppPlacementAllocator} accept resources on given node?
    *
-   * @param nodePartition nodePartition
+   * @param schedulerNode schedulerNode
    * @param schedulingMode schedulingMode
    * @return accepted/not
    */
-  boolean acceptNodePartition(String nodePartition,
+  public abstract boolean precheckNode(SchedulerNode schedulerNode,
       SchedulingMode schedulingMode);
 
   /**
@@ -142,7 +162,7 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    *
    * @return primary requested node partition
    */
-  String getPrimaryRequestedNodePartition();
+  public abstract String getPrimaryRequestedNodePartition();
 
   /**
    * @return number of unique location asks with #pending greater than 0,
@@ -152,18 +172,24 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    * and should belong to specific delay scheduling policy impl.
    * See YARN-7457 for more details.
    */
-  int getUniqueLocationAsks();
+  public abstract int getUniqueLocationAsks();
 
   /**
    * Print human-readable requests to LOG debug.
    */
-  void showRequests();
+  public abstract void showRequests();
 
   /**
-   * Set app scheduling info.
+   * Initialize this allocator, this will be called by Factory automatically
    *
-   * @param appSchedulingInfo
-   *          app info object.
+   * @param appSchedulingInfo appSchedulingInfo
+   * @param schedulerRequestKey schedulerRequestKey
+   * @param rmContext rmContext
    */
-  void setAppSchedulingInfo(AppSchedulingInfo appSchedulingInfo);
+  public void initialize(AppSchedulingInfo appSchedulingInfo,
+      SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {
+    this.appSchedulingInfo = appSchedulingInfo;
+    this.rmContext = rmContext;
+    this.schedulerRequestKey = schedulerRequestKey;
+  }
 }

+ 18 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java

@@ -22,8 +22,9 @@ import org.apache.commons.collections.IteratorUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
@@ -46,26 +47,18 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
  * containers.
  */
 public class LocalityAppPlacementAllocator <N extends SchedulerNode>
-    implements AppPlacementAllocator<N> {
+    extends AppPlacementAllocator<N> {
   private static final Log LOG =
       LogFactory.getLog(LocalityAppPlacementAllocator.class);
 
   private final Map<String, ResourceRequest> resourceRequestMap =
       new ConcurrentHashMap<>();
-  private AppSchedulingInfo appSchedulingInfo;
   private volatile String primaryRequestedPartition =
       RMNodeLabelsManager.NO_LABEL;
 
   private final ReentrantReadWriteLock.ReadLock readLock;
   private final ReentrantReadWriteLock.WriteLock writeLock;
 
-  public LocalityAppPlacementAllocator(AppSchedulingInfo info) {
-    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    readLock = lock.readLock();
-    writeLock = lock.writeLock();
-    this.appSchedulingInfo = info;
-  }
-
   public LocalityAppPlacementAllocator() {
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     readLock = lock.readLock();
@@ -181,6 +174,19 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
     }
   }
 
+  @Override
+  public PendingAskUpdateResult updatePendingAsk(
+      SchedulerRequestKey schedulerRequestKey,
+      SchedulingRequest schedulingRequest,
+      boolean recoverPreemptedRequestForAContainer)
+      throws SchedulerInvalidResoureRequestException {
+    throw new SchedulerInvalidResoureRequestException(this.getClass().getName()
+        + " not be able to handle SchedulingRequest, there exists a "
+        + "ResourceRequest with the same scheduler key=" + schedulerRequestKey
+        + ", please send SchedulingRequest with a different allocationId and "
+        + "priority");
+  }
+
   @Override
   public Map<String, ResourceRequest> getResourceRequests() {
     return resourceRequestMap;
@@ -362,13 +368,13 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
   }
 
   @Override
-  public boolean acceptNodePartition(String nodePartition,
+  public boolean precheckNode(SchedulerNode schedulerNode,
       SchedulingMode schedulingMode) {
     // We will only look at node label = nodeLabelToLookAt according to
     // schedulingMode and partition of node.
     String nodePartitionToLookAt;
     if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
-      nodePartitionToLookAt = nodePartition;
+      nodePartitionToLookAt = schedulerNode.getPartition();
     } else {
       nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL;
     }
@@ -425,9 +431,4 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
       writeLock.unlock();
     }
   }
-
-  @Override
-  public void setAppSchedulingInfo(AppSchedulingInfo appSchedulingInfo) {
-    this.appSchedulingInfo = appSchedulingInfo;
-  }
 }

+ 531 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java

@@ -0,0 +1,531 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.collections.IteratorUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.hadoop.yarn.api.records.impl.pb.SchedulingRequestPBImpl;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
+import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.InvalidAllocationTagsQueryException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintsUtil;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.APPLICATION_LABEL_INTRA_APPLICATION;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE_PARTITION;
+
+/**
+ * This is a simple implementation to do affinity or anti-affinity for
+ * inter/intra apps.
+ */
+public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
+    extends AppPlacementAllocator<N> {
+  private static final Log LOG =
+      LogFactory.getLog(SingleConstraintAppPlacementAllocator.class);
+
+  private ReentrantReadWriteLock.ReadLock readLock;
+  private ReentrantReadWriteLock.WriteLock writeLock;
+
+  private SchedulingRequest schedulingRequest = null;
+  private String targetNodePartition;
+  private Set<String> targetAllocationTags;
+  private AllocationTagsManager allocationTagsManager;
+
+  public SingleConstraintAppPlacementAllocator() {
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public Iterator<N> getPreferredNodeIterator(
+      CandidateNodeSet<N> candidateNodeSet) {
+    // Now only handle the case that single node in the candidateNodeSet
+    // TODO, Add support to multi-hosts inside candidateNodeSet which is passed
+    // in.
+
+    N singleNode = CandidateNodeSetUtils.getSingleNode(candidateNodeSet);
+    if (null != singleNode) {
+      return IteratorUtils.singletonIterator(singleNode);
+    }
+
+    return IteratorUtils.emptyIterator();
+  }
+
+  @Override
+  public PendingAskUpdateResult updatePendingAsk(
+      Collection<ResourceRequest> requests,
+      boolean recoverPreemptedRequestForAContainer) {
+    if (requests != null && !requests.isEmpty()) {
+      throw new SchedulerInvalidResoureRequestException(
+          this.getClass().getName()
+              + " not be able to handle ResourceRequest, there exists a "
+              + "SchedulingRequest with the same scheduler key="
+              + SchedulerRequestKey.create(requests.iterator().next())
+              + ", please send ResourceRequest with a different allocationId and "
+              + "priority");
+    }
+
+    // Do nothing
+    return null;
+  }
+
+  private PendingAskUpdateResult internalUpdatePendingAsk(
+      SchedulingRequest newSchedulingRequest, boolean recoverContainer) {
+    // When it is a recover container, there must exists an schedulingRequest.
+    if (recoverContainer && schedulingRequest == null) {
+      throw new SchedulerInvalidResoureRequestException("Trying to recover a "
+          + "container request=" + newSchedulingRequest.toString() + ", however"
+          + "there's no existing scheduling request, this should not happen.");
+    }
+
+    if (schedulingRequest != null) {
+      // If we have an old scheduling request, we will make sure that no changes
+      // made except sizing.
+      // To avoid unnecessary copy of the data structure, we do this by
+      // replacing numAllocations with old numAllocations in the
+      // newSchedulingRequest#getResourceSizing, and compare the two objects.
+      ResourceSizing sizing = newSchedulingRequest.getResourceSizing();
+      int existingNumAllocations =
+          schedulingRequest.getResourceSizing().getNumAllocations();
+
+      // When it is a recovered container request, just set
+      // #newAllocations = #existingAllocations + 1;
+      int newNumAllocations;
+      if (recoverContainer) {
+        newNumAllocations = existingNumAllocations + 1;
+      } else {
+        newNumAllocations = sizing.getNumAllocations();
+      }
+      sizing.setNumAllocations(existingNumAllocations);
+
+      // Compare two objects
+      if (!schedulingRequest.equals(newSchedulingRequest)) {
+        // Rollback #numAllocations
+        sizing.setNumAllocations(newNumAllocations);
+        throw new SchedulerInvalidResoureRequestException(
+            "Invalid updated SchedulingRequest added to scheduler, "
+                + " we only allows changing numAllocations for the updated "
+                + "SchedulingRequest. Old=" + schedulingRequest.toString()
+                + " new=" + newSchedulingRequest.toString()
+                + ", if any fields need to be updated, please cancel the "
+                + "old request (by setting numAllocations to 0) and send a "
+                + "SchedulingRequest with different combination of "
+                + "priority/allocationId");
+      } else {
+        if (newNumAllocations == existingNumAllocations) {
+          // No update on pending asks, return null.
+          return null;
+        }
+      }
+
+      // Rollback #numAllocations
+      sizing.setNumAllocations(newNumAllocations);
+
+      // Basic sanity check
+      if (newNumAllocations < 0) {
+        throw new SchedulerInvalidResoureRequestException(
+            "numAllocation in ResourceSizing field must be >= 0, "
+                + "updating schedulingRequest failed.");
+      }
+
+      PendingAskUpdateResult updateResult = new PendingAskUpdateResult(
+          new PendingAsk(schedulingRequest.getResourceSizing()),
+          new PendingAsk(newSchedulingRequest.getResourceSizing()),
+          targetNodePartition, targetNodePartition);
+
+      // Ok, now everything is same except numAllocation, update numAllocation.
+      this.schedulingRequest.getResourceSizing().setNumAllocations(
+          newNumAllocations);
+      LOG.info(
+          "Update numAllocation from old=" + existingNumAllocations + " to new="
+              + newNumAllocations);
+
+      return updateResult;
+    }
+
+    // For a new schedulingRequest, we need to validate if we support its asks.
+    // This will update internal partitions, etc. after the SchedulingRequest is
+    // valid.
+    validateAndSetSchedulingRequest(newSchedulingRequest);
+
+    return new PendingAskUpdateResult(null,
+        new PendingAsk(newSchedulingRequest.getResourceSizing()), null,
+        targetNodePartition);
+  }
+
+  @Override
+  public PendingAskUpdateResult updatePendingAsk(
+      SchedulerRequestKey schedulerRequestKey,
+      SchedulingRequest newSchedulingRequest,
+      boolean recoverPreemptedRequestForAContainer) {
+    writeLock.lock();
+    try {
+      return internalUpdatePendingAsk(newSchedulingRequest,
+          recoverPreemptedRequestForAContainer);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private String throwExceptionWithMetaInfo(String message) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("AppId=").append(appSchedulingInfo.getApplicationId()).append(
+        " Key=").append(this.schedulerRequestKey).append(". Exception message:")
+        .append(message);
+    throw new SchedulerInvalidResoureRequestException(sb.toString());
+  }
+
+  private void validateAndSetSchedulingRequest(SchedulingRequest newSchedulingRequest)
+      throws SchedulerInvalidResoureRequestException {
+    // Check sizing exists
+    if (newSchedulingRequest.getResourceSizing() == null
+        || newSchedulingRequest.getResourceSizing().getResources() == null) {
+      throwExceptionWithMetaInfo(
+          "No ResourceSizing found in the scheduling request, please double "
+              + "check");
+    }
+
+    // Check execution type == GUARANTEED
+    if (newSchedulingRequest.getExecutionType() != null
+        && newSchedulingRequest.getExecutionType().getExecutionType()
+        != ExecutionType.GUARANTEED) {
+      throwExceptionWithMetaInfo(
+          "Only GUARANTEED execution type is supported.");
+    }
+
+    PlacementConstraint constraint =
+        newSchedulingRequest.getPlacementConstraint();
+
+    // We only accept SingleConstraint
+    PlacementConstraint.AbstractConstraint ac = constraint.getConstraintExpr();
+    if (!(ac instanceof PlacementConstraint.SingleConstraint)) {
+      throwExceptionWithMetaInfo(
+          "Only accepts " + PlacementConstraint.SingleConstraint.class.getName()
+              + " as constraint-expression. Rejecting the new added "
+              + "constraint-expression.class=" + ac.getClass().getName());
+    }
+
+    PlacementConstraint.SingleConstraint singleConstraint =
+        (PlacementConstraint.SingleConstraint) ac;
+
+    // Make sure it is an anti-affinity request (actually this implementation
+    // should be able to support both affinity / anti-affinity without much
+    // effort. Considering potential test effort required. Limit to
+    // anti-affinity to intra-app and scope is node.
+    if (!singleConstraint.getScope().equals(PlacementConstraints.NODE)) {
+      throwExceptionWithMetaInfo(
+          "Only support scope=" + PlacementConstraints.NODE
+              + "now. PlacementConstraint=" + singleConstraint);
+    }
+
+    if (singleConstraint.getMinCardinality() != 0
+        || singleConstraint.getMaxCardinality() != 1) {
+      throwExceptionWithMetaInfo(
+          "Only support anti-affinity, which is: minCardinality=0, "
+              + "maxCardinality=1");
+    }
+
+    Set<PlacementConstraint.TargetExpression> targetExpressionSet =
+        singleConstraint.getTargetExpressions();
+    if (targetExpressionSet == null || targetExpressionSet.isEmpty()) {
+      throwExceptionWithMetaInfo(
+          "TargetExpression should not be null or empty");
+    }
+
+    // Set node partition
+    String nodePartition = null;
+
+    // Target allocation tags
+    Set<String> targetAllocationTags = null;
+
+    for (PlacementConstraint.TargetExpression targetExpression : targetExpressionSet) {
+      // Handle node partition
+      if (targetExpression.getTargetType().equals(
+          PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE)) {
+        // For node attribute target, we only support Partition now. And once
+        // YARN-3409 is merged, we will support node attribute.
+        if (!targetExpression.getTargetKey().equals(NODE_PARTITION)) {
+          throwExceptionWithMetaInfo("When TargetType="
+              + PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE
+              + " only " + NODE_PARTITION + " is accepted as TargetKey.");
+        }
+
+        if (nodePartition != null) {
+          // This means we have duplicated node partition entry inside placement
+          // constraint, which might be set by mistake.
+          throwExceptionWithMetaInfo(
+              "Only one node partition targetExpression is allowed");
+        }
+
+        Set<String> values = targetExpression.getTargetValues();
+        if (values == null || values.isEmpty()) {
+          nodePartition = RMNodeLabelsManager.NO_LABEL;
+          continue;
+        }
+
+        if (values.size() > 1) {
+          throwExceptionWithMetaInfo("Inside one targetExpression, we only "
+              + "support affinity to at most one node partition now");
+        }
+
+        nodePartition = values.iterator().next();
+      } else if (targetExpression.getTargetType().equals(
+          PlacementConstraint.TargetExpression.TargetType.ALLOCATION_TAG)) {
+        // Handle allocation tags
+        if (targetAllocationTags != null) {
+          // This means we have duplicated AllocationTag expressions entries
+          // inside placement constraint, which might be set by mistake.
+          throwExceptionWithMetaInfo(
+              "Only one AllocationTag targetExpression is allowed");
+        }
+
+        if (targetExpression.getTargetValues() == null || targetExpression
+            .getTargetValues().isEmpty()) {
+          throwExceptionWithMetaInfo("Failed to find allocation tags from "
+              + "TargetExpressions or couldn't find self-app target.");
+        }
+
+        targetAllocationTags = new HashSet<>(
+            targetExpression.getTargetValues());
+
+        if (targetExpression.getTargetKey() == null || !targetExpression
+            .getTargetKey().equals(APPLICATION_LABEL_INTRA_APPLICATION)) {
+          throwExceptionWithMetaInfo(
+              "As of now, the only accepted target key for targetKey of "
+                  + "allocation_tag target expression is: ["
+                  + APPLICATION_LABEL_INTRA_APPLICATION
+                  + "]. Please make changes to placement constraints "
+                  + "accordingly.");
+        }
+      }
+    }
+
+    if (targetAllocationTags == null) {
+      // That means we don't have ALLOCATION_TAG specified
+      throwExceptionWithMetaInfo(
+          "Couldn't find target expression with type == ALLOCATION_TAG, it is "
+              + "required to include one and only one target expression with "
+              + "type == ALLOCATION_TAG");
+
+    }
+
+    if (nodePartition == null) {
+      nodePartition = RMNodeLabelsManager.NO_LABEL;
+    }
+
+    // Validation is done. set local results:
+    this.targetNodePartition = nodePartition;
+    this.targetAllocationTags = targetAllocationTags;
+
+    this.schedulingRequest = new SchedulingRequestPBImpl(
+        ((SchedulingRequestPBImpl) newSchedulingRequest).getProto());
+
+    LOG.info("Successfully added SchedulingRequest to app=" + appSchedulingInfo
+        .getApplicationAttemptId() + " targetAllocationTags=[" + StringUtils
+        .join(",", targetAllocationTags) + "]. nodePartition="
+        + targetNodePartition);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public Map<String, ResourceRequest> getResourceRequests() {
+    return Collections.EMPTY_MAP;
+  }
+
+  @Override
+  public PendingAsk getPendingAsk(String resourceName) {
+    readLock.lock();
+    try {
+      if (resourceName.equals("*") && schedulingRequest != null) {
+        return new PendingAsk(schedulingRequest.getResourceSizing());
+      }
+      return PendingAsk.ZERO;
+    } finally {
+      readLock.unlock();
+    }
+
+  }
+
+  @Override
+  public int getOutstandingAsksCount(String resourceName) {
+    readLock.lock();
+    try {
+      if (resourceName.equals("*") && schedulingRequest != null) {
+        return schedulingRequest.getResourceSizing().getNumAllocations();
+      }
+      return 0;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  private void decreasePendingNumAllocation() {
+    // Deduct pending #allocations by 1
+    ResourceSizing sizing = schedulingRequest.getResourceSizing();
+    sizing.setNumAllocations(sizing.getNumAllocations() - 1);
+  }
+
+  @Override
+  public ContainerRequest allocate(SchedulerRequestKey schedulerKey,
+      NodeType type, SchedulerNode node) {
+    writeLock.lock();
+    try {
+      // Per container scheduling request, it is just a copy of existing
+      // scheduling request with #allocations=1
+      SchedulingRequest containerSchedulingRequest = new SchedulingRequestPBImpl(
+          ((SchedulingRequestPBImpl) schedulingRequest).getProto());
+      containerSchedulingRequest.getResourceSizing().setNumAllocations(1);
+
+      // Deduct sizing
+      decreasePendingNumAllocation();
+
+      return new ContainerRequest(containerSchedulingRequest);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private boolean checkCardinalityAndPending(SchedulerNode node) {
+    // Do we still have pending resource?
+    if (schedulingRequest.getResourceSizing().getNumAllocations() <= 0) {
+      return false;
+    }
+
+    // node type will be ignored.
+    try {
+      return PlacementConstraintsUtil.canSatisfySingleConstraint(
+          appSchedulingInfo.getApplicationId(),
+          this.schedulingRequest.getPlacementConstraint(), node,
+          allocationTagsManager);
+    } catch (InvalidAllocationTagsQueryException e) {
+      LOG.warn("Failed to query node cardinality:", e);
+      return false;
+    }
+  }
+
+  @Override
+  public boolean canAllocate(NodeType type, SchedulerNode node) {
+    try {
+      readLock.lock();
+      return checkCardinalityAndPending(node);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public boolean canDelayTo(String resourceName) {
+    return true;
+  }
+
+  @Override
+  public boolean precheckNode(SchedulerNode schedulerNode,
+      SchedulingMode schedulingMode) {
+    // We will only look at node label = nodeLabelToLookAt according to
+    // schedulingMode and partition of node.
+    String nodePartitionToLookAt;
+    if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
+      nodePartitionToLookAt = schedulerNode.getPartition();
+    } else{
+      nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL;
+    }
+
+    readLock.lock();
+    try {
+      // Check node partition as well as cardinality/pending resources.
+      return this.targetNodePartition.equals(nodePartitionToLookAt)
+          && checkCardinalityAndPending(schedulerNode);
+    } finally {
+      readLock.unlock();
+    }
+
+  }
+
+  @Override
+  public String getPrimaryRequestedNodePartition() {
+    return targetNodePartition;
+  }
+
+  @Override
+  public int getUniqueLocationAsks() {
+    return 1;
+  }
+
+  @Override
+  public void showRequests() {
+    try {
+      readLock.lock();
+      if (schedulingRequest != null) {
+        LOG.info(schedulingRequest.toString());
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @VisibleForTesting
+  SchedulingRequest getSchedulingRequest() {
+    return schedulingRequest;
+  }
+
+  @VisibleForTesting
+  String getTargetNodePartition() {
+    return targetNodePartition;
+  }
+
+  @VisibleForTesting
+  Set<String> getTargetAllocationTags() {
+    return targetAllocationTags;
+  }
+
+  @Override
+  public void initialize(AppSchedulingInfo appSchedulingInfo,
+      SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {
+    super.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    this.allocationTagsManager = rmContext.getAllocationTagsManager();
+  }
+}

+ 4 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java

@@ -331,8 +331,7 @@ public class Application {
     
     // Get resources from the ResourceManager
     Allocation allocation = resourceManager.getResourceScheduler().allocate(
-        applicationAttemptId, new ArrayList<ResourceRequest>(ask),
-        new ArrayList<ContainerId>(), null, null,
+        applicationAttemptId, new ArrayList<ResourceRequest>(ask), null, new ArrayList<ContainerId>(), null, null,
         new ContainerUpdates());
 
     if (LOG.isInfoEnabled()) {
@@ -431,7 +430,7 @@ public class Application {
     if (type == NodeType.NODE_LOCAL) {
       for (String host : task.getHosts()) {
         if(LOG.isDebugEnabled()) {
-          LOG.debug("updatePendingAsk:" + " application=" + applicationId
+          LOG.debug("updateResourceDemands:" + " application=" + applicationId
             + " type=" + type + " host=" + host
             + " request=" + ((requests == null) ? "null" : requests.get(host)));
         }
@@ -442,7 +441,7 @@ public class Application {
     if (type == NodeType.NODE_LOCAL || type == NodeType.RACK_LOCAL) {
       for (String rack : task.getRacks()) {
         if(LOG.isDebugEnabled()) {
-          LOG.debug("updatePendingAsk:" + " application=" + applicationId
+          LOG.debug("updateResourceDemands:" + " application=" + applicationId
             + " type=" + type + " rack=" + rack
             + " request=" + ((requests == null) ? "null" : requests.get(rack)));
         }
@@ -453,7 +452,7 @@ public class Application {
     updateResourceRequest(requests.get(ResourceRequest.ANY));
     
     if(LOG.isDebugEnabled()) {
-      LOG.debug("updatePendingAsk:" + " application=" + applicationId
+      LOG.debug("updateResourceDemands:" + " application=" + applicationId
         + " #asks=" + ask.size());
     }
   }

+ 51 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -37,14 +38,17 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 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.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 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.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -281,6 +285,53 @@ public class MockAM {
     }
     return allocate(req);
   }
+
+  public AllocateResponse allocate(List<ResourceRequest> resourceRequest,
+      List<SchedulingRequest> newSchedulingRequests, List<ContainerId> releases)
+      throws Exception {
+    final AllocateRequest req =
+        AllocateRequest.newInstance(0, 0F, resourceRequest,
+            releases, null);
+    if (newSchedulingRequests != null) {
+      addSchedulingRequest(newSchedulingRequests);
+    }
+    if (!schedulingRequests.isEmpty()) {
+      req.setSchedulingRequests(schedulingRequests);
+      schedulingRequests.clear();
+    }
+    return allocate(req);
+  }
+
+  public AllocateResponse allocateIntraAppAntiAffinity(
+      ResourceSizing resourceSizing, Priority priority, long allocationId,
+      Set<String> allocationTags, String... targetTags) throws Exception {
+    return this.allocate(null,
+        Arrays.asList(SchedulingRequest.newBuilder().executionType(
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+            .allocationRequestId(allocationId).priority(priority)
+            .allocationTags(allocationTags).placementConstraintExpression(
+                PlacementConstraints
+                    .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                        PlacementConstraints.PlacementTargets
+                            .allocationTagToIntraApp(targetTags)).build())
+            .resourceSizing(resourceSizing).build()), null);
+  }
+
+  public AllocateResponse allocateIntraAppAntiAffinity(
+      String nodePartition, ResourceSizing resourceSizing, Priority priority,
+      long allocationId, String... tags) throws Exception {
+    return this.allocate(null,
+        Arrays.asList(SchedulingRequest.newBuilder().executionType(
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+            .allocationRequestId(allocationId).priority(priority)
+            .placementConstraintExpression(PlacementConstraints
+                .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                    PlacementConstraints.PlacementTargets
+                        .allocationTagToIntraApp(tags),
+                    PlacementConstraints.PlacementTargets
+                        .nodePartition(nodePartition)).build())
+            .resourceSizing(resourceSizing).build()), null);
+  }
   
   public AllocateResponse sendContainerResizingRequest(
       List<UpdateContainerRequest> updateRequests) throws Exception {

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

@@ -474,7 +474,7 @@ public class TestRMAppAttemptTransitions {
 
     assertEquals(expectedState, applicationAttempt.getAppAttemptState());
     verify(scheduler, times(expectedAllocateCount)).allocate(
-        any(ApplicationAttemptId.class), any(List.class), any(List.class),
+        any(ApplicationAttemptId.class), any(List.class), eq(null), any(List.class),
         any(List.class), any(List.class), any(ContainerUpdates.class));
 
     assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
@@ -495,7 +495,7 @@ public class TestRMAppAttemptTransitions {
     // Check events
     verify(applicationMasterLauncher).handle(any(AMLauncherEvent.class));
     verify(scheduler, times(2)).allocate(any(ApplicationAttemptId.class),
-        any(List.class), any(List.class), any(List.class), any(List.class),
+        any(List.class), any(List.class), any(List.class), any(List.class), any(List.class),
         any(ContainerUpdates.class));
     verify(nmTokenManager).clearNodeSetForAttempt(
       applicationAttempt.getAppAttemptId());
@@ -643,7 +643,7 @@ public class TestRMAppAttemptTransitions {
     when(allocation.getContainers()).
         thenReturn(Collections.singletonList(container));
     when(scheduler.allocate(any(ApplicationAttemptId.class), any(List.class),
-        any(List.class), any(List.class), any(List.class),
+        any(List.class), any(List.class), any(List.class), any(List.class),
         any(ContainerUpdates.class))).
     thenReturn(allocation);
     RMContainer rmContainer = mock(RMContainerImpl.class);
@@ -1161,7 +1161,7 @@ public class TestRMAppAttemptTransitions {
     when(allocation.getContainers()).
         thenReturn(Collections.singletonList(amContainer));
     when(scheduler.allocate(any(ApplicationAttemptId.class), any(List.class),
-        any(List.class), any(List.class), any(List.class),
+        any(List.class), any(List.class), any(List.class), any(List.class),
         any(ContainerUpdates.class)))
         .thenReturn(allocation);
     RMContainer rmContainer = mock(RMContainerImpl.class);
@@ -1636,7 +1636,7 @@ public class TestRMAppAttemptTransitions {
   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),
+        any(List.class), any(List.class), any(List.class), any(List.class), any(List.class),
         any(ContainerUpdates.class)))
         .thenAnswer(new Answer<Allocation>() {
 

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java

@@ -33,6 +33,7 @@ import java.util.ArrayList;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -420,9 +421,10 @@ public class TestRMContainerImpl {
     when(rmContext.getYarnConfiguration()).thenReturn(conf);
 
     /* First container: ALLOCATED -> KILLED */
-    RMContainer rmContainer = new RMContainerImpl(container,
+    RMContainerImpl rmContainer = new RMContainerImpl(container,
         SchedulerRequestKey.extractFrom(container), appAttemptId,
         nodeId, "user", rmContext);
+    rmContainer.setAllocationTags(ImmutableSet.of("mapper"));
 
     Assert.assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
@@ -448,6 +450,7 @@ public class TestRMContainerImpl {
     Assert.assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
 
+    rmContainer.setAllocationTags(ImmutableSet.of("mapper"));
     rmContainer.handle(new RMContainerEvent(containerId,
         RMContainerEventType.START));
 
@@ -468,6 +471,7 @@ public class TestRMContainerImpl {
     rmContainer = new RMContainerImpl(container,
         SchedulerRequestKey.extractFrom(container), appAttemptId,
         nodeId, "user", rmContext);
+    rmContainer.setAllocationTags(ImmutableSet.of("mapper"));
 
     Assert.assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));

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

@@ -46,7 +46,7 @@ public class TestAppSchedulingInfo {
     doReturn("test").when(queue).getQueueName();
     AppSchedulingInfo appSchedulingInfo = new AppSchedulingInfo(appAttemptId,
         "test", queue, null, 0, new ResourceUsage(),
-        new HashMap<String, String>());
+        new HashMap<String, String>(), null);
 
     appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         new ArrayList<String>());
@@ -118,7 +118,7 @@ public class TestAppSchedulingInfo {
     doReturn(mock(QueueMetrics.class)).when(queue).getMetrics();
     AppSchedulingInfo  info = new AppSchedulingInfo(
         appAttemptId, "test", queue, mock(ActiveUsersManager.class), 0,
-        new ResourceUsage(), new HashMap<String, String>());
+        new ResourceUsage(), new HashMap<>(), null);
     Assert.assertEquals(0, info.getSchedulerKeys().size());
 
     Priority pri1 = Priority.newInstance(1);

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

@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import com.google.common.collect.Sets;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.junit.Assert;
+
+import java.util.Set;
+
+public class CapacitySchedulerTestBase {
+  protected final int GB = 1024;
+
+  protected static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+  protected static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+  protected static final String A1 = A + ".a1";
+  protected static final String A2 = A + ".a2";
+  protected static final String B1 = B + ".b1";
+  protected static final String B2 = B + ".b2";
+  protected static final String B3 = B + ".b3";
+  protected static float A_CAPACITY = 10.5f;
+  protected static float B_CAPACITY = 89.5f;
+  protected static final String P1 = CapacitySchedulerConfiguration.ROOT + ".p1";
+  protected static final String P2 = CapacitySchedulerConfiguration.ROOT + ".p2";
+  protected static final String X1 = P1 + ".x1";
+  protected static final String X2 = P1 + ".x2";
+  protected static final String Y1 = P2 + ".y1";
+  protected static final String Y2 = P2 + ".y2";
+  protected static float A1_CAPACITY = 30;
+  protected static float A2_CAPACITY = 70;
+  protected static float B1_CAPACITY = 79.2f;
+  protected static float B2_CAPACITY = 0.8f;
+  protected static float B3_CAPACITY = 20;
+
+
+  @SuppressWarnings("unchecked")
+  protected <E> Set<E> toSet(E... elements) {
+    Set<E> set = Sets.newHashSet(elements);
+    return set;
+  }
+
+  protected void checkPendingResource(MockRM rm, String queueName, int memory,
+      String label) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    CSQueue queue = cs.getQueue(queueName);
+    Assert.assertEquals(
+        memory,
+        queue.getQueueResourceUsage()
+            .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
+            .getMemorySize());
+  }
+
+
+  protected void checkPendingResourceGreaterThanZero(MockRM rm, String queueName,
+      String label) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    CSQueue queue = cs.getQueue(queueName);
+    Assert.assertTrue(queue.getQueueResourceUsage()
+        .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
+        .getMemorySize() > 0);
+  }
+}

+ 16 - 74
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

@@ -103,7 +103,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMW
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
-import org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule;
 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.RMAppMetrics;
@@ -167,33 +166,10 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 
-public class TestCapacityScheduler {
+public class TestCapacityScheduler extends CapacitySchedulerTestBase {
   private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
-  private final int GB = 1024;
   private final static ContainerUpdates NULL_UPDATE_REQUESTS =
       new ContainerUpdates();
-
-  private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
-  private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-  private static final String A1 = A + ".a1";
-  private static final String A2 = A + ".a2";
-  private static final String B1 = B + ".b1";
-  private static final String B2 = B + ".b2";
-  private static final String B3 = B + ".b3";
-  private static float A_CAPACITY = 10.5f;
-  private static float B_CAPACITY = 89.5f;
-  private static final String P1 = CapacitySchedulerConfiguration.ROOT + ".p1";
-  private static final String P2 = CapacitySchedulerConfiguration.ROOT + ".p2";
-  private static final String X1 = P1 + ".x1";
-  private static final String X2 = P1 + ".x2";
-  private static final String Y1 = P2 + ".y1";
-  private static final String Y2 = P2 + ".y2";
-  private static float A1_CAPACITY = 30;
-  private static float A2_CAPACITY = 70;
-  private static float B1_CAPACITY = 79.2f;
-  private static float B2_CAPACITY = 0.8f;
-  private static float B3_CAPACITY = 20;
-
   private ResourceManager resourceManager = null;
   private RMContext mockContext;
 
@@ -1116,12 +1092,12 @@ public class TestCapacityScheduler {
     cs.handle(addAttemptEvent);
 
     // Verify the blacklist can be updated independent of requesting containers
-    cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
+    cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), null,
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
     Assert.assertTrue(cs.getApplicationAttempt(appAttemptId)
         .isPlaceBlacklisted(host));
-    cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
+    cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), null,
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), NULL_UPDATE_REQUESTS);
     Assert.assertFalse(cs.getApplicationAttempt(appAttemptId)
@@ -1217,8 +1193,7 @@ public class TestCapacityScheduler {
 
     //This will allocate for app1
     cs.allocate(appAttemptId1,
-        Collections.<ResourceRequest>singletonList(r1),
-        Collections.<ContainerId>emptyList(),
+        Collections.<ResourceRequest>singletonList(r1), null, Collections.<ContainerId>emptyList(),
         null, null, NULL_UPDATE_REQUESTS);
 
     //And this will result in container assignment for app1
@@ -1234,8 +1209,7 @@ public class TestCapacityScheduler {
     //Now, allocate for app2 (this would be the first/AM allocation)
     ResourceRequest r2 = TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, true, priority, recordFactory);
     cs.allocate(appAttemptId2,
-        Collections.<ResourceRequest>singletonList(r2),
-        Collections.<ContainerId>emptyList(),
+        Collections.<ResourceRequest>singletonList(r2), null, Collections.<ContainerId>emptyList(),
         null, null, NULL_UPDATE_REQUESTS);
 
     //In this case we do not perform container assignment because we want to
@@ -3481,12 +3455,6 @@ public class TestCapacityScheduler {
         + "queue-a's max capacity will be violated if container allocated");
   }
 
-  @SuppressWarnings("unchecked")
-  private <E> Set<E> toSet(E... elements) {
-    Set<E> set = Sets.newHashSet(elements);
-    return set;
-  }
-
   @Test
   public void testQueueHierarchyPendingResourceUpdate() throws Exception {
     Configuration conf =
@@ -3618,26 +3586,6 @@ public class TestCapacityScheduler {
     checkPendingResource(rm, "root", 0 * GB, "x");
   }
 
-  private void checkPendingResource(MockRM rm, String queueName, int memory,
-      String label) {
-    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
-    CSQueue queue = cs.getQueue(queueName);
-    Assert.assertEquals(
-        memory,
-        queue.getQueueResourceUsage()
-            .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
-            .getMemorySize());
-  }
-
-  private void checkPendingResourceGreaterThanZero(MockRM rm, String queueName,
-      String label) {
-    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
-    CSQueue queue = cs.getQueue(queueName);
-    Assert.assertTrue(queue.getQueueResourceUsage()
-        .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
-        .getMemorySize() > 0);
-  }
-
   // Test verifies AM Used resource for LeafQueue when AM ResourceRequest is
   // lesser than minimumAllocation
   @Test(timeout = 30000)
@@ -3707,7 +3655,7 @@ public class TestCapacityScheduler {
 
     Allocation allocate =
         cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
-            Collections.<ContainerId> emptyList(), null, null,
+            null, Collections.<ContainerId> emptyList(), null, null,
             NULL_UPDATE_REQUESTS);
 
     Assert.assertNotNull(attempt);
@@ -3724,7 +3672,7 @@ public class TestCapacityScheduler {
 
     allocate =
         cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
-            Collections.<ContainerId> emptyList(), null, null,
+            null, Collections.<ContainerId> emptyList(), null, null,
             NULL_UPDATE_REQUESTS);
 
     // All resources should be sent as headroom
@@ -4250,8 +4198,7 @@ public class TestCapacityScheduler {
       y1Req = TestUtils.createResourceRequest(
           ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
       cs.allocate(appAttemptId3,
-          Collections.<ResourceRequest>singletonList(y1Req),
-          Collections.<ContainerId>emptyList(),
+          Collections.<ResourceRequest>singletonList(y1Req), null, Collections.<ContainerId>emptyList(),
           null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
     }
@@ -4264,8 +4211,7 @@ public class TestCapacityScheduler {
       x1Req = TestUtils.createResourceRequest(
           ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
       cs.allocate(appAttemptId1,
-          Collections.<ResourceRequest>singletonList(x1Req),
-          Collections.<ContainerId>emptyList(),
+          Collections.<ResourceRequest>singletonList(x1Req), null, Collections.<ContainerId>emptyList(),
           null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
     }
@@ -4277,8 +4223,7 @@ public class TestCapacityScheduler {
     x2Req = TestUtils.createResourceRequest(
         ResourceRequest.ANY, 2 * GB, 1, true, priority, recordFactory);
     cs.allocate(appAttemptId2,
-        Collections.<ResourceRequest>singletonList(x2Req),
-        Collections.<ContainerId>emptyList(),
+        Collections.<ResourceRequest>singletonList(x2Req), null, Collections.<ContainerId>emptyList(),
         null, null, NULL_UPDATE_REQUESTS);
     CapacityScheduler.schedule(cs);
     assertEquals("X2 Used Resource should be 0", 0,
@@ -4289,8 +4234,7 @@ public class TestCapacityScheduler {
     x1Req = TestUtils.createResourceRequest(
         ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
     cs.allocate(appAttemptId1,
-        Collections.<ResourceRequest>singletonList(x1Req),
-        Collections.<ContainerId>emptyList(),
+        Collections.<ResourceRequest>singletonList(x1Req), null, Collections.<ContainerId>emptyList(),
         null, null, NULL_UPDATE_REQUESTS);
     CapacityScheduler.schedule(cs);
     assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
@@ -4303,8 +4247,7 @@ public class TestCapacityScheduler {
       y1Req = TestUtils.createResourceRequest(
           ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
       cs.allocate(appAttemptId3,
-          Collections.<ResourceRequest>singletonList(y1Req),
-          Collections.<ContainerId>emptyList(),
+          Collections.<ResourceRequest>singletonList(y1Req), null, Collections.<ContainerId>emptyList(),
           null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
     }
@@ -4363,7 +4306,7 @@ public class TestCapacityScheduler {
         ResourceRequest.ANY, 2 * GB, 1, true, priority, recordFactory);
     //This will allocate for app1
     cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
-        Collections.<ContainerId>emptyList(),
+        null, Collections.<ContainerId>emptyList(),
         null, null, NULL_UPDATE_REQUESTS).getContainers().size();
     CapacityScheduler.schedule(cs);
     ResourceRequest r2 = null;
@@ -4371,8 +4314,7 @@ public class TestCapacityScheduler {
       r2 = TestUtils.createResourceRequest(
           ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
       cs.allocate(appAttemptId2,
-          Collections.<ResourceRequest>singletonList(r2),
-          Collections.<ContainerId>emptyList(),
+          Collections.<ResourceRequest>singletonList(r2), null, Collections.<ContainerId>emptyList(),
           null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
     }
@@ -4385,12 +4327,12 @@ public class TestCapacityScheduler {
     r2 = TestUtils.createResourceRequest(
         ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
     cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
-        Collections.<ContainerId>emptyList(),
+        null, Collections.<ContainerId>emptyList(),
         null, null, NULL_UPDATE_REQUESTS).getContainers().size();
     CapacityScheduler.schedule(cs);
 
     cs.allocate(appAttemptId2, Collections.<ResourceRequest>singletonList(r2),
-        Collections.<ContainerId>emptyList(), null, null, NULL_UPDATE_REQUESTS);
+        null, Collections.<ContainerId>emptyList(), null, null, NULL_UPDATE_REQUESTS);
     CapacityScheduler.schedule(cs);
     //Check blocked Resource
     assertEquals("A Used Resource should be 2 GB", 2 * GB,

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

@@ -106,7 +106,7 @@ public class TestCapacitySchedulerAsyncScheduling {
         CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD,
         numThreads);
     conf.setInt(CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_PREFIX
-        + ".scheduling-interval-ms", 100);
+        + ".scheduling-interval-ms", 0);
 
     final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
     mgr.init(conf);

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

@@ -170,7 +170,7 @@ public class TestCapacitySchedulerAutoQueueCreation
           1 * GB, 1, true, priority, recordFactory);
 
       cs.allocate(appAttemptId, Collections.<ResourceRequest>singletonList(r1),
-          Collections.<ContainerId>emptyList(), Collections.singletonList(host),
+          null, Collections.<ContainerId>emptyList(), Collections.singletonList(host),
           null, NULL_UPDATE_REQUESTS);
 
       //And this will result in container assignment for app1

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

@@ -0,0 +1,260 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.conf.Configuration;
+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.ResourceSizing;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class TestCapacitySchedulerSchedulingRequestUpdate
+    extends CapacitySchedulerTestBase {
+  @Test
+  public void testBasicPendingResourceUpdate() throws Exception {
+    Configuration conf = TestUtils.getConfigurationWithQueueLabels(
+        new Configuration(false));
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+
+    final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(
+        ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    MockRM rm = new MockRM(conf) {
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm.start();
+    MockNM nm1 = // label = x
+        new MockNM("h1:1234", 200 * GB, rm.getResourceTrackerService());
+    nm1.registerNode();
+
+    MockNM nm2 = // label = ""
+        new MockNM("h2:1234", 200 * GB, rm.getResourceTrackerService());
+    nm2.registerNode();
+
+    // Launch app1 in queue=a1
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
+
+    // Launch app2 in queue=b1
+    RMApp app2 = rm.submitApp(8 * GB, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+    // am1 asks for 8 * 1GB container for no label
+    am1.allocateIntraAppAntiAffinity(
+        ResourceSizing.newInstance(8, Resource.newInstance(1 * GB, 1)),
+        Priority.newInstance(1), 0, ImmutableSet.of("mapper", "reducer"),
+        "mapper", "reducer");
+
+    checkPendingResource(rm, "a1", 8 * GB, null);
+    checkPendingResource(rm, "a", 8 * GB, null);
+    checkPendingResource(rm, "root", 8 * GB, null);
+
+    // am2 asks for 8 * 1GB container for no label
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(1), "*",
+            Resources.createResource(1 * GB), 8)), null);
+
+    checkPendingResource(rm, "a1", 8 * GB, null);
+    checkPendingResource(rm, "a", 8 * GB, null);
+    checkPendingResource(rm, "b1", 8 * GB, null);
+    checkPendingResource(rm, "b", 8 * GB, null);
+    // root = a + b
+    checkPendingResource(rm, "root", 16 * GB, null);
+
+    // am2 asks for 8 * 1GB container in another priority for no label
+    am2.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(2), "*",
+            Resources.createResource(1 * GB), 8)), null);
+
+    checkPendingResource(rm, "a1", 8 * GB, null);
+    checkPendingResource(rm, "a", 8 * GB, null);
+    checkPendingResource(rm, "b1", 16 * GB, null);
+    checkPendingResource(rm, "b", 16 * GB, null);
+    // root = a + b
+    checkPendingResource(rm, "root", 24 * GB, null);
+
+    // am1 asks 4 GB resource instead of 8 * GB for priority=1
+    // am1 asks for 8 * 1GB container for no label
+    am1.allocateIntraAppAntiAffinity(
+        ResourceSizing.newInstance(4, Resource.newInstance(1 * GB, 1)),
+        Priority.newInstance(1), 0, ImmutableSet.of("mapper", "reducer"),
+        "mapper", "reducer");
+
+    checkPendingResource(rm, "a1", 4 * GB, null);
+    checkPendingResource(rm, "a", 4 * GB, null);
+    checkPendingResource(rm, "b1", 16 * GB, null);
+    checkPendingResource(rm, "b", 16 * GB, null);
+    // root = a + b
+    checkPendingResource(rm, "root", 20 * GB, null);
+
+    // am1 asks 8 * GB resource which label=x
+    am1.allocate(Arrays.asList(ResourceRequest
+        .newInstance(Priority.newInstance(2), "*",
+            Resources.createResource(8 * GB), 1, true, "x")), null);
+
+    checkPendingResource(rm, "a1", 4 * GB, null);
+    checkPendingResource(rm, "a", 4 * GB, null);
+    checkPendingResource(rm, "a1", 8 * GB, "x");
+    checkPendingResource(rm, "a", 8 * GB, "x");
+    checkPendingResource(rm, "b1", 16 * GB, null);
+    checkPendingResource(rm, "b", 16 * GB, null);
+    // root = a + b
+    checkPendingResource(rm, "root", 20 * GB, null);
+    checkPendingResource(rm, "root", 8 * GB, "x");
+
+    // complete am1/am2, pending resource should be 0 now
+    AppAttemptRemovedSchedulerEvent appRemovedEvent =
+        new AppAttemptRemovedSchedulerEvent(am2.getApplicationAttemptId(),
+            RMAppAttemptState.FINISHED, false);
+    rm.getResourceScheduler().handle(appRemovedEvent);
+    appRemovedEvent = new AppAttemptRemovedSchedulerEvent(
+        am1.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false);
+    rm.getResourceScheduler().handle(appRemovedEvent);
+
+    checkPendingResource(rm, "a1", 0 * GB, null);
+    checkPendingResource(rm, "a", 0 * GB, null);
+    checkPendingResource(rm, "a1", 0 * GB, "x");
+    checkPendingResource(rm, "a", 0 * GB, "x");
+    checkPendingResource(rm, "b1", 0 * GB, null);
+    checkPendingResource(rm, "b", 0 * GB, null);
+    checkPendingResource(rm, "root", 0 * GB, null);
+    checkPendingResource(rm, "root", 0 * GB, "x");
+  }
+
+  @Test
+  public void testNodePartitionPendingResourceUpdate() throws Exception {
+    Configuration conf = TestUtils.getConfigurationWithQueueLabels(
+        new Configuration(false));
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+
+    final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(
+        ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    MockRM rm = new MockRM(conf) {
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm.start();
+    MockNM nm1 = // label = x
+        new MockNM("h1:1234", 200 * GB, rm.getResourceTrackerService());
+    nm1.registerNode();
+
+    MockNM nm2 = // label = ""
+        new MockNM("h2:1234", 200 * GB, rm.getResourceTrackerService());
+    nm2.registerNode();
+
+    // Launch app1 in queue=a1
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
+
+    // Launch app2 in queue=b1
+    RMApp app2 = rm.submitApp(8 * GB, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+    // am1 asks for 8 * 1GB container for "x"
+    am1.allocateIntraAppAntiAffinity("x",
+        ResourceSizing.newInstance(8, Resource.newInstance(1 * GB, 1)),
+        Priority.newInstance(1), 0, "mapper", "reducer");
+
+    checkPendingResource(rm, "a1", 8 * GB, "x");
+    checkPendingResource(rm, "a", 8 * GB, "x");
+    checkPendingResource(rm, "root", 8 * GB, "x");
+
+    // am2 asks for 8 * 1GB container for "x"
+    am2.allocateIntraAppAntiAffinity("x",
+        ResourceSizing.newInstance(8, Resource.newInstance(1 * GB, 1)),
+        Priority.newInstance(1), 0, "mapper", "reducer");
+
+    checkPendingResource(rm, "a1", 8 * GB, "x");
+    checkPendingResource(rm, "a", 8 * GB, "x");
+    checkPendingResource(rm, "b1", 8 * GB, "x");
+    checkPendingResource(rm, "b", 8 * GB, "x");
+    // root = a + b
+    checkPendingResource(rm, "root", 16 * GB, "x");
+
+    // am1 asks for 6 * 1GB container for "x" in another priority
+    am1.allocateIntraAppAntiAffinity("x",
+        ResourceSizing.newInstance(6, Resource.newInstance(1 * GB, 1)),
+        Priority.newInstance(2), 0, "mapper", "reducer");
+
+    checkPendingResource(rm, "a1", 14 * GB, "x");
+    checkPendingResource(rm, "a", 14 * GB, "x");
+    checkPendingResource(rm, "b1", 8 * GB, "x");
+    checkPendingResource(rm, "b", 8 * GB, "x");
+    // root = a + b
+    checkPendingResource(rm, "root", 22 * GB, "x");
+
+    // am1 asks for 4 * 1GB container for "x" in priority=1, which should
+    // override 8 * 1GB
+    am1.allocateIntraAppAntiAffinity("x",
+        ResourceSizing.newInstance(4, Resource.newInstance(1 * GB, 1)),
+        Priority.newInstance(1), 0, "mapper", "reducer");
+
+    checkPendingResource(rm, "a1", 10 * GB, "x");
+    checkPendingResource(rm, "a", 10 * GB, "x");
+    checkPendingResource(rm, "b1", 8 * GB, "x");
+    checkPendingResource(rm, "b", 8 * GB, "x");
+    // root = a + b
+    checkPendingResource(rm, "root", 18 * GB, "x");
+
+    // complete am1/am2, pending resource should be 0 now
+    AppAttemptRemovedSchedulerEvent appRemovedEvent =
+        new AppAttemptRemovedSchedulerEvent(am2.getApplicationAttemptId(),
+            RMAppAttemptState.FINISHED, false);
+    rm.getResourceScheduler().handle(appRemovedEvent);
+    appRemovedEvent = new AppAttemptRemovedSchedulerEvent(
+        am1.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false);
+    rm.getResourceScheduler().handle(appRemovedEvent);
+
+    checkPendingResource(rm, "a1", 0 * GB, null);
+    checkPendingResource(rm, "a", 0 * GB, null);
+    checkPendingResource(rm, "a1", 0 * GB, "x");
+    checkPendingResource(rm, "a", 0 * GB, "x");
+    checkPendingResource(rm, "b1", 0 * GB, null);
+    checkPendingResource(rm, "b", 0 * GB, null);
+    checkPendingResource(rm, "root", 0 * GB, null);
+    checkPendingResource(rm, "root", 0 * GB, "x");
+  }
+}

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

@@ -132,7 +132,7 @@ public class TestIncreaseAllocationExpirer {
     Assert.assertEquals(RMContainerState.RUNNING,
         rm1.getResourceScheduler().getRMContainer(containerId2).getState());
     // Verify container size is 3G
-    Assert.assertEquals(
+      Assert.assertEquals(
         3 * GB, rm1.getResourceScheduler().getRMContainer(containerId2)
             .getAllocatedResource().getMemorySize());
     // Verify total resource usage

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

@@ -0,0 +1,277 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+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.rmnode.RMNode;
+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.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestSchedulingRequestContainerAllocation {
+  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 NullRMNodeLabelsManager();
+    mgr.init(conf);
+  }
+
+  @Test
+  public void testIntraAppAntiAffinity() throws Exception {
+    Configuration csConf = TestUtils.getConfigurationWithMultipleQueues(
+        new Configuration());
+    csConf.setBoolean(CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
+        true);
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+
+    // 4 NMs.
+    MockNM[] nms = new MockNM[4];
+    RMNode[] rmNodes = new RMNode[4];
+    for (int i = 0; i < 4; i++) {
+      nms[i] = rm1.registerNode("192.168.0." + i + ":1234", 10 * GB);
+      rmNodes[i] = rm1.getRMContext().getRMNodes().get(nms[i].getNodeId());
+    }
+
+    // app1 -> c
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nms[0]);
+
+    // app1 asks for 10 anti-affinity containers for the same app. It should
+    // only get 4 containers allocated because we only have 4 nodes.
+    am1.allocateIntraAppAntiAffinity(
+        ResourceSizing.newInstance(10, Resource.newInstance(1024, 1)),
+        Priority.newInstance(1), 1L, ImmutableSet.of("mapper"), "mapper");
+
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 4; j++) {
+        cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
+      }
+    }
+
+    // App1 should get 5 containers allocated (1 AM + 1 node each).
+    FiCaSchedulerApp schedulerApp = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(5, schedulerApp.getLiveContainers().size());
+
+    // Similarly, app1 asks 10 anti-affinity containers at different priority,
+    // it should be satisfied as well.
+    // app1 asks for 10 anti-affinity containers for the same app. It should
+    // only get 4 containers allocated because we only have 4 nodes.
+    am1.allocateIntraAppAntiAffinity(
+        ResourceSizing.newInstance(10, Resource.newInstance(2048, 1)),
+        Priority.newInstance(2), 1L, ImmutableSet.of("reducer"), "reducer");
+
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 4; j++) {
+        cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
+      }
+    }
+
+    // App1 should get 9 containers allocated (1 AM + 8 containers).
+    Assert.assertEquals(9, schedulerApp.getLiveContainers().size());
+
+    // Test anti-affinity to both of "mapper/reducer", we should only get no
+    // container allocated
+    am1.allocateIntraAppAntiAffinity(
+        ResourceSizing.newInstance(10, Resource.newInstance(2048, 1)),
+        Priority.newInstance(3), 1L, ImmutableSet.of("reducer2"), "mapper");
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 4; j++) {
+        cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
+      }
+    }
+
+    // App1 should get 10 containers allocated (1 AM + 9 containers).
+    Assert.assertEquals(9, schedulerApp.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test
+  public void testIntraAppAntiAffinityWithMultipleTags() throws Exception {
+    Configuration csConf = TestUtils.getConfigurationWithMultipleQueues(
+        new Configuration());
+    csConf.setBoolean(CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
+        true);
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+
+    // 4 NMs.
+    MockNM[] nms = new MockNM[4];
+    RMNode[] rmNodes = new RMNode[4];
+    for (int i = 0; i < 4; i++) {
+      nms[i] = rm1.registerNode("192.168.0." + i + ":1234", 10 * GB);
+      rmNodes[i] = rm1.getRMContext().getRMNodes().get(nms[i].getNodeId());
+    }
+
+    // app1 -> c
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nms[0]);
+
+    // app1 asks for 2 anti-affinity containers for the same app.
+    am1.allocateIntraAppAntiAffinity(
+        ResourceSizing.newInstance(2, Resource.newInstance(1024, 1)),
+        Priority.newInstance(1), 1L, ImmutableSet.of("tag_1_1", "tag_1_2"),
+        "tag_1_1", "tag_1_2");
+
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 4; j++) {
+        cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
+      }
+    }
+
+    // App1 should get 3 containers allocated (1 AM + 2 task).
+    FiCaSchedulerApp schedulerApp = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(3, schedulerApp.getLiveContainers().size());
+
+    // app1 asks for 1 anti-affinity containers for the same app. anti-affinity
+    // to tag_1_1/tag_1_2. With allocation_tag = tag_2_1/tag_2_2
+    am1.allocateIntraAppAntiAffinity(
+        ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)),
+        Priority.newInstance(2), 1L, ImmutableSet.of("tag_2_1", "tag_2_2"),
+        "tag_1_1", "tag_1_2");
+
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 4; j++) {
+        cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
+      }
+    }
+
+    // App1 should get 4 containers allocated (1 AM + 2 task (first request) +
+    // 1 task (2nd request).
+    Assert.assertEquals(4, schedulerApp.getLiveContainers().size());
+
+    // app1 asks for 10 anti-affinity containers for the same app. anti-affinity
+    // to tag_1_1/tag_1_2/tag_2_1/tag_2_2. With allocation_tag = tag_3
+    am1.allocateIntraAppAntiAffinity(
+        ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)),
+        Priority.newInstance(3), 1L, ImmutableSet.of("tag_3"),
+        "tag_1_1", "tag_1_2", "tag_2_1", "tag_2_2");
+
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 4; j++) {
+        cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
+      }
+    }
+
+    // App1 should get 1 more containers allocated
+    // 1 AM + 2 task (first request) + 1 task (2nd request) +
+    // 1 task (3rd request)
+    Assert.assertEquals(5, schedulerApp.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test
+  public void testSchedulingRequestDisabledByDefault() throws Exception {
+    Configuration csConf = TestUtils.getConfigurationWithMultipleQueues(
+        new Configuration());
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+
+    // 4 NMs.
+    MockNM[] nms = new MockNM[4];
+    RMNode[] rmNodes = new RMNode[4];
+    for (int i = 0; i < 4; i++) {
+      nms[i] = rm1.registerNode("192.168.0." + i + ":1234", 10 * GB);
+      rmNodes[i] = rm1.getRMContext().getRMNodes().get(nms[i].getNodeId());
+    }
+
+    // app1 -> c
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nms[0]);
+
+    // app1 asks for 2 anti-affinity containers for the same app.
+    boolean caughtException = false;
+    try {
+      // Since feature is disabled by default, we should expect exception.
+      am1.allocateIntraAppAntiAffinity(
+          ResourceSizing.newInstance(2, Resource.newInstance(1024, 1)),
+          Priority.newInstance(1), 1L, ImmutableSet.of("tag_1_1", "tag_1_2"),
+          "tag_1_1", "tag_1_2");
+    } catch (Exception e) {
+      caughtException = true;
+    }
+    Assert.assertTrue(caughtException);
+    rm1.close();
+  }
+}

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

@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestSchedulingRequestContainerAllocationAsync {
+  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 NullRMNodeLabelsManager();
+    mgr.init(conf);
+  }
+
+  private void testIntraAppAntiAffinityAsync(int numThreads) throws Exception {
+    Configuration csConf = TestUtils.getConfigurationWithMultipleQueues(
+        new Configuration());
+    csConf.setBoolean(CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
+        true);
+    csConf.setInt(
+        CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD,
+        numThreads);
+    csConf.setInt(CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_PREFIX
+        + ".scheduling-interval-ms", 0);
+
+    // inject node label manager
+    MockRM rm1 = new MockRM(csConf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+
+    // 200 NMs.
+    int nNMs = 200;
+    MockNM[] nms = new MockNM[nNMs];
+    RMNode[] rmNodes = new RMNode[nNMs];
+    for (int i = 0; i < nNMs; i++) {
+      nms[i] = rm1.registerNode("127.0.0." + i + ":1234", 10 * GB);
+      rmNodes[i] = rm1.getRMContext().getRMNodes().get(nms[i].getNodeId());
+    }
+
+    // app1 -> c
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nms[0]);
+
+    // app1 asks for 10 anti-affinity containers for the same app. It should
+    // only get 4 containers allocated because we only have 4 nodes.
+    am1.allocateIntraAppAntiAffinity(
+        ResourceSizing.newInstance(1000, Resource.newInstance(1024, 1)),
+        Priority.newInstance(1), 1L, ImmutableSet.of("mapper"), "mapper");
+
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < nNMs; j++) {
+        cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
+      }
+    }
+
+    // App1 should get #NM + 1 containers allocated (1 node each + 1 AM).
+    FiCaSchedulerApp schedulerApp = cs.getApplicationAttempt(
+        am1.getApplicationAttemptId());
+    Assert.assertEquals(nNMs + 1, schedulerApp.getLiveContainers().size());
+
+    rm1.close();
+  }
+
+  @Test(timeout = 300000)
+  public void testSingleThreadAsyncContainerAllocation() throws Exception {
+    testIntraAppAntiAffinityAsync(1);
+  }
+
+  @Test(timeout = 300000)
+  public void testTwoThreadsAsyncContainerAllocation() throws Exception {
+    testIntraAppAntiAffinityAsync(2);
+  }
+
+  @Test(timeout = 300000)
+  public void testThreeThreadsAsyncContainerAllocation() throws Exception {
+    testIntraAppAntiAffinityAsync(3);
+  }
+
+  @Test(timeout = 300000)
+  public void testFourThreadsAsyncContainerAllocation() throws Exception {
+    testIntraAppAntiAffinityAsync(4);
+  }
+
+  @Test(timeout = 300000)
+  public void testFiveThreadsAsyncContainerAllocation() throws Exception {
+    testIntraAppAntiAffinityAsync(5);
+  }
+}

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

@@ -275,6 +275,8 @@ public class TestUtils {
   public static Configuration getConfigurationWithQueueLabels(Configuration config) {
     CapacitySchedulerConfiguration conf =
         new CapacitySchedulerConfiguration(config);
+    conf.setBoolean(CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
+        true);
     
     // Define top-level queues
     conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});

+ 15 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsManager.java

@@ -20,10 +20,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint;
 
-import java.util.List;
-
+import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -33,7 +33,7 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableSet;
+import java.util.List;
 
 /**
  * Test functionality of AllocationTagsManager.
@@ -54,7 +54,6 @@ public class TestAllocationTagsManager {
     rmContext = rm.getRMContext();
   }
 
-
   @Test
   public void testAllocationTagsManagerSimpleCases()
       throws InvalidAllocationTagsQueryException {
@@ -141,30 +140,31 @@ public class TestAllocationTagsManager {
 
     // Get Node Cardinality of app1 on node2, with tag "<applicationId>", op=max
     // (Expect this returns #containers from app1 on node2)
-    Assert
-        .assertEquals(2,
-            atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
-                TestUtils.getMockApplicationId(1),
-                ImmutableSet.of(AllocationTagsNamespaces.APP_ID
-                    + TestUtils.getMockApplicationId(1).toString()),
-                Long::max));
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
+            TestUtils.getMockApplicationId(1), null, Long::max));
 
     // Get Node Cardinality of app1 on node2, with empty tag set, op=max
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
+            TestUtils.getMockApplicationId(1), null, Long::max));
+
+    // Get Cardinality of app1 on node2, with empty tag set, op=max
     Assert.assertEquals(2,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::max));
 
     // Get Node Cardinality of all apps on node2, with empty tag set, op=sum
-    Assert.assertEquals(7, atm.getNodeCardinalityByOp(
+    Assert.assertEquals(4, atm.getNodeCardinalityByOp(
         NodeId.fromString("host2:123"), null, ImmutableSet.of(), Long::sum));
 
     // Get Node Cardinality of app_1 on node2, with empty tag set, op=sum
-    Assert.assertEquals(5,
+    Assert.assertEquals(3,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::sum));
 
     // Get Node Cardinality of app_1 on node2, with empty tag set, op=sum
-    Assert.assertEquals(2,
+    Assert.assertEquals(1,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             TestUtils.getMockApplicationId(2), ImmutableSet.of(), Long::sum));
 
@@ -296,7 +296,7 @@ public class TestAllocationTagsManager {
     Assert.assertEquals(3, atm.getRackCardinality("rack0", null, "reducer"));
 
     // Get Rack Cardinality of app_1 on rack0, with empty tag set, op=max
-    Assert.assertEquals(2, atm.getRackCardinalityByOp("rack0",
+    Assert.assertEquals(1, atm.getRackCardinalityByOp("rack0",
         TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::max));
 
     // Get Rack Cardinality of app_1 on rack0, with empty tag set, op=min

+ 18 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintsUtil.java

@@ -117,9 +117,9 @@ public class TestPlacementConstraintsUtil {
       RMNode currentNode = nodeIterator.next();
       FiCaSchedulerNode schedulerNode = TestUtils.getMockNode(
           currentNode.getHostName(), currentNode.getRackName(), 123, 4 * GB);
-      Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+      Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
           sourceTag1, schedulerNode, pcm, tm));
-      Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+      Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
           sourceTag2, schedulerNode, pcm, tm));
     }
     /**
@@ -145,14 +145,14 @@ public class TestPlacementConstraintsUtil {
     tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m"));
 
     // 'spark' placement on Node0 should now SUCCEED
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag1, schedulerNode0, pcm, tm));
     // FAIL on the rest of the nodes
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag1, schedulerNode1, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag1, schedulerNode2, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag1, schedulerNode3, pcm, tm));
   }
 
@@ -187,15 +187,15 @@ public class TestPlacementConstraintsUtil {
     FiCaSchedulerNode schedulerNode3 = TestUtils
         .getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
     // 'zk' placement on Rack1 should now SUCCEED
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag2, schedulerNode0, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag2, schedulerNode1, pcm, tm));
 
     // FAIL on the rest of the RACKs
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag2, schedulerNode2, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag2, schedulerNode3, pcm, tm));
   }
 
@@ -230,14 +230,14 @@ public class TestPlacementConstraintsUtil {
     tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m"));
 
     // 'spark' placement on Node0 should now FAIL
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag1, schedulerNode0, pcm, tm));
     // SUCCEED on the rest of the nodes
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag1, schedulerNode1, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag1, schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag1, schedulerNode3, pcm, tm));
   }
 
@@ -273,15 +273,15 @@ public class TestPlacementConstraintsUtil {
         .getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
 
     // 'zk' placement on Rack1 should FAIL
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag2, schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag2, schedulerNode1, pcm, tm));
 
     // SUCCEED on the rest of the RACKs
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag2, schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
         sourceTag2, schedulerNode3, pcm, tm));
   }
 }

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

@@ -192,7 +192,7 @@ public class FairSchedulerTestBase {
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
 
-    scheduler.allocate(id, ask, new ArrayList<ContainerId>(),
+    scheduler.allocate(id, ask, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
     scheduler.update();
     return id;
@@ -222,7 +222,7 @@ public class FairSchedulerTestBase {
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
 
-    scheduler.allocate(id, ask, new ArrayList<ContainerId>(),
+    scheduler.allocate(id, ask, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
     return id;
   }
@@ -245,7 +245,7 @@ public class FairSchedulerTestBase {
       ResourceRequest request, ApplicationAttemptId attId) {
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     ask.add(request);
-    scheduler.allocate(attId, ask,  new ArrayList<ContainerId>(),
+    scheduler.allocate(attId, ask, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
     scheduler.update();
   }

+ 4 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java

@@ -125,7 +125,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     List<ResourceRequest> ask = new ArrayList<>();
     ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true));
     scheduler.allocate(
-        appAttemptId, ask, new ArrayList<ContainerId>(),
+        appAttemptId, ask, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
     FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
 
@@ -163,8 +163,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     ResourceRequest request =
         createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
     ask.add(request);
-    scheduler.allocate(appAttemptId, ask,
-        new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
+    scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
     triggerSchedulingAttempt();
 
     FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
@@ -175,8 +174,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
         createResourceRequest(1024, 1, ResourceRequest.ANY, 2, 1, true);
     ask.clear();
     ask.add(request);
-    scheduler.allocate(appAttemptId, ask,
-        new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
+    scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
     triggerSchedulingAttempt();
 
     checkAppConsumption(app, Resources.createResource(2048,2));
@@ -373,7 +371,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
         true);
     ask1.add(request1);
     ask1.add(request2);
-    scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null,
+    scheduler.allocate(id11, ask1, null, new ArrayList<ContainerId>(), null, null,
         NULL_UPDATE_REQUESTS);
 
     NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);

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

@@ -1280,7 +1280,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     List<ResourceRequest> asks = new ArrayList<ResourceRequest>();
     asks.add(createResourceRequest(2048, node2.getRackName(), 1, 1, false));
 
-    scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
+    scheduler.allocate(attemptId, asks, null, new ArrayList<ContainerId>(), null,
             null, NULL_UPDATE_REQUESTS);
 
     ApplicationAttemptId attId = createSchedulingRequest(2048, "queue1", "user1", 1);
@@ -2125,7 +2125,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     ResourceRequest request1 = createResourceRequest(minReqSize * 2,
         ResourceRequest.ANY, 1, 1, true);
     ask1.add(request1);
-    scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(),
+    scheduler.allocate(id11, ask1, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     // Second ask, queue2 requests 1 large.
@@ -2141,7 +2141,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         ResourceRequest.ANY, 1, 1, false);
     ask2.add(request2);
     ask2.add(request3);
-    scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(),
+    scheduler.allocate(id21, ask2, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     // Third ask, queue2 requests 2 small (minReqSize).
@@ -2157,7 +2157,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         ResourceRequest.ANY, 2, 2, true);
     ask3.add(request4);
     ask3.add(request5);
-    scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(),
+    scheduler.allocate(id22, ask3, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     scheduler.update();
@@ -2683,7 +2683,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     // Complete the first container so we can trigger allocation for app2
     ContainerId containerId =
         app1.getLiveContainers().iterator().next().getContainerId();
-    scheduler.allocate(app1.getApplicationAttemptId(), new ArrayList<>(),
+    scheduler.allocate(app1.getApplicationAttemptId(), new ArrayList<>(), null,
         Arrays.asList(containerId), null, null, NULL_UPDATE_REQUESTS);
 
     // Trigger allocation for app2
@@ -2769,7 +2769,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     asks.add(createResourceRequest(1024, node3.getRackName(), 1, 1, true));
     asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true));
 
-    scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
+    scheduler.allocate(attemptId, asks, null, new ArrayList<ContainerId>(), null,
         null, NULL_UPDATE_REQUESTS);
     
     // node 1 checks in
@@ -3216,7 +3216,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         createResourceRequest(1024, node1.getHostName(), 1, 0, true),
         createResourceRequest(1024, "rack1", 1, 0, true),
         createResourceRequest(1024, ResourceRequest.ANY, 1, 1, true));
-    scheduler.allocate(attId1, update, new ArrayList<ContainerId>(),
+    scheduler.allocate(attId1, update, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
     
     // then node2 should get the container
@@ -4432,7 +4432,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         createResourceRequest(1024, 8, ResourceRequest.ANY, 1, 1, true);
 
     ask1.add(request1);
-    scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null,
+    scheduler.allocate(id11, ask1, null, new ArrayList<ContainerId>(), null,
         null, NULL_UPDATE_REQUESTS);
 
     String hostName = "127.0.0.1";
@@ -4508,11 +4508,11 @@ public class TestFairScheduler extends FairSchedulerTestBase {
 
     // Verify the blacklist can be updated independent of requesting containers
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
-        Collections.<ContainerId>emptyList(),
+        null, Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
     assertTrue(app.isPlaceBlacklisted(host));
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
-        Collections.<ContainerId>emptyList(), null,
+        null, Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), NULL_UPDATE_REQUESTS);
     assertFalse(scheduler.getSchedulerApp(appAttemptId)
         .isPlaceBlacklisted(host));
@@ -4521,8 +4521,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         createResourceRequest(GB, node.getHostName(), 1, 0, true));
 
     // Verify a container does not actually get placed on the blacklisted host
-    scheduler.allocate(appAttemptId, update,
-        Collections.<ContainerId>emptyList(),
+    scheduler.allocate(appAttemptId, update, null, Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
     assertTrue(app.isPlaceBlacklisted(host));
     scheduler.update();
@@ -4531,8 +4530,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         .getLiveContainers().size());
 
     // Verify a container gets placed on the empty blacklist
-    scheduler.allocate(appAttemptId, update,
-        Collections.<ContainerId>emptyList(), null,
+    scheduler.allocate(appAttemptId, update, null, Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), NULL_UPDATE_REQUESTS);
     assertFalse(app.isPlaceBlacklisted(host));
     createSchedulingRequest(GB, "root.default", "user", 1);
@@ -5391,8 +5389,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     ask1.add(request3);
 
     // Perform allocation
-    scheduler.allocate(appAttemptId, ask1, new ArrayList<ContainerId>(), null,
-        null, NULL_UPDATE_REQUESTS);
+    scheduler.allocate(appAttemptId, ask1, null, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
     scheduler.update();
     scheduler.handle(new NodeUpdateSchedulerEvent(node));
 

+ 14 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

@@ -281,7 +281,7 @@ public class TestFifoScheduler {
     ask.add(nodeLocal);
     ask.add(rackLocal);
     ask.add(any);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(),
+    scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
@@ -378,7 +378,7 @@ public class TestFifoScheduler {
     ask.add(nodeLocal);
     ask.add(rackLocal);
     ask.add(any);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(),
+    scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     // Before the node update event, there are one local request
@@ -954,7 +954,7 @@ public class TestFifoScheduler {
     ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1,
         RMNodeLabelsManager.NO_LABEL));
-    fs.allocate(appAttemptId1, ask1, emptyId,
+    fs.allocate(appAttemptId1, ask1, null, emptyId,
         Collections.singletonList(host_1_0), null, NULL_UPDATE_REQUESTS);
 
     // Trigger container assignment
@@ -963,7 +963,7 @@ public class TestFifoScheduler {
     // Get the allocation for the application and verify no allocation on
     // blacklist node
     Allocation allocation1 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+        fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
 
     Assert.assertEquals("allocation1", 0, allocation1.getContainers().size());
@@ -971,7 +971,7 @@ public class TestFifoScheduler {
     // verify host_1_1 can get allocated as not in blacklist
     fs.handle(new NodeUpdateSchedulerEvent(n4));
     Allocation allocation2 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+        fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation2", 1, allocation2.getContainers().size());
     List<Container> containerList = allocation2.getContainers();
@@ -986,33 +986,33 @@ public class TestFifoScheduler {
     // be assigned
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
-    fs.allocate(appAttemptId1, ask2, emptyId,
+    fs.allocate(appAttemptId1, ask2, null, emptyId,
         Collections.singletonList("rack0"), null, NULL_UPDATE_REQUESTS);
 
     // verify n1 is not qualified to be allocated
     fs.handle(new NodeUpdateSchedulerEvent(n1));
     Allocation allocation3 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+        fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation3", 0, allocation3.getContainers().size());
 
     // verify n2 is not qualified to be allocated
     fs.handle(new NodeUpdateSchedulerEvent(n2));
     Allocation allocation4 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+        fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation4", 0, allocation4.getContainers().size());
 
     // verify n3 is not qualified to be allocated
     fs.handle(new NodeUpdateSchedulerEvent(n3));
     Allocation allocation5 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+        fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation5", 0, allocation5.getContainers().size());
 
     fs.handle(new NodeUpdateSchedulerEvent(n4));
     Allocation allocation6 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+        fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation6", 1, allocation6.getContainers().size());
 
@@ -1072,14 +1072,14 @@ public class TestFifoScheduler {
     List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
     ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
-    fs.allocate(appAttemptId1, ask1, emptyId,
+    fs.allocate(appAttemptId1, ask1, null, emptyId,
         null, null, NULL_UPDATE_REQUESTS);
 
     // Ask for a 2 GB container for app 2
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(2 * GB, 1), 1));
-    fs.allocate(appAttemptId2, ask2, emptyId,
+    fs.allocate(appAttemptId2, ask2, null, emptyId,
         null, null, NULL_UPDATE_REQUESTS);
 
     // Trigger container assignment
@@ -1087,13 +1087,13 @@ public class TestFifoScheduler {
 
     // Get the allocation for the applications and verify headroom
     Allocation allocation1 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+        fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("Allocation headroom", 1 * GB, allocation1
         .getResourceLimit().getMemorySize());
 
     Allocation allocation2 =
-        fs.allocate(appAttemptId2, emptyAsk, emptyId,
+        fs.allocate(appAttemptId2, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("Allocation headroom", 1 * GB, allocation2
         .getResourceLimit().getMemorySize());

+ 403 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java

@@ -0,0 +1,403 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+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.ResourceSizing;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
+import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.InvalidAllocationTagsQueryException;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.function.LongBinaryOperator;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test behaviors of single constraint app placement allocator.
+ */
+public class TestSingleConstraintAppPlacementAllocator {
+  private AppSchedulingInfo appSchedulingInfo;
+  private AllocationTagsManager spyAllocationTagsManager;
+  private RMContext rmContext;
+  private SchedulerRequestKey schedulerRequestKey;
+  private SingleConstraintAppPlacementAllocator allocator;
+
+  @Before
+  public void setup() throws Exception {
+    // stub app scheduling info.
+    appSchedulingInfo = mock(AppSchedulingInfo.class);
+    when(appSchedulingInfo.getApplicationId()).thenReturn(
+        TestUtils.getMockApplicationId(1));
+    when(appSchedulingInfo.getApplicationAttemptId()).thenReturn(
+        TestUtils.getMockApplicationAttemptId(1, 1));
+
+    // stub RMContext
+    rmContext = TestUtils.getMockRMContext();
+
+    // Create allocation tags manager
+    AllocationTagsManager allocationTagsManager = new AllocationTagsManager(
+        rmContext);
+    spyAllocationTagsManager = spy(allocationTagsManager);
+    schedulerRequestKey = new SchedulerRequestKey(Priority.newInstance(1), 2L,
+        TestUtils.getMockContainerId(1, 1));
+    rmContext.setAllocationTagsManager(spyAllocationTagsManager);
+
+    // Create allocator
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+  }
+
+  private void assertValidSchedulingRequest(
+      SchedulingRequest schedulingRequest) {
+    // Create allocator to avoid fields polluted by previous runs
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
+  }
+
+  private void assertInvalidSchedulingRequest(
+      SchedulingRequest schedulingRequest, boolean recreateAllocator) {
+    try {
+      // Create allocator
+      if (recreateAllocator) {
+        allocator = new SingleConstraintAppPlacementAllocator();
+        allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+      }
+      allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
+    } catch (SchedulerInvalidResoureRequestException e) {
+      // Expected
+      return;
+    }
+    Assert.fail(
+        "Expect failure for schedulingRequest=" + schedulingRequest.toString());
+  }
+
+  @Test
+  public void testSchedulingRequestValidation() {
+    // Valid
+    assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper", "reducer"),
+                PlacementConstraints.PlacementTargets.nodePartition(""))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build());
+    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
+        allocator.getTargetAllocationTags());
+    Assert.assertEquals("", allocator.getTargetNodePartition());
+
+    // Valid (with partition)
+    assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper", "reducer"),
+                PlacementConstraints.PlacementTargets.nodePartition("x"))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build());
+    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
+        allocator.getTargetAllocationTags());
+    Assert.assertEquals("x", allocator.getTargetNodePartition());
+
+    // Valid (without specifying node partition)
+    assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper", "reducer")).build())
+        .resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build());
+    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
+        allocator.getTargetAllocationTags());
+    Assert.assertEquals("", allocator.getTargetNodePartition());
+
+    // Valid (with application Id target)
+    assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper", "reducer")).build())
+        .resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build());
+    // Allocation tags should not include application Id
+    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
+        allocator.getTargetAllocationTags());
+    Assert.assertEquals("", allocator.getTargetNodePartition());
+
+    // Invalid (without sizing)
+    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper", "reducer")).build())
+        .build(), true);
+
+    // Invalid (without target tags)
+    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1).build())
+        .build(), true);
+
+    // Invalid (with multiple allocation tags expression specified)
+    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper"),
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("reducer"),
+                PlacementConstraints.PlacementTargets.nodePartition(""))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build(), true);
+
+    // Invalid (with multiple node partition target expression specified)
+    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper"),
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp(""),
+                PlacementConstraints.PlacementTargets.nodePartition("x"))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build(), true);
+
+    // Invalid (not anti-affinity cardinality)
+    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 1, 2,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper"),
+                PlacementConstraints.PlacementTargets.nodePartition(""))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build(), true);
+
+    // Invalid (not anti-affinity cardinality)
+    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 2,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper"),
+                PlacementConstraints.PlacementTargets.nodePartition(""))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build(), true);
+
+    // Invalid (not NODE scope)
+    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.RACK, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper", "reducer"),
+                PlacementConstraints.PlacementTargets.nodePartition(""))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build(), true);
+
+    // Invalid (not GUARANTEED)
+    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.OPPORTUNISTIC))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper", "reducer"),
+                PlacementConstraints.PlacementTargets.nodePartition(""))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build(), true);
+  }
+
+  @Test
+  public void testSchedulingRequestUpdate() {
+    SchedulingRequest schedulingRequest =
+        SchedulingRequest.newBuilder().executionType(
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+            .allocationRequestId(10L).priority(Priority.newInstance(1))
+            .placementConstraintExpression(PlacementConstraints
+                .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                    PlacementConstraints.PlacementTargets
+                        .allocationTagToIntraApp("mapper", "reducer"),
+                    PlacementConstraints.PlacementTargets.nodePartition(""))
+                .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+            .build();
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
+
+    // Update allocator with exactly same scheduling request, should succeeded.
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
+
+    // Update allocator with scheduling request different at #allocations,
+    // should succeeded.
+    schedulingRequest.getResourceSizing().setNumAllocations(10);
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
+
+    // Update allocator with scheduling request different at resource,
+    // should failed.
+    schedulingRequest.getResourceSizing().setResources(
+        Resource.newInstance(2048, 1));
+    assertInvalidSchedulingRequest(schedulingRequest, false);
+
+    // Update allocator with a different placement target (allocator tag),
+    // should failed
+    schedulingRequest = SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper"),
+                PlacementConstraints.PlacementTargets.nodePartition(""))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build();
+    assertInvalidSchedulingRequest(schedulingRequest, false);
+
+    // Update allocator with recover == true
+    int existingNumAllocations =
+        allocator.getSchedulingRequest().getResourceSizing()
+            .getNumAllocations();
+    schedulingRequest = SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper", "reducer"),
+                PlacementConstraints.PlacementTargets.nodePartition(""))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build();
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, true);
+    Assert.assertEquals(existingNumAllocations + 1,
+        allocator.getSchedulingRequest().getResourceSizing()
+            .getNumAllocations());
+  }
+
+  @Test
+  public void testFunctionality() throws InvalidAllocationTagsQueryException {
+    SchedulingRequest schedulingRequest =
+        SchedulingRequest.newBuilder().executionType(
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+            .allocationRequestId(10L).priority(Priority.newInstance(1))
+            .placementConstraintExpression(PlacementConstraints
+                .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                    PlacementConstraints.PlacementTargets
+                        .allocationTagToIntraApp("mapper", "reducer"),
+                    PlacementConstraints.PlacementTargets.nodePartition(""))
+                .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+            .build();
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
+    allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNode("host1", "/rack1", 123, 1024));
+    verify(spyAllocationTagsManager, Mockito.times(1)).getNodeCardinalityByOp(
+        eq(NodeId.fromString("host1:123")), eq(TestUtils.getMockApplicationId(1)),
+        eq(ImmutableSet.of("mapper", "reducer")),
+        any(LongBinaryOperator.class));
+
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    // Valid (with partition)
+    schedulingRequest = SchedulingRequest.newBuilder().executionType(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+        .allocationRequestId(10L).priority(Priority.newInstance(1))
+        .placementConstraintExpression(PlacementConstraints
+            .targetCardinality(PlacementConstraints.NODE, 0, 1,
+                PlacementConstraints.PlacementTargets
+                    .allocationTagToIntraApp("mapper", "reducer"),
+                PlacementConstraints.PlacementTargets.nodePartition("x"))
+            .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+        .build();
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
+    allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNode("host1", "/rack1", 123, 1024));
+    verify(spyAllocationTagsManager, Mockito.atLeast(1)).getNodeCardinalityByOp(
+        eq(NodeId.fromString("host1:123")),
+        eq(TestUtils.getMockApplicationId(1)), eq(ImmutableSet
+            .of("mapper", "reducer")), any(LongBinaryOperator.class));
+
+    SchedulerNode node1 = mock(SchedulerNode.class);
+    when(node1.getPartition()).thenReturn("x");
+    when(node1.getNodeID()).thenReturn(NodeId.fromString("host1:123"));
+
+    Assert.assertTrue(allocator
+        .precheckNode(node1, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY));
+
+    SchedulerNode node2 = mock(SchedulerNode.class);
+    when(node1.getPartition()).thenReturn("");
+    when(node1.getNodeID()).thenReturn(NodeId.fromString("host2:123"));
+    Assert.assertFalse(allocator
+        .precheckNode(node2, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY));
+  }
+}