Browse Source

YARN-5392. Replace use of Priority in the Scheduling infrastructure with an opaque ShedulerRequestKey. (asuresh and subru)

Arun Suresh 8 years ago
parent
commit
5aace38b74
32 changed files with 925 additions and 623 deletions
  1. 4 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java
  2. 1 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java
  3. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
  4. 13 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
  5. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerReservedEvent.java
  6. 86 74
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
  7. 90 70
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
  8. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
  9. 99 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerRequestKey.java
  10. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  11. 15 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java
  12. 97 84
      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
  13. 30 26
      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
  14. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
  15. 119 94
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
  16. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
  17. 1 1
      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
  18. 46 41
      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
  19. 66 54
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
  20. 10 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Task.java
  21. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
  22. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
  23. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
  24. 19 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java
  25. 6 4
      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
  26. 98 72
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
  27. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
  28. 36 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
  29. 12 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
  30. 12 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
  31. 11 4
      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
  32. 15 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java

+ 4 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java

@@ -23,7 +23,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 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.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -343,12 +342,10 @@ public class FifoCandidatesSelector
     Collections.sort(containers, new Comparator<RMContainer>() {
     Collections.sort(containers, new Comparator<RMContainer>() {
       @Override
       @Override
       public int compare(RMContainer a, RMContainer b) {
       public int compare(RMContainer a, RMContainer b) {
-        Comparator<Priority> c = new org.apache.hadoop.yarn.server
-            .resourcemanager.resource.Priority.Comparator();
-        int priorityComp = c.compare(b.getContainer().getPriority(),
-            a.getContainer().getPriority());
-        if (priorityComp != 0) {
-          return priorityComp;
+        int schedKeyComp = b.getAllocatedSchedulerKey()
+                .compareTo(a.getAllocatedSchedulerKey());
+        if (schedKeyComp != 0) {
+          return schedKeyComp;
         }
         }
         return b.getContainerId().compareTo(a.getContainerId());
         return b.getContainerId().compareTo(a.getContainerId());
       }
       }

+ 1 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java

@@ -27,13 +27,5 @@ public class Priority {
     priority.setPriority(prio);
     priority.setPriority(prio);
     return priority;
     return priority;
   }
   }
-  
-  public static class Comparator 
-  implements java.util.Comparator<org.apache.hadoop.yarn.api.records.Priority> {
-    @Override
-    public int compare(org.apache.hadoop.yarn.api.records.Priority o1, org.apache.hadoop.yarn.api.records.Priority o2) {
-      return o1.getPriority() - o2.getPriority();
-    }
-  }
-  
+
 }
 }

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java

@@ -31,7 +31,8 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
+
 
 
 /**
 /**
  * Represents the ResourceManager's view of an application container. See 
  * Represents the ResourceManager's view of an application container. See 
@@ -55,7 +56,7 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
 
 
   NodeId getReservedNode();
   NodeId getReservedNode();
   
   
-  Priority getReservedPriority();
+  SchedulerRequestKey getReservedSchedulerKey();
 
 
   Resource getAllocatedResource();
   Resource getAllocatedResource();
 
 
@@ -63,6 +64,8 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
 
 
   NodeId getAllocatedNode();
   NodeId getAllocatedNode();
 
 
+  SchedulerRequestKey getAllocatedSchedulerKey();
+
   Priority getAllocatedPriority();
   Priority getAllocatedPriority();
 
 
   long getCreationTime();
   long getCreationTime();

+ 13 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java

@@ -53,12 +53,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode
     .RMNodeDecreaseContainerEvent;
     .RMNodeDecreaseContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 
@@ -173,7 +173,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
 
 
   private Resource reservedResource;
   private Resource reservedResource;
   private NodeId reservedNode;
   private NodeId reservedNode;
-  private Priority reservedPriority;
+  private SchedulerRequestKey reservedSchedulerKey;
   private long creationTime;
   private long creationTime;
   private long finishTime;
   private long finishTime;
   private ContainerStatus finishedStatus;
   private ContainerStatus finishedStatus;
@@ -187,6 +187,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
   private volatile String queueName;
   private volatile String queueName;
 
 
   private boolean isExternallyAllocated;
   private boolean isExternallyAllocated;
+  private SchedulerRequestKey allocatedSchedulerKey;
 
 
   public RMContainerImpl(Container container,
   public RMContainerImpl(Container container,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
@@ -226,6 +227,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     this.containerId = container.getId();
     this.containerId = container.getId();
     this.nodeId = nodeId;
     this.nodeId = nodeId;
     this.container = container;
     this.container = container;
+    this.allocatedSchedulerKey = SchedulerRequestKey.extractFrom(container);
     this.appAttemptId = appAttemptId;
     this.appAttemptId = appAttemptId;
     this.user = user;
     this.user = user;
     this.creationTime = creationTime;
     this.creationTime = creationTime;
@@ -296,8 +298,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
   }
   }
 
 
   @Override
   @Override
-  public Priority getReservedPriority() {
-    return reservedPriority;
+  public SchedulerRequestKey getReservedSchedulerKey() {
+    return reservedSchedulerKey;
   }
   }
 
 
   @Override
   @Override
@@ -325,6 +327,11 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     return container.getNodeId();
     return container.getNodeId();
   }
   }
 
 
+  @Override
+  public SchedulerRequestKey getAllocatedSchedulerKey() {
+    return allocatedSchedulerKey;
+  }
+
   @Override
   @Override
   public Priority getAllocatedPriority() {
   public Priority getAllocatedPriority() {
     return container.getPriority();
     return container.getPriority();
@@ -535,7 +542,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
       RMContainerReservedEvent e = (RMContainerReservedEvent)event;
       RMContainerReservedEvent e = (RMContainerReservedEvent)event;
       container.reservedResource = e.getReservedResource();
       container.reservedResource = e.getReservedResource();
       container.reservedNode = e.getReservedNode();
       container.reservedNode = e.getReservedNode();
-      container.reservedPriority = e.getReservedPriority();
+      container.reservedSchedulerKey = e.getReservedSchedulerKey();
       
       
       if (!EnumSet.of(RMContainerState.NEW, RMContainerState.RESERVED)
       if (!EnumSet.of(RMContainerState.NEW, RMContainerState.RESERVED)
           .contains(container.getState())) {
           .contains(container.getState())) {
@@ -768,7 +775,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     try {
     try {
       containerReport = ContainerReport.newInstance(this.getContainerId(),
       containerReport = ContainerReport.newInstance(this.getContainerId(),
           this.getAllocatedResource(), this.getAllocatedNode(),
           this.getAllocatedResource(), this.getAllocatedNode(),
-          this.getAllocatedPriority(), this.getCreationTime(),
+          this.getAllocatedSchedulerKey().getPriority(), this.getCreationTime(),
           this.getFinishTime(), this.getDiagnosticsInfo(), this.getLogURL(),
           this.getFinishTime(), this.getDiagnosticsInfo(), this.getLogURL(),
           this.getContainerExitStatus(), this.getContainerState(),
           this.getContainerExitStatus(), this.getContainerState(),
           this.getNodeHttpAddress());
           this.getNodeHttpAddress());

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerReservedEvent.java

@@ -20,8 +20,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
 
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 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.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 
 
 /**
 /**
  * The event signifying that a container has been reserved.
  * The event signifying that a container has been reserved.
@@ -33,15 +33,15 @@ public class RMContainerReservedEvent extends RMContainerEvent {
 
 
   private final Resource reservedResource;
   private final Resource reservedResource;
   private final NodeId reservedNode;
   private final NodeId reservedNode;
-  private final Priority reservedPriority;
+  private final SchedulerRequestKey reservedSchedulerKey;
   
   
   public RMContainerReservedEvent(ContainerId containerId,
   public RMContainerReservedEvent(ContainerId containerId,
       Resource reservedResource, NodeId reservedNode, 
       Resource reservedResource, NodeId reservedNode, 
-      Priority reservedPriority) {
+      SchedulerRequestKey reservedSchedulerKey) {
     super(containerId, RMContainerEventType.RESERVED);
     super(containerId, RMContainerEventType.RESERVED);
     this.reservedResource = reservedResource;
     this.reservedResource = reservedResource;
     this.reservedNode = reservedNode;
     this.reservedNode = reservedNode;
-    this.reservedPriority = reservedPriority;
+    this.reservedSchedulerKey = reservedSchedulerKey;
   }
   }
 
 
   public Resource getReservedResource() {
   public Resource getReservedResource() {
@@ -52,8 +52,8 @@ public class RMContainerReservedEvent extends RMContainerEvent {
     return reservedNode;
     return reservedNode;
   }
   }
 
 
-  public Priority getReservedPriority() {
-    return reservedPriority;
+  public SchedulerRequestKey getReservedSchedulerKey() {
+    return reservedSchedulerKey;
   }
   }
 
 
 }
 }

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

@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
@@ -41,7 +40,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 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.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -60,8 +58,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 public class AppSchedulingInfo {
 public class AppSchedulingInfo {
   
   
   private static final Log LOG = LogFactory.getLog(AppSchedulingInfo.class);
   private static final Log LOG = LogFactory.getLog(AppSchedulingInfo.class);
-  private static final Comparator<Priority> COMPARATOR =
-      new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator();
   private static final int EPOCH_BIT_SHIFT = 40;
   private static final int EPOCH_BIT_SHIFT = 40;
 
 
   private final ApplicationId applicationId;
   private final ApplicationId applicationId;
@@ -82,10 +78,10 @@ public class AppSchedulingInfo {
 
 
   private Set<String> requestedPartitions = new HashSet<>();
   private Set<String> requestedPartitions = new HashSet<>();
 
 
-  final Set<Priority> priorities = new TreeSet<>(COMPARATOR);
-  final Map<Priority, Map<String, ResourceRequest>> resourceRequestMap =
-      new ConcurrentHashMap<>();
-  final Map<NodeId, Map<Priority, Map<ContainerId,
+  final Set<SchedulerRequestKey> schedulerKeys = new TreeSet<>();
+  final Map<SchedulerRequestKey, Map<String, ResourceRequest>>
+      resourceRequestMap = new ConcurrentHashMap<>();
+  final Map<NodeId, Map<SchedulerRequestKey, Map<ContainerId,
       SchedContainerChangeRequest>>> containerIncreaseRequestMap =
       SchedContainerChangeRequest>>> containerIncreaseRequestMap =
       new ConcurrentHashMap<>();
       new ConcurrentHashMap<>();
 
 
@@ -134,22 +130,23 @@ public class AppSchedulingInfo {
    * Clear any pending requests from this application.
    * Clear any pending requests from this application.
    */
    */
   private synchronized void clearRequests() {
   private synchronized void clearRequests() {
-    priorities.clear();
+    schedulerKeys.clear();
     resourceRequestMap.clear();
     resourceRequestMap.clear();
     LOG.info("Application " + applicationId + " requests cleared");
     LOG.info("Application " + applicationId + " requests cleared");
   }
   }
 
 
   public synchronized boolean hasIncreaseRequest(NodeId nodeId) {
   public synchronized boolean hasIncreaseRequest(NodeId nodeId) {
-    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
-        containerIncreaseRequestMap.get(nodeId);
+    Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
+        requestsOnNode = containerIncreaseRequestMap.get(nodeId);
     return requestsOnNode == null ? false : requestsOnNode.size() > 0;
     return requestsOnNode == null ? false : requestsOnNode.size() > 0;
   }
   }
-  
+
   public synchronized Map<ContainerId, SchedContainerChangeRequest>
   public synchronized Map<ContainerId, SchedContainerChangeRequest>
-      getIncreaseRequests(NodeId nodeId, Priority priority) {
-    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
-        containerIncreaseRequestMap.get(nodeId);
-    return requestsOnNode == null ? null : requestsOnNode.get(priority);
+      getIncreaseRequests(NodeId nodeId, SchedulerRequestKey schedulerKey) {
+    Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
+        requestsOnNode = containerIncreaseRequestMap.get(nodeId);
+    return requestsOnNode == null ? null : requestsOnNode.get(
+        schedulerKey);
   }
   }
 
 
   /**
   /**
@@ -175,15 +172,17 @@ public class AppSchedulingInfo {
       }
       }
       NodeId nodeId = r.getRMContainer().getAllocatedNode();
       NodeId nodeId = r.getRMContainer().getAllocatedNode();
 
 
-      Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
-          containerIncreaseRequestMap.get(nodeId);
+      Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
+          requestsOnNode = containerIncreaseRequestMap.get(nodeId);
       if (null == requestsOnNode) {
       if (null == requestsOnNode) {
         requestsOnNode = new TreeMap<>();
         requestsOnNode = new TreeMap<>();
         containerIncreaseRequestMap.put(nodeId, requestsOnNode);
         containerIncreaseRequestMap.put(nodeId, requestsOnNode);
       }
       }
 
 
       SchedContainerChangeRequest prevChangeRequest =
       SchedContainerChangeRequest prevChangeRequest =
-          getIncreaseRequest(nodeId, r.getPriority(), r.getContainerId());
+          getIncreaseRequest(nodeId,
+              r.getRMContainer().getAllocatedSchedulerKey(),
+              r.getContainerId());
       if (null != prevChangeRequest) {
       if (null != prevChangeRequest) {
         if (Resources.equals(prevChangeRequest.getTargetCapacity(),
         if (Resources.equals(prevChangeRequest.getTargetCapacity(),
             r.getTargetCapacity())) {
             r.getTargetCapacity())) {
@@ -192,7 +191,8 @@ public class AppSchedulingInfo {
         }
         }
 
 
         // remove the old one, as we will use the new one going forward
         // remove the old one, as we will use the new one going forward
-        removeIncreaseRequest(nodeId, prevChangeRequest.getPriority(),
+        removeIncreaseRequest(nodeId,
+            prevChangeRequest.getRMContainer().getAllocatedSchedulerKey(),
             prevChangeRequest.getContainerId());
             prevChangeRequest.getContainerId());
       }
       }
 
 
@@ -219,21 +219,22 @@ public class AppSchedulingInfo {
    */
    */
   private void insertIncreaseRequest(SchedContainerChangeRequest request) {
   private void insertIncreaseRequest(SchedContainerChangeRequest request) {
     NodeId nodeId = request.getNodeId();
     NodeId nodeId = request.getNodeId();
-    Priority priority = request.getPriority();
+    SchedulerRequestKey schedulerKey =
+        request.getRMContainer().getAllocatedSchedulerKey();
     ContainerId containerId = request.getContainerId();
     ContainerId containerId = request.getContainerId();
     
     
-    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
-        containerIncreaseRequestMap.get(nodeId);
+    Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
+        requestsOnNode = containerIncreaseRequestMap.get(nodeId);
     if (null == requestsOnNode) {
     if (null == requestsOnNode) {
       requestsOnNode = new HashMap<>();
       requestsOnNode = new HashMap<>();
       containerIncreaseRequestMap.put(nodeId, requestsOnNode);
       containerIncreaseRequestMap.put(nodeId, requestsOnNode);
     }
     }
 
 
     Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
     Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
-        requestsOnNode.get(priority);
+        requestsOnNode.get(schedulerKey);
     if (null == requestsOnNodeWithPriority) {
     if (null == requestsOnNodeWithPriority) {
       requestsOnNodeWithPriority = new TreeMap<>();
       requestsOnNodeWithPriority = new TreeMap<>();
-      requestsOnNode.put(priority, requestsOnNodeWithPriority);
+      requestsOnNode.put(schedulerKey, requestsOnNodeWithPriority);
     }
     }
 
 
     requestsOnNodeWithPriority.put(containerId, request);
     requestsOnNodeWithPriority.put(containerId, request);
@@ -249,20 +250,20 @@ public class AppSchedulingInfo {
           + " delta=" + delta);
           + " delta=" + delta);
     }
     }
     
     
-    // update priorities
-    priorities.add(priority);
+    // update Scheduler Keys
+    schedulerKeys.add(schedulerKey);
   }
   }
   
   
-  public synchronized boolean removeIncreaseRequest(NodeId nodeId, Priority priority,
-      ContainerId containerId) {
-    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
-        containerIncreaseRequestMap.get(nodeId);
+  public synchronized boolean removeIncreaseRequest(NodeId nodeId,
+      SchedulerRequestKey schedulerKey, ContainerId containerId) {
+    Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
+        requestsOnNode = containerIncreaseRequestMap.get(nodeId);
     if (null == requestsOnNode) {
     if (null == requestsOnNode) {
       return false;
       return false;
     }
     }
 
 
     Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
     Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
-        requestsOnNode.get(priority);
+        requestsOnNode.get(schedulerKey);
     if (null == requestsOnNodeWithPriority) {
     if (null == requestsOnNodeWithPriority) {
       return false;
       return false;
     }
     }
@@ -272,7 +273,7 @@ public class AppSchedulingInfo {
     
     
     // remove hierarchies if it becomes empty
     // remove hierarchies if it becomes empty
     if (requestsOnNodeWithPriority.isEmpty()) {
     if (requestsOnNodeWithPriority.isEmpty()) {
-      requestsOnNode.remove(priority);
+      requestsOnNode.remove(schedulerKey);
     }
     }
     if (requestsOnNode.isEmpty()) {
     if (requestsOnNode.isEmpty()) {
       containerIncreaseRequestMap.remove(nodeId);
       containerIncreaseRequestMap.remove(nodeId);
@@ -296,15 +297,15 @@ public class AppSchedulingInfo {
   }
   }
   
   
   public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
   public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
-      Priority priority, ContainerId containerId) {
-    Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
-        containerIncreaseRequestMap.get(nodeId);
+      SchedulerRequestKey schedulerKey, ContainerId containerId) {
+    Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
+        requestsOnNode = containerIncreaseRequestMap.get(nodeId);
     if (null == requestsOnNode) {
     if (null == requestsOnNode) {
       return null;
       return null;
     }
     }
 
 
     Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
     Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
-        requestsOnNode.get(priority);
+        requestsOnNode.get(schedulerKey);
     return requestsOnNodeWithPriority == null ? null
     return requestsOnNodeWithPriority == null ? null
         : requestsOnNodeWithPriority.get(containerId);
         : requestsOnNodeWithPriority.get(containerId);
   }
   }
@@ -328,17 +329,18 @@ public class AppSchedulingInfo {
 
 
     // Update resource requests
     // Update resource requests
     for (ResourceRequest request : requests) {
     for (ResourceRequest request : requests) {
-      Priority priority = request.getPriority();
+      SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request);
       String resourceName = request.getResourceName();
       String resourceName = request.getResourceName();
 
 
       // Update node labels if required
       // Update node labels if required
       updateNodeLabels(request);
       updateNodeLabels(request);
 
 
-      Map<String, ResourceRequest> asks = this.resourceRequestMap.get(priority);
+      Map<String, ResourceRequest> asks =
+          this.resourceRequestMap.get(schedulerKey);
       if (asks == null) {
       if (asks == null) {
         asks = new ConcurrentHashMap<>();
         asks = new ConcurrentHashMap<>();
-        this.resourceRequestMap.put(priority, asks);
-        this.priorities.add(priority);
+        this.resourceRequestMap.put(schedulerKey, asks);
+        this.schedulerKeys.add(schedulerKey);
       }
       }
 
 
       // Increment number of containers if recovering preempted resources
       // Increment number of containers if recovering preempted resources
@@ -405,11 +407,11 @@ public class AppSchedulingInfo {
   }
   }
 
 
   private void updateNodeLabels(ResourceRequest request) {
   private void updateNodeLabels(ResourceRequest request) {
-    Priority priority = request.getPriority();
+    SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request);
     String resourceName = request.getResourceName();
     String resourceName = request.getResourceName();
     if (resourceName.equals(ResourceRequest.ANY)) {
     if (resourceName.equals(ResourceRequest.ANY)) {
       ResourceRequest previousAnyRequest =
       ResourceRequest previousAnyRequest =
-          getResourceRequest(priority, resourceName);
+          getResourceRequest(schedulerKey, resourceName);
 
 
       // When there is change in ANY request label expression, we should
       // When there is change in ANY request label expression, we should
       // update label for all resource requests already added of same
       // update label for all resource requests already added of same
@@ -417,7 +419,7 @@ public class AppSchedulingInfo {
       if ((null == previousAnyRequest)
       if ((null == previousAnyRequest)
           || hasRequestLabelChanged(previousAnyRequest, request)) {
           || hasRequestLabelChanged(previousAnyRequest, request)) {
         Map<String, ResourceRequest> resourceRequest =
         Map<String, ResourceRequest> resourceRequest =
-            getResourceRequests(priority);
+            getResourceRequests(schedulerKey);
         if (resourceRequest != null) {
         if (resourceRequest != null) {
           for (ResourceRequest r : resourceRequest.values()) {
           for (ResourceRequest r : resourceRequest.values()) {
             if (!r.getResourceName().equals(ResourceRequest.ANY)) {
             if (!r.getResourceName().equals(ResourceRequest.ANY)) {
@@ -428,7 +430,7 @@ public class AppSchedulingInfo {
       }
       }
     } else {
     } else {
       ResourceRequest anyRequest =
       ResourceRequest anyRequest =
-          getResourceRequest(priority, ResourceRequest.ANY);
+          getResourceRequest(schedulerKey, ResourceRequest.ANY);
       if (anyRequest != null) {
       if (anyRequest != null) {
         request.setNodeLabelExpression(anyRequest.getNodeLabelExpression());
         request.setNodeLabelExpression(anyRequest.getNodeLabelExpression());
       }
       }
@@ -501,13 +503,13 @@ public class AppSchedulingInfo {
     return userBlacklistChanged.getAndSet(false);
     return userBlacklistChanged.getAndSet(false);
   }
   }
 
 
-  public synchronized Collection<Priority> getPriorities() {
-    return priorities;
+  public synchronized Collection<SchedulerRequestKey> getSchedulerKeys() {
+    return schedulerKeys;
   }
   }
 
 
   public synchronized Map<String, ResourceRequest> getResourceRequests(
   public synchronized Map<String, ResourceRequest> getResourceRequests(
-      Priority priority) {
-    return resourceRequestMap.get(priority);
+      SchedulerRequestKey schedulerKey) {
+    return resourceRequestMap.get(schedulerKey);
   }
   }
 
 
   public synchronized List<ResourceRequest> getAllResourceRequests() {
   public synchronized List<ResourceRequest> getAllResourceRequests() {
@@ -518,14 +520,16 @@ public class AppSchedulingInfo {
     return ret;
     return ret;
   }
   }
 
 
-  public synchronized ResourceRequest getResourceRequest(Priority priority,
-      String resourceName) {
-    Map<String, ResourceRequest> nodeRequests = resourceRequestMap.get(priority);
+  public synchronized ResourceRequest getResourceRequest(
+      SchedulerRequestKey schedulerKey, String resourceName) {
+    Map<String, ResourceRequest> nodeRequests =
+        resourceRequestMap.get(schedulerKey);
     return (nodeRequests == null) ? null : nodeRequests.get(resourceName);
     return (nodeRequests == null) ? null : nodeRequests.get(resourceName);
   }
   }
 
 
-  public synchronized Resource getResource(Priority priority) {
-    ResourceRequest request = getResourceRequest(priority, ResourceRequest.ANY);
+  public synchronized Resource getResource(SchedulerRequestKey schedulerKey) {
+    ResourceRequest request =
+        getResourceRequest(schedulerKey, ResourceRequest.ANY);
     return (request == null) ? null : request.getCapability();
     return (request == null) ? null : request.getCapability();
   }
   }
 
 
@@ -555,7 +559,8 @@ public class AppSchedulingInfo {
   public synchronized void increaseContainer(
   public synchronized void increaseContainer(
       SchedContainerChangeRequest increaseRequest) {
       SchedContainerChangeRequest increaseRequest) {
     NodeId nodeId = increaseRequest.getNodeId();
     NodeId nodeId = increaseRequest.getNodeId();
-    Priority priority = increaseRequest.getPriority();
+    SchedulerRequestKey schedulerKey =
+        increaseRequest.getRMContainer().getAllocatedSchedulerKey();
     ContainerId containerId = increaseRequest.getContainerId();
     ContainerId containerId = increaseRequest.getContainerId();
     Resource deltaCapacity = increaseRequest.getDeltaCapacity();
     Resource deltaCapacity = increaseRequest.getDeltaCapacity();
 
 
@@ -568,7 +573,7 @@ public class AppSchedulingInfo {
     // Set queue metrics
     // Set queue metrics
     queue.getMetrics().allocateResources(user, deltaCapacity);
     queue.getMetrics().allocateResources(user, deltaCapacity);
     // remove the increase request from pending increase request map
     // remove the increase request from pending increase request map
-    removeIncreaseRequest(nodeId, priority, containerId);
+    removeIncreaseRequest(nodeId, schedulerKey, containerId);
     // update usage
     // update usage
     appResourceUsage.incUsed(increaseRequest.getNodePartition(), deltaCapacity);
     appResourceUsage.incUsed(increaseRequest.getNodePartition(), deltaCapacity);
   }
   }
@@ -591,19 +596,25 @@ public class AppSchedulingInfo {
     // update usage
     // update usage
     appResourceUsage.decUsed(decreaseRequest.getNodePartition(), absDelta);
     appResourceUsage.decUsed(decreaseRequest.getNodePartition(), absDelta);
   }
   }
-  
+
   /**
   /**
    * Resources have been allocated to this application by the resource
    * Resources have been allocated to this application by the resource
    * scheduler. Track them.
    * scheduler. Track them.
+   * @param type Node Type
+   * @param node SchedulerNode
+   * @param schedulerKey SchedulerRequestKey
+   * @param request ResourceRequest
+   * @param containerAllocated Container Allocated
+   * @return List of ResourceRequests
    */
    */
   public synchronized List<ResourceRequest> allocate(NodeType type,
   public synchronized List<ResourceRequest> allocate(NodeType type,
-      SchedulerNode node, Priority priority, ResourceRequest request,
-      Container containerAllocated) {
+      SchedulerNode node, SchedulerRequestKey schedulerKey,
+      ResourceRequest request, Container containerAllocated) {
     List<ResourceRequest> resourceRequests = new ArrayList<>();
     List<ResourceRequest> resourceRequests = new ArrayList<>();
     if (type == NodeType.NODE_LOCAL) {
     if (type == NodeType.NODE_LOCAL) {
-      allocateNodeLocal(node, priority, request, resourceRequests);
+      allocateNodeLocal(node, schedulerKey, request, resourceRequests);
     } else if (type == NodeType.RACK_LOCAL) {
     } else if (type == NodeType.RACK_LOCAL) {
-      allocateRackLocal(node, priority, request, resourceRequests);
+      allocateRackLocal(node, schedulerKey, request, resourceRequests);
     } else {
     } else {
       allocateOffSwitch(request, resourceRequests);
       allocateOffSwitch(request, resourceRequests);
     }
     }
@@ -633,16 +644,16 @@ public class AppSchedulingInfo {
    * application.
    * application.
    */
    */
   private synchronized void allocateNodeLocal(SchedulerNode node,
   private synchronized void allocateNodeLocal(SchedulerNode node,
-      Priority priority, ResourceRequest nodeLocalRequest,
+      SchedulerRequestKey schedulerKey, ResourceRequest nodeLocalRequest,
       List<ResourceRequest> resourceRequests) {
       List<ResourceRequest> resourceRequests) {
     // Update future requirements
     // Update future requirements
-    decResourceRequest(node.getNodeName(), priority, nodeLocalRequest);
+    decResourceRequest(node.getNodeName(), schedulerKey, nodeLocalRequest);
 
 
-    ResourceRequest rackLocalRequest = resourceRequestMap.get(priority).get(
+    ResourceRequest rackLocalRequest = resourceRequestMap.get(schedulerKey).get(
         node.getRackName());
         node.getRackName());
-    decResourceRequest(node.getRackName(), priority, rackLocalRequest);
+    decResourceRequest(node.getRackName(), schedulerKey, rackLocalRequest);
 
 
-    ResourceRequest offRackRequest = resourceRequestMap.get(priority).get(
+    ResourceRequest offRackRequest = resourceRequestMap.get(schedulerKey).get(
         ResourceRequest.ANY);
         ResourceRequest.ANY);
     decrementOutstanding(offRackRequest);
     decrementOutstanding(offRackRequest);
 
 
@@ -652,11 +663,11 @@ public class AppSchedulingInfo {
     resourceRequests.add(cloneResourceRequest(offRackRequest));
     resourceRequests.add(cloneResourceRequest(offRackRequest));
   }
   }
 
 
-  private void decResourceRequest(String resourceName, Priority priority,
-      ResourceRequest request) {
+  private void decResourceRequest(String resourceName,
+      SchedulerRequestKey schedulerKey, ResourceRequest request) {
     request.setNumContainers(request.getNumContainers() - 1);
     request.setNumContainers(request.getNumContainers() - 1);
     if (request.getNumContainers() == 0) {
     if (request.getNumContainers() == 0) {
-      resourceRequestMap.get(priority).remove(resourceName);
+      resourceRequestMap.get(schedulerKey).remove(resourceName);
     }
     }
   }
   }
 
 
@@ -665,12 +676,12 @@ public class AppSchedulingInfo {
    * application.
    * application.
    */
    */
   private synchronized void allocateRackLocal(SchedulerNode node,
   private synchronized void allocateRackLocal(SchedulerNode node,
-      Priority priority, ResourceRequest rackLocalRequest,
+      SchedulerRequestKey schedulerKey, ResourceRequest rackLocalRequest,
       List<ResourceRequest> resourceRequests) {
       List<ResourceRequest> resourceRequests) {
     // Update future requirements
     // Update future requirements
-    decResourceRequest(node.getRackName(), priority, rackLocalRequest);
+    decResourceRequest(node.getRackName(), schedulerKey, rackLocalRequest);
     
     
-    ResourceRequest offRackRequest = resourceRequestMap.get(priority).get(
+    ResourceRequest offRackRequest = resourceRequestMap.get(schedulerKey).get(
         ResourceRequest.ANY);
         ResourceRequest.ANY);
     decrementOutstanding(offRackRequest);
     decrementOutstanding(offRackRequest);
 
 
@@ -712,8 +723,9 @@ public class AppSchedulingInfo {
   
   
   private synchronized void checkForDeactivation() {
   private synchronized void checkForDeactivation() {
     boolean deactivate = true;
     boolean deactivate = true;
-    for (Priority priority : getPriorities()) {
-      ResourceRequest request = getResourceRequest(priority, ResourceRequest.ANY);
+    for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) {
+      ResourceRequest request =
+          getResourceRequest(schedulerKey, ResourceRequest.ANY);
       if (request != null) {
       if (request != null) {
         if (request.getNumContainers() > 0) {
         if (request.getNumContainers() > 0) {
           deactivate = false;
           deactivate = false;

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

@@ -98,10 +98,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   protected ApplicationAttemptId attemptId;
   protected ApplicationAttemptId attemptId;
   protected Map<ContainerId, RMContainer> liveContainers =
   protected Map<ContainerId, RMContainer> liveContainers =
       new HashMap<ContainerId, RMContainer>();
       new HashMap<ContainerId, RMContainer>();
-  protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers = 
-      new HashMap<Priority, Map<NodeId, RMContainer>>();
+  protected final Map<SchedulerRequestKey, Map<NodeId, RMContainer>>
+      reservedContainers = new HashMap<>();
 
 
-  private final Multiset<Priority> reReservations = HashMultiset.create();
+  private final Multiset<SchedulerRequestKey> reReservations =
+      HashMultiset.create();
   
   
   private Resource resourceLimit = Resource.newInstance(0, 0);
   private Resource resourceLimit = Resource.newInstance(0, 0);
   private boolean unmanagedAM = true;
   private boolean unmanagedAM = true;
@@ -137,7 +138,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    * the application successfully schedules a task (at rack or node local), it
    * the application successfully schedules a task (at rack or node local), it
    * is reset to 0.
    * is reset to 0.
    */
    */
-  Multiset<Priority> schedulingOpportunities = HashMultiset.create();
+  Multiset<SchedulerRequestKey> schedulingOpportunities = HashMultiset.create();
   
   
   /**
   /**
    * Count how many times the application has been given an opportunity to
    * Count how many times the application has been given an opportunity to
@@ -146,12 +147,12 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    * incremented, and each time the application successfully schedules a task,
    * incremented, and each time the application successfully schedules a task,
    * it is reset to 0 when schedule any task at corresponding priority.
    * it is reset to 0 when schedule any task at corresponding priority.
    */
    */
-  Multiset<Priority> missedNonPartitionedRequestSchedulingOpportunity =
+  Multiset<SchedulerRequestKey> missedNonPartitionedReqSchedulingOpportunity =
       HashMultiset.create();
       HashMultiset.create();
   
   
   // Time of the last container scheduled at the current allowed level
   // Time of the last container scheduled at the current allowed level
-  protected Map<Priority, Long> lastScheduledContainer =
-      new HashMap<Priority, Long>();
+  protected Map<SchedulerRequestKey, Long> lastScheduledContainer =
+      new HashMap<>();
 
 
   protected Queue queue;
   protected Queue queue;
   protected boolean isStopped = false;
   protected boolean isStopped = false;
@@ -225,8 +226,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return appSchedulingInfo.getUser();
     return appSchedulingInfo.getUser();
   }
   }
 
 
-  public Map<String, ResourceRequest> getResourceRequests(Priority priority) {
-    return appSchedulingInfo.getResourceRequests(priority);
+  public Map<String, ResourceRequest> getResourceRequests(
+      SchedulerRequestKey schedulerKey) {
+    return appSchedulingInfo.getResourceRequests(schedulerKey);
   }
   }
 
 
   public Set<ContainerId> getPendingRelease() {
   public Set<ContainerId> getPendingRelease() {
@@ -237,22 +239,24 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return appSchedulingInfo.getNewContainerId();
     return appSchedulingInfo.getNewContainerId();
   }
   }
 
 
-  public Collection<Priority> getPriorities() {
-    return appSchedulingInfo.getPriorities();
+  public Collection<SchedulerRequestKey> getSchedulerKeys() {
+    return appSchedulingInfo.getSchedulerKeys();
   }
   }
   
   
-  public synchronized ResourceRequest getResourceRequest(Priority priority,
-      String resourceName) {
-    return this.appSchedulingInfo.getResourceRequest(priority, resourceName);
+  public synchronized ResourceRequest getResourceRequest(
+      SchedulerRequestKey schedulerKey, String resourceName) {
+    return appSchedulingInfo.getResourceRequest(schedulerKey, resourceName);
   }
   }
 
 
-  public synchronized int getTotalRequiredResources(Priority priority) {
-    ResourceRequest request = getResourceRequest(priority, ResourceRequest.ANY);
+  public synchronized int getTotalRequiredResources(
+      SchedulerRequestKey schedulerKey) {
+    ResourceRequest request =
+        getResourceRequest(schedulerKey, ResourceRequest.ANY);
     return request == null ? 0 : request.getNumContainers();
     return request == null ? 0 : request.getNumContainers();
   }
   }
 
 
-  public synchronized Resource getResource(Priority priority) {
-    return appSchedulingInfo.getResource(priority);
+  public synchronized Resource getResource(SchedulerRequestKey schedulerKey) {
+    return appSchedulingInfo.getResource(schedulerKey);
   }
   }
 
 
   public String getQueueName() {
   public String getQueueName() {
@@ -308,16 +312,18 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     }
     }
   }
   }
 
 
-  protected synchronized void resetReReservations(Priority priority) {
-    reReservations.setCount(priority, 0);
+  protected synchronized void resetReReservations(
+      SchedulerRequestKey schedulerKey) {
+    reReservations.setCount(schedulerKey, 0);
   }
   }
 
 
-  protected synchronized void addReReservation(Priority priority) {
-    reReservations.add(priority);
+  protected synchronized void addReReservation(
+      SchedulerRequestKey schedulerKey) {
+    reReservations.add(schedulerKey);
   }
   }
 
 
-  public synchronized int getReReservations(Priority priority) {
-    return reReservations.count(priority);
+  public synchronized int getReReservations(SchedulerRequestKey schedulerKey) {
+    return reReservations.count(schedulerKey);
   }
   }
 
 
   /**
   /**
@@ -366,7 +372,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    */
    */
   public synchronized List<RMContainer> getReservedContainers() {
   public synchronized List<RMContainer> getReservedContainers() {
     List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
     List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
-    for (Map.Entry<Priority, Map<NodeId, RMContainer>> e : 
+    for (Map.Entry<SchedulerRequestKey, Map<NodeId, RMContainer>> e :
       this.reservedContainers.entrySet()) {
       this.reservedContainers.entrySet()) {
       reservedContainers.addAll(e.getValue().values());
       reservedContainers.addAll(e.getValue().values());
     }
     }
@@ -374,8 +380,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   }
   }
   
   
   public synchronized boolean reserveIncreasedContainer(SchedulerNode node,
   public synchronized boolean reserveIncreasedContainer(SchedulerNode node,
-      Priority priority, RMContainer rmContainer, Resource reservedResource) {
-    if (commonReserve(node, priority, rmContainer, reservedResource)) {
+      SchedulerRequestKey schedulerKey, RMContainer rmContainer,
+      Resource reservedResource) {
+    if (commonReserve(node, schedulerKey, rmContainer, reservedResource)) {
       attemptResourceUsage.incReserved(node.getPartition(),
       attemptResourceUsage.incReserved(node.getPartition(),
           reservedResource);
           reservedResource);
       // succeeded
       // succeeded
@@ -386,10 +393,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   }
   }
   
   
   private synchronized boolean commonReserve(SchedulerNode node,
   private synchronized boolean commonReserve(SchedulerNode node,
-      Priority priority, RMContainer rmContainer, Resource reservedResource) {
+      SchedulerRequestKey schedulerKey, RMContainer rmContainer,
+      Resource reservedResource) {
     try {
     try {
       rmContainer.handle(new RMContainerReservedEvent(rmContainer
       rmContainer.handle(new RMContainerReservedEvent(rmContainer
-          .getContainerId(), reservedResource, node.getNodeID(), priority));
+          .getContainerId(), reservedResource, node.getNodeID(), schedulerKey));
     } catch (InvalidStateTransitionException e) {
     } catch (InvalidStateTransitionException e) {
       // We reach here could be caused by container already finished, return
       // We reach here could be caused by container already finished, return
       // false indicate it fails
       // false indicate it fails
@@ -397,10 +405,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     }
     }
     
     
     Map<NodeId, RMContainer> reservedContainers = 
     Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(priority);
+        this.reservedContainers.get(schedulerKey);
     if (reservedContainers == null) {
     if (reservedContainers == null) {
       reservedContainers = new HashMap<NodeId, RMContainer>();
       reservedContainers = new HashMap<NodeId, RMContainer>();
-      this.reservedContainers.put(priority, reservedContainers);
+      this.reservedContainers.put(schedulerKey, reservedContainers);
     }
     }
     reservedContainers.put(node.getNodeID(), rmContainer);
     reservedContainers.put(node.getNodeID(), rmContainer);
 
 
@@ -408,7 +416,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       LOG.debug("Application attempt " + getApplicationAttemptId()
       LOG.debug("Application attempt " + getApplicationAttemptId()
           + " reserved container " + rmContainer + " on node " + node
           + " reserved container " + rmContainer + " on node " + node
           + ". This attempt currently has " + reservedContainers.size()
           + ". This attempt currently has " + reservedContainers.size()
-          + " reserved containers at priority " + priority
+          + " reserved containers at priority " + schedulerKey.getPriority()
           + "; currentReservation " + reservedResource);
           + "; currentReservation " + reservedResource);
     }
     }
     
     
@@ -416,7 +424,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   }
   }
   
   
   public synchronized RMContainer reserve(SchedulerNode node,
   public synchronized RMContainer reserve(SchedulerNode node,
-      Priority priority, RMContainer rmContainer, Container container) {
+      SchedulerRequestKey schedulerKey, RMContainer rmContainer,
+      Container container) {
     // Create RMContainer if necessary
     // Create RMContainer if necessary
     if (rmContainer == null) {
     if (rmContainer == null) {
       rmContainer =
       rmContainer =
@@ -427,13 +436,13 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       ((RMContainerImpl)rmContainer).setQueueName(this.getQueueName());
       ((RMContainerImpl)rmContainer).setQueueName(this.getQueueName());
 
 
       // Reset the re-reservation count
       // Reset the re-reservation count
-      resetReReservations(priority);
+      resetReReservations(schedulerKey);
     } else {
     } else {
       // Note down the re-reservation
       // Note down the re-reservation
-      addReReservation(priority);
+      addReReservation(schedulerKey);
     }
     }
     
     
-    commonReserve(node, priority, rmContainer, container.getResource());
+    commonReserve(node, schedulerKey, rmContainer, container.getResource());
 
 
     return rmContainer;
     return rmContainer;
   }
   }
@@ -442,12 +451,13 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    * Has the application reserved the given <code>node</code> at the
    * Has the application reserved the given <code>node</code> at the
    * given <code>priority</code>?
    * given <code>priority</code>?
    * @param node node to be checked
    * @param node node to be checked
-   * @param priority priority of reserved container
+   * @param schedulerKey scheduler key  of reserved container
    * @return true is reserved, false if not
    * @return true is reserved, false if not
    */
    */
-  public synchronized boolean isReserved(SchedulerNode node, Priority priority) {
+  public synchronized boolean isReserved(SchedulerNode node,
+      SchedulerRequestKey schedulerKey) {
     Map<NodeId, RMContainer> reservedContainers = 
     Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(priority);
+        this.reservedContainers.get(schedulerKey);
     if (reservedContainers != null) {
     if (reservedContainers != null) {
       return reservedContainers.containsKey(node.getNodeID());
       return reservedContainers.containsKey(node.getNodeID());
     }
     }
@@ -471,9 +481,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return resourceLimit;
     return resourceLimit;
   }
   }
   
   
-  public synchronized int getNumReservedContainers(Priority priority) {
+  public synchronized int getNumReservedContainers(
+      SchedulerRequestKey schedulerKey) {
     Map<NodeId, RMContainer> reservedContainers = 
     Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(priority);
+        this.reservedContainers.get(schedulerKey);
     return (reservedContainers == null) ? 0 : reservedContainers.size();
     return (reservedContainers == null) ? 0 : reservedContainers.size();
   }
   }
   
   
@@ -495,8 +506,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   
   
   public synchronized void showRequests() {
   public synchronized void showRequests() {
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
-      for (Priority priority : getPriorities()) {
-        Map<String, ResourceRequest> requests = getResourceRequests(priority);
+      for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) {
+        Map<String, ResourceRequest> requests =
+            getResourceRequests(schedulerKey);
         if (requests != null) {
         if (requests != null) {
           LOG.debug("showRequests:" + " application=" + getApplicationId()
           LOG.debug("showRequests:" + " application=" + getApplicationId()
               + " headRoom=" + getHeadroom() + " currentConsumption="
               + " headRoom=" + getHeadroom() + " currentConsumption="
@@ -635,59 +647,66 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   }
   }
 
 
   public synchronized int addMissedNonPartitionedRequestSchedulingOpportunity(
   public synchronized int addMissedNonPartitionedRequestSchedulingOpportunity(
-      Priority priority) {
-    missedNonPartitionedRequestSchedulingOpportunity.add(priority);
-    return missedNonPartitionedRequestSchedulingOpportunity.count(priority);
+      SchedulerRequestKey schedulerKey) {
+    missedNonPartitionedReqSchedulingOpportunity.add(schedulerKey);
+    return missedNonPartitionedReqSchedulingOpportunity.count(schedulerKey);
   }
   }
 
 
   public synchronized void
   public synchronized void
-      resetMissedNonPartitionedRequestSchedulingOpportunity(Priority priority) {
-    missedNonPartitionedRequestSchedulingOpportunity.setCount(priority, 0);
+      resetMissedNonPartitionedRequestSchedulingOpportunity(
+      SchedulerRequestKey schedulerKey) {
+    missedNonPartitionedReqSchedulingOpportunity.setCount(schedulerKey, 0);
   }
   }
 
 
   
   
-  public synchronized void addSchedulingOpportunity(Priority priority) {
-    int count = schedulingOpportunities.count(priority);
+  public synchronized void addSchedulingOpportunity(
+      SchedulerRequestKey schedulerKey) {
+    int count = schedulingOpportunities.count(schedulerKey);
     if (count < Integer.MAX_VALUE) {
     if (count < Integer.MAX_VALUE) {
-      schedulingOpportunities.setCount(priority, count + 1);
+      schedulingOpportunities.setCount(schedulerKey, count + 1);
     }
     }
   }
   }
   
   
-  public synchronized void subtractSchedulingOpportunity(Priority priority) {
-    int count = schedulingOpportunities.count(priority) - 1;
-    this.schedulingOpportunities.setCount(priority, Math.max(count,  0));
+  public synchronized void subtractSchedulingOpportunity(
+      SchedulerRequestKey schedulerKey) {
+    int count = schedulingOpportunities.count(schedulerKey) - 1;
+    this.schedulingOpportunities.setCount(schedulerKey, Math.max(count,  0));
   }
   }
 
 
   /**
   /**
    * Return the number of times the application has been given an opportunity
    * Return the number of times the application has been given an opportunity
    * to schedule a task at the given priority since the last time it
    * to schedule a task at the given priority since the last time it
    * successfully did so.
    * successfully did so.
+   * @param schedulerKey Scheduler Key
+   * @return number of scheduling opportunities
    */
    */
-  public synchronized int getSchedulingOpportunities(Priority priority) {
-    return schedulingOpportunities.count(priority);
+  public synchronized int getSchedulingOpportunities(
+      SchedulerRequestKey schedulerKey) {
+    return schedulingOpportunities.count(schedulerKey);
   }
   }
   
   
   /**
   /**
-   * Should be called when an application has successfully scheduled a container,
-   * or when the scheduling locality threshold is relaxed.
+   * Should be called when an application has successfully scheduled a
+   * container, or when the scheduling locality threshold is relaxed.
    * Reset various internal counters which affect delay scheduling
    * Reset various internal counters which affect delay scheduling
    *
    *
-   * @param priority The priority of the container scheduled.
+   * @param schedulerKey The priority of the container scheduled.
    */
    */
-  public synchronized void resetSchedulingOpportunities(Priority priority) {
-    resetSchedulingOpportunities(priority, System.currentTimeMillis());
+  public synchronized void resetSchedulingOpportunities(
+      SchedulerRequestKey schedulerKey) {
+    resetSchedulingOpportunities(schedulerKey, System.currentTimeMillis());
   }
   }
 
 
   // used for continuous scheduling
   // used for continuous scheduling
-  public synchronized void resetSchedulingOpportunities(Priority priority,
-      long currentTimeMs) {
-    lastScheduledContainer.put(priority, currentTimeMs);
-    schedulingOpportunities.setCount(priority, 0);
+  public synchronized void resetSchedulingOpportunities(
+      SchedulerRequestKey schedulerKey, long currentTimeMs) {
+    lastScheduledContainer.put(schedulerKey, currentTimeMs);
+    schedulingOpportunities.setCount(schedulerKey, 0);
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
-  void setSchedulingOpportunities(Priority priority, int count) {
-    schedulingOpportunities.setCount(priority, count);
+  void setSchedulingOpportunities(SchedulerRequestKey schedulerKey, int count) {
+    schedulingOpportunities.setCount(schedulerKey, count);
   }
   }
 
 
   synchronized AggregateAppResourceUsage getRunningAggregateAppResourceUsage() {
   synchronized AggregateAppResourceUsage getRunningAggregateAppResourceUsage() {
@@ -747,7 +766,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return this.resourceLimit;
     return this.resourceLimit;
   }
   }
 
 
-  public synchronized Map<Priority, Long> getLastScheduledContainer() {
+  public synchronized Map<SchedulerRequestKey, Long>
+      getLastScheduledContainer() {
     return this.lastScheduledContainer;
     return this.lastScheduledContainer;
   }
   }
 
 
@@ -892,8 +912,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   }
   }
   
   
   public synchronized boolean removeIncreaseRequest(NodeId nodeId,
   public synchronized boolean removeIncreaseRequest(NodeId nodeId,
-      Priority priority, ContainerId containerId) {
-    return appSchedulingInfo.removeIncreaseRequest(nodeId, priority,
+      SchedulerRequestKey schedulerKey, ContainerId containerId) {
+    return appSchedulingInfo.removeIncreaseRequest(nodeId, schedulerKey,
         containerId);
         containerId);
   }
   }
   
   

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

@@ -31,7 +31,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 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.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -308,11 +307,11 @@ public abstract class SchedulerNode {
   /**
   /**
    * Reserve container for the attempt on this node.
    * Reserve container for the attempt on this node.
    * @param attempt Application attempt asking for the reservation.
    * @param attempt Application attempt asking for the reservation.
-   * @param priority Priority of the reservation.
+   * @param schedulerKey Priority of the reservation.
    * @param container Container reserving resources for.
    * @param container Container reserving resources for.
    */
    */
   public abstract void reserveResource(SchedulerApplicationAttempt attempt,
   public abstract void reserveResource(SchedulerApplicationAttempt attempt,
-      Priority priority, RMContainer container);
+      SchedulerRequestKey schedulerKey, RMContainer container);
 
 
   /**
   /**
    * Unreserve resources on this node.
    * Unreserve resources on this node.

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

@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+
+/**
+ * Composite key for outstanding scheduler requests for any schedulable entity.
+ * Currently it includes {@link Priority}.
+ */
+public final class SchedulerRequestKey implements
+    Comparable<SchedulerRequestKey> {
+
+  private final Priority priority;
+
+  public static final SchedulerRequestKey UNDEFINED =
+      new SchedulerRequestKey(Priority.UNDEFINED);
+
+  /**
+   * Factory method to generate a SchedulerRequestKey from a ResourceRequest.
+   * @param req ResourceRequest
+   * @return SchedulerRequestKey
+   */
+  public static SchedulerRequestKey create(ResourceRequest req) {
+    return new SchedulerRequestKey(req.getPriority());
+  }
+
+  /**
+   * Convenience method to extract the SchedulerRequestKey used to schedule the
+   * Container.
+   * @param container Container
+   * @return SchedulerRequestKey
+   */
+  public static SchedulerRequestKey extractFrom(Container container) {
+    return new SchedulerRequestKey(container.getPriority());
+  }
+
+  private SchedulerRequestKey(Priority priority) {
+    this.priority = priority;
+  }
+
+  /**
+   * Get the {@link Priority} of the request.
+   *
+   * @return the {@link Priority} of the request
+   */
+  public Priority getPriority() {
+    return priority;
+  }
+
+  @Override
+  public int compareTo(SchedulerRequestKey o) {
+    if (o == null) {
+      return (priority != null) ? -1 : 0;
+    } else {
+      if (priority == null) {
+        return 1;
+      }
+    }
+    return o.getPriority().compareTo(priority);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof SchedulerRequestKey)) {
+      return false;
+    }
+
+    SchedulerRequestKey that = (SchedulerRequestKey) o;
+    return getPriority().equals(that.getPriority());
+
+  }
+
+  @Override
+  public int hashCode() {
+    return getPriority().hashCode();
+  }
+}

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -1265,7 +1265,8 @@ public class LeafQueue extends AbstractCSQueue {
       }
       }
 
 
       if (null != priority) {
       if (null != priority) {
-        removed = app.unreserve(rmContainer.getContainer().getPriority(), node,
+        removed = app.unreserve(
+            rmContainer.getAllocatedSchedulerKey(), node,
             rmContainer);
             rmContainer);
       }
       }
 
 
@@ -1321,7 +1322,7 @@ public class LeafQueue extends AbstractCSQueue {
       
       
       // Remove container increase request if it exists
       // Remove container increase request if it exists
       application.removeIncreaseRequest(node.getNodeID(),
       application.removeIncreaseRequest(node.getNodeID(),
-          rmContainer.getAllocatedPriority(), rmContainer.getContainerId());
+          rmContainer.getAllocatedSchedulerKey(), rmContainer.getContainerId());
 
 
       boolean removed = false;
       boolean removed = false;
 
 
@@ -1335,7 +1336,7 @@ public class LeafQueue extends AbstractCSQueue {
         // happen under scheduler's lock... 
         // happen under scheduler's lock... 
         // So, this is, in effect, a transaction across application & node
         // So, this is, in effect, a transaction across application & node
         if (rmContainer.getState() == RMContainerState.RESERVED) {
         if (rmContainer.getState() == RMContainerState.RESERVED) {
-          removed = application.unreserve(rmContainer.getReservedPriority(),
+          removed = application.unreserve(rmContainer.getReservedSchedulerKey(),
               node, rmContainer);
               node, rmContainer);
         } else {
         } else {
           removed =
           removed =
@@ -1785,7 +1786,8 @@ public class LeafQueue extends AbstractCSQueue {
       // Do we have increase request for the same container? If so, remove it
       // Do we have increase request for the same container? If so, remove it
       boolean hasIncreaseRequest =
       boolean hasIncreaseRequest =
           app.removeIncreaseRequest(decreaseRequest.getNodeId(),
           app.removeIncreaseRequest(decreaseRequest.getNodeId(),
-              decreaseRequest.getPriority(), decreaseRequest.getContainerId());
+              decreaseRequest.getRMContainer().getAllocatedSchedulerKey(),
+              decreaseRequest.getContainerId());
       if (hasIncreaseRequest) {
       if (hasIncreaseRequest) {
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("While processing decrease requests, found an increase"
           LOG.debug("While processing decrease requests, found an increase"

+ 15 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java

@@ -29,7 +29,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.NodeId;
 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.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -37,6 +36,8 @@ 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.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
@@ -115,7 +116,8 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
         node.getUnallocatedResource())) {
         node.getUnallocatedResource())) {
       // OK, we can allocate this increase request
       // OK, we can allocate this increase request
       // Unreserve it first
       // Unreserve it first
-      application.unreserve(increaseRequest.getPriority(),
+      application.unreserve(
+          increaseRequest.getRMContainer().getAllocatedSchedulerKey(),
           (FiCaSchedulerNode) node, increaseRequest.getRMContainer());
           (FiCaSchedulerNode) node, increaseRequest.getRMContainer());
       
       
       // Notify application
       // Notify application
@@ -152,7 +154,8 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
       return createSuccessfullyIncreasedCSAssignment(increaseRequest, false);
       return createSuccessfullyIncreasedCSAssignment(increaseRequest, false);
     } else {
     } else {
       boolean reservationSucceeded =
       boolean reservationSucceeded =
-          application.reserveIncreasedContainer(increaseRequest.getPriority(),
+          application.reserveIncreasedContainer(
+              increaseRequest.getRMContainer().getAllocatedSchedulerKey(),
               node, increaseRequest.getRMContainer(),
               node, increaseRequest.getRMContainer(),
               increaseRequest.getDeltaCapacity());
               increaseRequest.getDeltaCapacity());
       
       
@@ -228,11 +231,11 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
        * priority, but will skip increase request and move to next increase
        * priority, but will skip increase request and move to next increase
        * request if queue-limit or user-limit aren't satisfied 
        * request if queue-limit or user-limit aren't satisfied 
        */
        */
-      for (Priority priority : application.getPriorities()) {
+      for (SchedulerRequestKey schedulerKey : application.getSchedulerKeys()) {
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Looking at increase request for application="
           LOG.debug("Looking at increase request for application="
               + application.getApplicationAttemptId() + " priority="
               + application.getApplicationAttemptId() + " priority="
-              + priority);
+              + schedulerKey.getPriority());
         }
         }
 
 
         /*
         /*
@@ -242,14 +245,14 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
          * cannot be allocated.
          * cannot be allocated.
          */
          */
         Map<ContainerId, SchedContainerChangeRequest> increaseRequestMap =
         Map<ContainerId, SchedContainerChangeRequest> increaseRequestMap =
-            sinfo.getIncreaseRequests(nodeId, priority);
+            sinfo.getIncreaseRequests(nodeId, schedulerKey);
 
 
         // We don't have more increase request on this priority, skip..
         // We don't have more increase request on this priority, skip..
         if (null == increaseRequestMap) {
         if (null == increaseRequestMap) {
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("There's no increase request for "
             LOG.debug("There's no increase request for "
                 + application.getApplicationAttemptId() + " priority="
                 + application.getApplicationAttemptId() + " priority="
-                + priority);
+                + schedulerKey.getPriority());
           }
           }
           continue;
           continue;
         }
         }
@@ -318,7 +321,8 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
         // Remove invalid in request requests
         // Remove invalid in request requests
         if (!toBeRemovedRequests.isEmpty()) {
         if (!toBeRemovedRequests.isEmpty()) {
           for (SchedContainerChangeRequest req : toBeRemovedRequests) {
           for (SchedContainerChangeRequest req : toBeRemovedRequests) {
-            sinfo.removeIncreaseRequest(req.getNodeId(), req.getPriority(),
+            sinfo.removeIncreaseRequest(req.getNodeId(),
+                req.getRMContainer().getAllocatedSchedulerKey(),
                 req.getContainerId());
                 req.getContainerId());
           }
           }
         }
         }
@@ -337,8 +341,9 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
       
       
       // We already reserved this increase container
       // We already reserved this increase container
       SchedContainerChangeRequest request =
       SchedContainerChangeRequest request =
-          sinfo.getIncreaseRequest(nodeId, reservedContainer.getContainer()
-              .getPriority(), reservedContainer.getContainerId());
+          sinfo.getIncreaseRequest(nodeId,
+              reservedContainer.getAllocatedSchedulerKey(),
+              reservedContainer.getContainerId());
       
       
       // We will cancel the reservation any of following happens
       // We will cancel the reservation any of following happens
       // - Container finished
       // - Container finished

+ 97 - 84
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

@@ -23,7 +23,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 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.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -32,6 +31,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
+
+
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMContainerLaunchDiagnosticsConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMContainerLaunchDiagnosticsConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
@@ -80,7 +83,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   
   
   private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
   private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
       FiCaSchedulerNode node, SchedulingMode schedulingMode,
       FiCaSchedulerNode node, SchedulingMode schedulingMode,
-      ResourceLimits resourceLimits, Priority priority) {
+      ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey) {
     if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) {
     if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) {
       application.updateAppSkipNodeDiagnostics(
       application.updateAppSkipNodeDiagnostics(
           CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE);
           CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE);
@@ -88,7 +91,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     }
     }
 
 
     ResourceRequest anyRequest =
     ResourceRequest anyRequest =
-        application.getResourceRequest(priority, ResourceRequest.ANY);
+        application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
     if (null == anyRequest) {
     if (null == anyRequest) {
       return ContainerAllocation.PRIORITY_SKIPPED;
       return ContainerAllocation.PRIORITY_SKIPPED;
     }
     }
@@ -97,7 +100,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     Resource required = anyRequest.getCapability();
     Resource required = anyRequest.getCapability();
 
 
     // Do we need containers at this 'priority'?
     // Do we need containers at this 'priority'?
-    if (application.getTotalRequiredResources(priority) <= 0) {
+    if (application.getTotalRequiredResources(schedulerKey) <= 0) {
       return ContainerAllocation.PRIORITY_SKIPPED;
       return ContainerAllocation.PRIORITY_SKIPPED;
     }
     }
 
 
@@ -126,7 +129,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     }
     }
 
 
     if (!application.getCSLeafQueue().getReservationContinueLooking()) {
     if (!application.getCSLeafQueue().getReservationContinueLooking()) {
-      if (!shouldAllocOrReserveNewContainer(priority, required)) {
+      if (!shouldAllocOrReserveNewContainer(schedulerKey, required)) {
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("doesn't need containers based on reservation algo!");
           LOG.debug("doesn't need containers based on reservation algo!");
         }
         }
@@ -143,7 +146,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     }
     }
 
 
     // Inform the application it is about to get a scheduling opportunity
     // Inform the application it is about to get a scheduling opportunity
-    application.addSchedulingOpportunity(priority);
+    application.addSchedulingOpportunity(schedulerKey);
 
 
     // Increase missed-non-partitioned-resource-request-opportunity.
     // Increase missed-non-partitioned-resource-request-opportunity.
     // This is to make sure non-partitioned-resource-request will prefer
     // This is to make sure non-partitioned-resource-request will prefer
@@ -152,8 +155,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     if (anyRequest.getNodeLabelExpression()
     if (anyRequest.getNodeLabelExpression()
         .equals(RMNodeLabelsManager.NO_LABEL)) {
         .equals(RMNodeLabelsManager.NO_LABEL)) {
       missedNonPartitionedRequestSchedulingOpportunity =
       missedNonPartitionedRequestSchedulingOpportunity =
-          application
-              .addMissedNonPartitionedRequestSchedulingOpportunity(priority);
+          application.addMissedNonPartitionedRequestSchedulingOpportunity(
+              schedulerKey);
     }
     }
 
 
     if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
     if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
@@ -164,7 +167,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
           .getScheduler().getNumClusterNodes()) {
           .getScheduler().getNumClusterNodes()) {
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Skip app_attempt=" + application.getApplicationAttemptId()
           LOG.debug("Skip app_attempt=" + application.getApplicationAttemptId()
-              + " priority=" + priority
+              + " priority=" + schedulerKey.getPriority()
               + " because missed-non-partitioned-resource-request"
               + " because missed-non-partitioned-resource-request"
               + " opportunity under requred:" + " Now="
               + " opportunity under requred:" + " Now="
               + missedNonPartitionedRequestSchedulingOpportunity + " required="
               + missedNonPartitionedRequestSchedulingOpportunity + " required="
@@ -180,20 +183,20 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
   ContainerAllocation preAllocation(Resource clusterResource,
   ContainerAllocation preAllocation(Resource clusterResource,
       FiCaSchedulerNode node, SchedulingMode schedulingMode,
       FiCaSchedulerNode node, SchedulingMode schedulingMode,
-      ResourceLimits resourceLimits, Priority priority,
+      ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey,
       RMContainer reservedContainer) {
       RMContainer reservedContainer) {
     ContainerAllocation result;
     ContainerAllocation result;
     if (null == reservedContainer) {
     if (null == reservedContainer) {
       // pre-check when allocating new container
       // pre-check when allocating new container
       result =
       result =
           preCheckForNewContainer(clusterResource, node, schedulingMode,
           preCheckForNewContainer(clusterResource, node, schedulingMode,
-              resourceLimits, priority);
+              resourceLimits, schedulerKey);
       if (null != result) {
       if (null != result) {
         return result;
         return result;
       }
       }
     } else {
     } else {
       // pre-check when allocating reserved container
       // pre-check when allocating reserved container
-      if (application.getTotalRequiredResources(priority) == 0) {
+      if (application.getTotalRequiredResources(schedulerKey) == 0) {
         // Release
         // Release
         return new ContainerAllocation(reservedContainer, null,
         return new ContainerAllocation(reservedContainer, null,
             AllocationState.QUEUE_SKIPPED);
             AllocationState.QUEUE_SKIPPED);
@@ -202,13 +205,13 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
     // Try to allocate containers on node
     // Try to allocate containers on node
     result =
     result =
-        assignContainersOnNode(clusterResource, node, priority,
+        assignContainersOnNode(clusterResource, node, schedulerKey,
             reservedContainer, schedulingMode, resourceLimits);
             reservedContainer, schedulingMode, resourceLimits);
     
     
     if (null == reservedContainer) {
     if (null == reservedContainer) {
       if (result.state == AllocationState.PRIORITY_SKIPPED) {
       if (result.state == AllocationState.PRIORITY_SKIPPED) {
         // Don't count 'skipped nodes' as a scheduling opportunity!
         // Don't count 'skipped nodes' as a scheduling opportunity!
-        application.subtractSchedulingOpportunity(priority);
+        application.subtractSchedulingOpportunity(schedulerKey);
       }
       }
     }
     }
     
     
@@ -216,10 +219,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
   }
   
   
   public synchronized float getLocalityWaitFactor(
   public synchronized float getLocalityWaitFactor(
-      Priority priority, int clusterNodes) {
+      SchedulerRequestKey schedulerKey, int clusterNodes) {
     // Estimate: Required unique resources (i.e. hosts + racks)
     // Estimate: Required unique resources (i.e. hosts + racks)
     int requiredResources = 
     int requiredResources = 
-        Math.max(application.getResourceRequests(priority).size() - 1, 0);
+        Math.max(application.getResourceRequests(schedulerKey).size() - 1, 0);
     
     
     // waitFactor can't be more than '1' 
     // waitFactor can't be more than '1' 
     // i.e. no point skipping more than clustersize opportunities
     // i.e. no point skipping more than clustersize opportunities
@@ -231,8 +234,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
         .getCSLeafQueue().getNodeLocalityDelay());
         .getCSLeafQueue().getNodeLocalityDelay());
   }
   }
 
 
-  private boolean canAssign(Priority priority, FiCaSchedulerNode node,
-      NodeType type, RMContainer reservedContainer) {
+  private boolean canAssign(SchedulerRequestKey schedulerKey,
+      FiCaSchedulerNode node, NodeType type, RMContainer reservedContainer) {
 
 
     // Clearly we need containers for this application...
     // Clearly we need containers for this application...
     if (type == NodeType.OFF_SWITCH) {
     if (type == NodeType.OFF_SWITCH) {
@@ -242,15 +245,16 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
       // 'Delay' off-switch
       // 'Delay' off-switch
       ResourceRequest offSwitchRequest =
       ResourceRequest offSwitchRequest =
-          application.getResourceRequest(priority, ResourceRequest.ANY);
-      long missedOpportunities = application.getSchedulingOpportunities(priority);
+          application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
+      long missedOpportunities =
+          application.getSchedulingOpportunities(schedulerKey);
       long requiredContainers = offSwitchRequest.getNumContainers();
       long requiredContainers = offSwitchRequest.getNumContainers();
 
 
       float localityWaitFactor =
       float localityWaitFactor =
-          getLocalityWaitFactor(priority, rmContext.getScheduler()
+          getLocalityWaitFactor(schedulerKey, rmContext.getScheduler()
               .getNumClusterNodes());
               .getNumClusterNodes());
-      // Cap the delay by the number of nodes in the cluster. Under most conditions
-      // this means we will consider each node in the cluster before
+      // Cap the delay by the number of nodes in the cluster. Under most
+      // conditions this means we will consider each node in the cluster before
       // accepting an off-switch assignment.
       // accepting an off-switch assignment.
       return (Math.min(rmContext.getScheduler().getNumClusterNodes(),
       return (Math.min(rmContext.getScheduler().getNumClusterNodes(),
           (requiredContainers * localityWaitFactor)) < missedOpportunities);
           (requiredContainers * localityWaitFactor)) < missedOpportunities);
@@ -258,7 +262,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
     // Check if we need containers on this rack
     // Check if we need containers on this rack
     ResourceRequest rackLocalRequest =
     ResourceRequest rackLocalRequest =
-        application.getResourceRequest(priority, node.getRackName());
+        application.getResourceRequest(schedulerKey, node.getRackName());
     if (rackLocalRequest == null || rackLocalRequest.getNumContainers() <= 0) {
     if (rackLocalRequest == null || rackLocalRequest.getNumContainers() <= 0) {
       return false;
       return false;
     }
     }
@@ -266,7 +270,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     // If we are here, we do need containers on this rack for RACK_LOCAL req
     // If we are here, we do need containers on this rack for RACK_LOCAL req
     if (type == NodeType.RACK_LOCAL) {
     if (type == NodeType.RACK_LOCAL) {
       // 'Delay' rack-local just a little bit...
       // 'Delay' rack-local just a little bit...
-      long missedOpportunities = application.getSchedulingOpportunities(priority);
+      long missedOpportunities =
+          application.getSchedulingOpportunities(schedulerKey);
       return getActualNodeLocalityDelay() < missedOpportunities;
       return getActualNodeLocalityDelay() < missedOpportunities;
     }
     }
 
 
@@ -274,7 +279,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     if (type == NodeType.NODE_LOCAL) {
     if (type == NodeType.NODE_LOCAL) {
       // Now check if we need containers on this host...
       // Now check if we need containers on this host...
       ResourceRequest nodeLocalRequest =
       ResourceRequest nodeLocalRequest =
-          application.getResourceRequest(priority, node.getNodeName());
+          application.getResourceRequest(schedulerKey, node.getNodeName());
       if (nodeLocalRequest != null) {
       if (nodeLocalRequest != null) {
         return nodeLocalRequest.getNumContainers() > 0;
         return nodeLocalRequest.getNumContainers() > 0;
       }
       }
@@ -285,10 +290,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
   private ContainerAllocation assignNodeLocalContainers(
   private ContainerAllocation assignNodeLocalContainers(
       Resource clusterResource, ResourceRequest nodeLocalResourceRequest,
       Resource clusterResource, ResourceRequest nodeLocalResourceRequest,
-      FiCaSchedulerNode node, Priority priority, RMContainer reservedContainer,
-      SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
-    if (canAssign(priority, node, NodeType.NODE_LOCAL, reservedContainer)) {
-      return assignContainer(clusterResource, node, priority,
+      FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
+      RMContainer reservedContainer, SchedulingMode schedulingMode,
+      ResourceLimits currentResoureLimits) {
+    if (canAssign(schedulerKey, node, NodeType.NODE_LOCAL, reservedContainer)) {
+      return assignContainer(clusterResource, node, schedulerKey,
           nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer,
           nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer,
           schedulingMode, currentResoureLimits);
           schedulingMode, currentResoureLimits);
     }
     }
@@ -299,10 +305,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
   private ContainerAllocation assignRackLocalContainers(
   private ContainerAllocation assignRackLocalContainers(
       Resource clusterResource, ResourceRequest rackLocalResourceRequest,
       Resource clusterResource, ResourceRequest rackLocalResourceRequest,
-      FiCaSchedulerNode node, Priority priority, RMContainer reservedContainer,
-      SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
-    if (canAssign(priority, node, NodeType.RACK_LOCAL, reservedContainer)) {
-      return assignContainer(clusterResource, node, priority,
+      FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
+      RMContainer reservedContainer, SchedulingMode schedulingMode,
+      ResourceLimits currentResoureLimits) {
+    if (canAssign(schedulerKey, node, NodeType.RACK_LOCAL, reservedContainer)) {
+      return assignContainer(clusterResource, node, schedulerKey,
           rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer,
           rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer,
           schedulingMode, currentResoureLimits);
           schedulingMode, currentResoureLimits);
     }
     }
@@ -313,10 +320,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
   private ContainerAllocation assignOffSwitchContainers(
   private ContainerAllocation assignOffSwitchContainers(
       Resource clusterResource, ResourceRequest offSwitchResourceRequest,
       Resource clusterResource, ResourceRequest offSwitchResourceRequest,
-      FiCaSchedulerNode node, Priority priority, RMContainer reservedContainer,
-      SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
-    if (canAssign(priority, node, NodeType.OFF_SWITCH, reservedContainer)) {
-      return assignContainer(clusterResource, node, priority,
+      FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
+      RMContainer reservedContainer, SchedulingMode schedulingMode,
+      ResourceLimits currentResoureLimits) {
+    if (canAssign(schedulerKey, node, NodeType.OFF_SWITCH, reservedContainer)) {
+      return assignContainer(clusterResource, node, schedulerKey,
           offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer,
           offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer,
           schedulingMode, currentResoureLimits);
           schedulingMode, currentResoureLimits);
     }
     }
@@ -327,20 +335,21 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
   }
 
 
   private ContainerAllocation assignContainersOnNode(Resource clusterResource,
   private ContainerAllocation assignContainersOnNode(Resource clusterResource,
-      FiCaSchedulerNode node, Priority priority, RMContainer reservedContainer,
-      SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
+      FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
+      RMContainer reservedContainer, SchedulingMode schedulingMode,
+      ResourceLimits currentResoureLimits) {
 
 
     ContainerAllocation allocation;
     ContainerAllocation allocation;
 
 
     NodeType requestType = null;
     NodeType requestType = null;
     // Data-local
     // Data-local
     ResourceRequest nodeLocalResourceRequest =
     ResourceRequest nodeLocalResourceRequest =
-        application.getResourceRequest(priority, node.getNodeName());
+        application.getResourceRequest(schedulerKey, node.getNodeName());
     if (nodeLocalResourceRequest != null) {
     if (nodeLocalResourceRequest != null) {
       requestType = NodeType.NODE_LOCAL;
       requestType = NodeType.NODE_LOCAL;
       allocation =
       allocation =
           assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest,
           assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest,
-              node, priority, reservedContainer, schedulingMode,
+              node, schedulerKey, reservedContainer, schedulingMode,
               currentResoureLimits);
               currentResoureLimits);
       if (Resources.greaterThan(rc, clusterResource,
       if (Resources.greaterThan(rc, clusterResource,
           allocation.getResourceToBeAllocated(), Resources.none())) {
           allocation.getResourceToBeAllocated(), Resources.none())) {
@@ -351,7 +360,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
     // Rack-local
     // Rack-local
     ResourceRequest rackLocalResourceRequest =
     ResourceRequest rackLocalResourceRequest =
-        application.getResourceRequest(priority, node.getRackName());
+        application.getResourceRequest(schedulerKey, node.getRackName());
     if (rackLocalResourceRequest != null) {
     if (rackLocalResourceRequest != null) {
       if (!rackLocalResourceRequest.getRelaxLocality()) {
       if (!rackLocalResourceRequest.getRelaxLocality()) {
         return ContainerAllocation.PRIORITY_SKIPPED;
         return ContainerAllocation.PRIORITY_SKIPPED;
@@ -363,7 +372,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
       allocation =
       allocation =
           assignRackLocalContainers(clusterResource, rackLocalResourceRequest,
           assignRackLocalContainers(clusterResource, rackLocalResourceRequest,
-              node, priority, reservedContainer, schedulingMode,
+              node, schedulerKey, reservedContainer, schedulingMode,
               currentResoureLimits);
               currentResoureLimits);
       if (Resources.greaterThan(rc, clusterResource,
       if (Resources.greaterThan(rc, clusterResource,
           allocation.getResourceToBeAllocated(), Resources.none())) {
           allocation.getResourceToBeAllocated(), Resources.none())) {
@@ -374,7 +383,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
     // Off-switch
     // Off-switch
     ResourceRequest offSwitchResourceRequest =
     ResourceRequest offSwitchResourceRequest =
-        application.getResourceRequest(priority, ResourceRequest.ANY);
+        application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
     if (offSwitchResourceRequest != null) {
     if (offSwitchResourceRequest != null) {
       if (!offSwitchResourceRequest.getRelaxLocality()) {
       if (!offSwitchResourceRequest.getRelaxLocality()) {
         return ContainerAllocation.PRIORITY_SKIPPED;
         return ContainerAllocation.PRIORITY_SKIPPED;
@@ -386,7 +395,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
       allocation =
       allocation =
           assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
           assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
-              node, priority, reservedContainer, schedulingMode,
+              node, schedulerKey, reservedContainer, schedulingMode,
               currentResoureLimits);
               currentResoureLimits);
       allocation.requestNodeType = requestType;
       allocation.requestNodeType = requestType;
       
       
@@ -403,21 +412,22 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
   }
 
 
   private ContainerAllocation assignContainer(Resource clusterResource,
   private ContainerAllocation assignContainer(Resource clusterResource,
-      FiCaSchedulerNode node, Priority priority, ResourceRequest request,
-      NodeType type, RMContainer rmContainer, SchedulingMode schedulingMode,
-      ResourceLimits currentResoureLimits) {
+      FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
+      ResourceRequest request, NodeType type, RMContainer rmContainer,
+      SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
     lastResourceRequest = request;
     lastResourceRequest = request;
     
     
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("assignContainers: node=" + node.getNodeName()
       LOG.debug("assignContainers: node=" + node.getNodeName()
-        + " application=" + application.getApplicationId()
-        + " priority=" + priority.getPriority()
-        + " request=" + request + " type=" + type);
+          + " application=" + application.getApplicationId()
+          + " priority=" + schedulerKey.getPriority()
+          + " request=" + request + " type=" + type);
     }
     }
 
 
     // check if the resource request can access the label
     // check if the resource request can access the label
     if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
     if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
-        request.getNodeLabelExpression(), node.getPartition(), schedulingMode)) {
+        request.getNodeLabelExpression(), node.getPartition(),
+        schedulingMode)) {
       // this is a reserved container, but we cannot allocate it now according
       // this is a reserved container, but we cannot allocate it now according
       // to label not match. This can be caused by node label changed
       // to label not match. This can be caused by node label changed
       // We should un-reserve this container.
       // We should un-reserve this container.
@@ -439,7 +449,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     }
     }
 
 
     boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
     boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
-        priority, capability);
+        schedulerKey, capability);
 
 
     // Can we allocate a container on this node?
     // Can we allocate a container on this node?
     long availableContainers =
     long availableContainers =
@@ -504,8 +514,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
             resourceNeedToUnReserve = capability;
             resourceNeedToUnReserve = capability;
           }
           }
           unreservedContainer =
           unreservedContainer =
-              application.findNodeToUnreserve(clusterResource, node, priority,
-                  resourceNeedToUnReserve);
+              application.findNodeToUnreserve(clusterResource, node,
+                  schedulerKey, resourceNeedToUnReserve);
           // When (minimum-unreserved-resource > 0 OR we cannot allocate
           // When (minimum-unreserved-resource > 0 OR we cannot allocate
           // new/reserved
           // new/reserved
           // container (That means we *have to* unreserve some resource to
           // container (That means we *have to* unreserve some resource to
@@ -553,28 +563,28 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     }
     }
   }
   }
 
 
-  boolean
-      shouldAllocOrReserveNewContainer(Priority priority, Resource required) {
-    int requiredContainers = application.getTotalRequiredResources(priority);
-    int reservedContainers = application.getNumReservedContainers(priority);
+  boolean shouldAllocOrReserveNewContainer(
+      SchedulerRequestKey schedulerKey, Resource required) {
+    int requiredContainers =
+        application.getTotalRequiredResources(schedulerKey);
+    int reservedContainers = application.getNumReservedContainers(schedulerKey);
     int starvation = 0;
     int starvation = 0;
     if (reservedContainers > 0) {
     if (reservedContainers > 0) {
-      float nodeFactor =
-          Resources
-              .ratio(rc, required, application.getCSLeafQueue().getMaximumAllocation());
+      float nodeFactor = Resources.ratio(
+          rc, required, application.getCSLeafQueue().getMaximumAllocation());
 
 
       // Use percentage of node required to bias against large containers...
       // Use percentage of node required to bias against large containers...
       // Protect against corner case where you need the whole node with
       // Protect against corner case where you need the whole node with
       // Math.min(nodeFactor, minimumAllocationFactor)
       // Math.min(nodeFactor, minimumAllocationFactor)
       starvation =
       starvation =
-          (int) ((application.getReReservations(priority) / 
+          (int) ((application.getReReservations(schedulerKey) /
               (float) reservedContainers) * (1.0f - (Math.min(
               (float) reservedContainers) * (1.0f - (Math.min(
                   nodeFactor, application.getCSLeafQueue()
                   nodeFactor, application.getCSLeafQueue()
                   .getMinimumAllocationFactor()))));
                   .getMinimumAllocationFactor()))));
 
 
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("needsContainers:" + " app.#re-reserve="
         LOG.debug("needsContainers:" + " app.#re-reserve="
-            + application.getReReservations(priority) + " reserved="
+            + application.getReReservations(schedulerKey) + " reserved="
             + reservedContainers + " nodeFactor=" + nodeFactor
             + reservedContainers + " nodeFactor=" + nodeFactor
             + " minAllocFactor="
             + " minAllocFactor="
             + application.getCSLeafQueue().getMinimumAllocationFactor()
             + application.getCSLeafQueue().getMinimumAllocationFactor()
@@ -585,13 +595,14 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
   }
   
   
   private Container getContainer(RMContainer rmContainer,
   private Container getContainer(RMContainer rmContainer,
-      FiCaSchedulerNode node, Resource capability, Priority priority) {
+      FiCaSchedulerNode node, Resource capability,
+      SchedulerRequestKey schedulerKey) {
     return (rmContainer != null) ? rmContainer.getContainer()
     return (rmContainer != null) ? rmContainer.getContainer()
-        : createContainer(node, capability, priority);
+        : createContainer(node, capability, schedulerKey);
   }
   }
 
 
   private Container createContainer(FiCaSchedulerNode node, Resource capability,
   private Container createContainer(FiCaSchedulerNode node, Resource capability,
-      Priority priority) {
+      SchedulerRequestKey schedulerKey) {
 
 
     NodeId nodeId = node.getRMNode().getNodeID();
     NodeId nodeId = node.getRMNode().getNodeID();
     ContainerId containerId =
     ContainerId containerId =
@@ -600,22 +611,23 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
 
 
     // Create the container
     // Create the container
     return BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
     return BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
-        .getHttpAddress(), capability, priority, null);
+        .getHttpAddress(), capability, schedulerKey.getPriority(), null);
   }
   }
   
   
   private ContainerAllocation handleNewContainerAllocation(
   private ContainerAllocation handleNewContainerAllocation(
       ContainerAllocation allocationResult, FiCaSchedulerNode node,
       ContainerAllocation allocationResult, FiCaSchedulerNode node,
-      Priority priority, RMContainer reservedContainer, Container container) {
+      SchedulerRequestKey schedulerKey, RMContainer reservedContainer,
+      Container container) {
     // Handling container allocation
     // Handling container allocation
     // Did we previously reserve containers at this 'priority'?
     // Did we previously reserve containers at this 'priority'?
     if (reservedContainer != null) {
     if (reservedContainer != null) {
-      application.unreserve(priority, node, reservedContainer);
+      application.unreserve(schedulerKey, node, reservedContainer);
     }
     }
     
     
     // Inform the application
     // Inform the application
     RMContainer allocatedContainer =
     RMContainer allocatedContainer =
         application.allocate(allocationResult.containerNodeType, node,
         application.allocate(allocationResult.containerNodeType, node,
-            priority, lastResourceRequest, container);
+            schedulerKey, lastResourceRequest, container);
 
 
     // Does the application need this resource?
     // Does the application need this resource?
     if (allocatedContainer == null) {
     if (allocatedContainer == null) {
@@ -637,12 +649,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   }
   }
 
 
   ContainerAllocation doAllocation(ContainerAllocation allocationResult,
   ContainerAllocation doAllocation(ContainerAllocation allocationResult,
-      FiCaSchedulerNode node, Priority priority,
+      FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
       RMContainer reservedContainer) {
       RMContainer reservedContainer) {
     // Create the container if necessary
     // Create the container if necessary
     Container container =
     Container container =
         getContainer(reservedContainer, node,
         getContainer(reservedContainer, node,
-            allocationResult.getResourceToBeAllocated(), priority);
+            allocationResult.getResourceToBeAllocated(), schedulerKey);
 
 
     // something went wrong getting/creating the container
     // something went wrong getting/creating the container
     if (container == null) {
     if (container == null) {
@@ -655,11 +667,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     if (allocationResult.getAllocationState() == AllocationState.ALLOCATED) {
     if (allocationResult.getAllocationState() == AllocationState.ALLOCATED) {
       // When allocating container
       // When allocating container
       allocationResult =
       allocationResult =
-          handleNewContainerAllocation(allocationResult, node, priority,
+          handleNewContainerAllocation(allocationResult, node, schedulerKey,
               reservedContainer, container);
               reservedContainer, container);
     } else {
     } else {
       // When reserving container
       // When reserving container
-      application.reserve(priority, node, reservedContainer, container);
+      application.reserve(schedulerKey, node, reservedContainer, container);
     }
     }
     allocationResult.updatedContainer = container;
     allocationResult.updatedContainer = container;
 
 
@@ -678,14 +690,15 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
         // RACK_LOCAL without delay.
         // RACK_LOCAL without delay.
         if (allocationResult.containerNodeType == NodeType.NODE_LOCAL
         if (allocationResult.containerNodeType == NodeType.NODE_LOCAL
             || application.getCSLeafQueue().getRackLocalityFullReset()) {
             || application.getCSLeafQueue().getRackLocalityFullReset()) {
-          application.resetSchedulingOpportunities(priority);
+          application.resetSchedulingOpportunities(schedulerKey);
         }
         }
       }
       }
 
 
       // Non-exclusive scheduling opportunity is different: we need reset
       // Non-exclusive scheduling opportunity is different: we need reset
       // it every time to make sure non-labeled resource request will be
       // it every time to make sure non-labeled resource request will be
       // most likely allocated on non-labeled nodes first.
       // most likely allocated on non-labeled nodes first.
-      application.resetMissedNonPartitionedRequestSchedulingOpportunity(priority);
+      application.resetMissedNonPartitionedRequestSchedulingOpportunity(
+          schedulerKey);
     }
     }
 
 
     return allocationResult;
     return allocationResult;
@@ -693,15 +706,15 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   
   
   private ContainerAllocation allocate(Resource clusterResource,
   private ContainerAllocation allocate(Resource clusterResource,
       FiCaSchedulerNode node, SchedulingMode schedulingMode,
       FiCaSchedulerNode node, SchedulingMode schedulingMode,
-      ResourceLimits resourceLimits, Priority priority,
+      ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey,
       RMContainer reservedContainer) {
       RMContainer reservedContainer) {
     ContainerAllocation result =
     ContainerAllocation result =
         preAllocation(clusterResource, node, schedulingMode, resourceLimits,
         preAllocation(clusterResource, node, schedulingMode, resourceLimits,
-            priority, reservedContainer);
+            schedulerKey, reservedContainer);
 
 
     if (AllocationState.ALLOCATED == result.state
     if (AllocationState.ALLOCATED == result.state
         || AllocationState.RESERVED == result.state) {
         || AllocationState.RESERVED == result.state) {
-      result = doAllocation(result, node, priority, reservedContainer);
+      result = doAllocation(result, node, schedulerKey, reservedContainer);
     }
     }
 
 
     return result;
     return result;
@@ -725,10 +738,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
       }
       }
       
       
       // Schedule in priority order
       // Schedule in priority order
-      for (Priority priority : application.getPriorities()) {
+      for (SchedulerRequestKey schedulerKey : application.getSchedulerKeys()) {
         ContainerAllocation result =
         ContainerAllocation result =
             allocate(clusterResource, node, schedulingMode, resourceLimits,
             allocate(clusterResource, node, schedulingMode, resourceLimits,
-                priority, null);
+                schedulerKey, null);
 
 
         AllocationState allocationState = result.getAllocationState();
         AllocationState allocationState = result.getAllocationState();
         if (allocationState == AllocationState.PRIORITY_SKIPPED) {
         if (allocationState == AllocationState.PRIORITY_SKIPPED) {
@@ -744,7 +757,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     } else {
     } else {
       ContainerAllocation result =
       ContainerAllocation result =
           allocate(clusterResource, node, schedulingMode, resourceLimits,
           allocate(clusterResource, node, schedulingMode, resourceLimits,
-              reservedContainer.getReservedPriority(), reservedContainer);
+              reservedContainer.getReservedSchedulerKey(), reservedContainer);
       return getCSAssignmentFromAllocateResult(clusterResource, result,
       return getCSAssignmentFromAllocateResult(clusterResource, result,
           reservedContainer);
           reservedContainer);
     }
     }

+ 30 - 26
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

@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMContainerLaunchDiagnosticsConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMContainerLaunchDiagnosticsConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
@@ -181,7 +182,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
   }
 
 
   public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
   public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
-      Priority priority, ResourceRequest request, 
+      SchedulerRequestKey schedulerKey, ResourceRequest request,
       Container container) {
       Container container) {
 
 
     if (isStopped) {
     if (isStopped) {
@@ -190,10 +191,10 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     
     
     // Required sanity check - AM can call 'allocate' to update resource 
     // Required sanity check - AM can call 'allocate' to update resource 
     // request without locking the scheduler, hence we need to check
     // request without locking the scheduler, hence we need to check
-    if (getTotalRequiredResources(priority) <= 0) {
+    if (getTotalRequiredResources(schedulerKey) <= 0) {
       return null;
       return null;
     }
     }
-    
+
     // Create RMContainer
     // Create RMContainer
     RMContainer rmContainer =
     RMContainer rmContainer =
         new RMContainerImpl(container, this.getApplicationAttemptId(),
         new RMContainerImpl(container, this.getApplicationAttemptId(),
@@ -211,7 +212,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
 
     // Update consumption and track allocations
     // Update consumption and track allocations
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
-        type, node, priority, request, container);
+        type, node, schedulerKey, request, container);
 
 
     attemptResourceUsage.incUsed(node.getPartition(), container.getResource());
     attemptResourceUsage.incUsed(node.getPartition(), container.getResource());
 
 
@@ -235,13 +236,13 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return rmContainer;
     return rmContainer;
   }
   }
 
 
-  public synchronized boolean unreserve(Priority priority,
+  public synchronized boolean unreserve(SchedulerRequestKey schedulerKey,
       FiCaSchedulerNode node, RMContainer rmContainer) {
       FiCaSchedulerNode node, RMContainer rmContainer) {
     // Cancel increase request (if it has reserved increase request 
     // Cancel increase request (if it has reserved increase request 
     rmContainer.cancelIncreaseReservation();
     rmContainer.cancelIncreaseReservation();
     
     
     // Done with the reservation?
     // Done with the reservation?
-    if (internalUnreserve(node, priority)) {
+    if (internalUnreserve(node, schedulerKey)) {
       node.unreserveResource(this);
       node.unreserveResource(this);
 
 
       // Update reserved metrics
       // Update reserved metrics
@@ -254,12 +255,14 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return false;
     return false;
   }
   }
 
 
-  private boolean internalUnreserve(FiCaSchedulerNode node, Priority priority) {
+  private boolean internalUnreserve(FiCaSchedulerNode node,
+      SchedulerRequestKey schedulerKey) {
     Map<NodeId, RMContainer> reservedContainers =
     Map<NodeId, RMContainer> reservedContainers =
-      this.reservedContainers.get(priority);
+        this.reservedContainers.get(schedulerKey);
 
 
     if (reservedContainers != null) {
     if (reservedContainers != null) {
-      RMContainer reservedContainer = reservedContainers.remove(node.getNodeID());
+      RMContainer reservedContainer =
+          reservedContainers.remove(node.getNodeID());
 
 
       // unreserve is now triggered in new scenarios (preemption)
       // unreserve is now triggered in new scenarios (preemption)
       // as a consequence reservedcontainer might be null, adding NP-checks
       // as a consequence reservedcontainer might be null, adding NP-checks
@@ -268,17 +271,18 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
           && reservedContainer.getContainer().getResource() != null) {
           && reservedContainer.getContainer().getResource() != null) {
 
 
         if (reservedContainers.isEmpty()) {
         if (reservedContainers.isEmpty()) {
-          this.reservedContainers.remove(priority);
+          this.reservedContainers.remove(schedulerKey);
         }
         }
         // Reset the re-reservation count
         // Reset the re-reservation count
-        resetReReservations(priority);
+        resetReReservations(schedulerKey);
 
 
         Resource resource = reservedContainer.getReservedResource();
         Resource resource = reservedContainer.getReservedResource();
         this.attemptResourceUsage.decReserved(node.getPartition(), resource);
         this.attemptResourceUsage.decReserved(node.getPartition(), resource);
 
 
         LOG.info("Application " + getApplicationId() + " unreserved "
         LOG.info("Application " + getApplicationId() + " unreserved "
             + " on node " + node + ", currently has "
             + " on node " + node + ", currently has "
-            + reservedContainers.size() + " at priority " + priority
+            + reservedContainers.size()
+            + " at priority " + schedulerKey.getPriority()
             + "; currentReservation " + this.attemptResourceUsage.getReserved()
             + "; currentReservation " + this.attemptResourceUsage.getReserved()
             + " on node-label=" + node.getPartition());
             + " on node-label=" + node.getPartition());
         return true;
         return true;
@@ -288,10 +292,10 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
   }
 
 
   public synchronized float getLocalityWaitFactor(
   public synchronized float getLocalityWaitFactor(
-      Priority priority, int clusterNodes) {
+      SchedulerRequestKey schedulerKey, int clusterNodes) {
     // Estimate: Required unique resources (i.e. hosts + racks)
     // Estimate: Required unique resources (i.e. hosts + racks)
     int requiredResources = 
     int requiredResources = 
-        Math.max(this.getResourceRequests(priority).size() - 1, 0);
+        Math.max(this.getResourceRequests(schedulerKey).size() - 1, 0);
     
     
     // waitFactor can't be more than '1' 
     // waitFactor can't be more than '1' 
     // i.e. no point skipping more than clustersize opportunities
     // i.e. no point skipping more than clustersize opportunities
@@ -354,14 +358,14 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
         newlyIncreasedContainers, newlyDecreasedContainers);
         newlyIncreasedContainers, newlyDecreasedContainers);
   }
   }
   
   
-  synchronized public NodeId getNodeIdToUnreserve(Priority priority,
-      Resource resourceNeedUnreserve, ResourceCalculator rc,
-      Resource clusterResource) {
+  synchronized public NodeId getNodeIdToUnreserve(
+      SchedulerRequestKey schedulerKey, Resource resourceNeedUnreserve,
+      ResourceCalculator rc, Resource clusterResource) {
 
 
     // first go around make this algorithm simple and just grab first
     // first go around make this algorithm simple and just grab first
     // reservation that has enough resources
     // reservation that has enough resources
     Map<NodeId, RMContainer> reservedContainers = this.reservedContainers
     Map<NodeId, RMContainer> reservedContainers = this.reservedContainers
-        .get(priority);
+        .get(schedulerKey);
 
 
     if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
     if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
       for (Map.Entry<NodeId, RMContainer> entry : reservedContainers.entrySet()) {
       for (Map.Entry<NodeId, RMContainer> entry : reservedContainers.entrySet()) {
@@ -417,17 +421,17 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       ((FiCaSchedulerApp) appAttempt).getHeadroomProvider();
       ((FiCaSchedulerApp) appAttempt).getHeadroomProvider();
   }
   }
   
   
-  public boolean reserveIncreasedContainer(Priority priority, 
+  public boolean reserveIncreasedContainer(SchedulerRequestKey schedulerKey,
       FiCaSchedulerNode node,
       FiCaSchedulerNode node,
       RMContainer rmContainer, Resource reservedResource) {
       RMContainer rmContainer, Resource reservedResource) {
     // Inform the application
     // Inform the application
-    if (super.reserveIncreasedContainer(node, priority, rmContainer,
+    if (super.reserveIncreasedContainer(node, schedulerKey, rmContainer,
         reservedResource)) {
         reservedResource)) {
 
 
       queue.getMetrics().reserveResource(getUser(), reservedResource);
       queue.getMetrics().reserveResource(getUser(), reservedResource);
 
 
       // Update the node
       // Update the node
-      node.reserveResource(this, priority, rmContainer);
+      node.reserveResource(this, schedulerKey, rmContainer);
       
       
       // Succeeded
       // Succeeded
       return true;
       return true;
@@ -436,7 +440,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return false;
     return false;
   }
   }
 
 
-  public void reserve(Priority priority,
+  public void reserve(SchedulerRequestKey schedulerKey,
       FiCaSchedulerNode node, RMContainer rmContainer, Container container) {
       FiCaSchedulerNode node, RMContainer rmContainer, Container container) {
     // Update reserved metrics if this is the first reservation
     // Update reserved metrics if this is the first reservation
     if (rmContainer == null) {
     if (rmContainer == null) {
@@ -445,19 +449,19 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
     }
 
 
     // Inform the application
     // Inform the application
-    rmContainer = super.reserve(node, priority, rmContainer, container);
+    rmContainer = super.reserve(node, schedulerKey, rmContainer, container);
 
 
     // Update the node
     // Update the node
-    node.reserveResource(this, priority, rmContainer);
+    node.reserveResource(this, schedulerKey, rmContainer);
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
   public RMContainer findNodeToUnreserve(Resource clusterResource,
   public RMContainer findNodeToUnreserve(Resource clusterResource,
-      FiCaSchedulerNode node, Priority priority,
+      FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
       Resource minimumUnreservedResource) {
       Resource minimumUnreservedResource) {
     // need to unreserve some other container first
     // need to unreserve some other container first
     NodeId idToUnreserve =
     NodeId idToUnreserve =
-        getNodeIdToUnreserve(priority, minimumUnreservedResource,
+        getNodeIdToUnreserve(schedulerKey, minimumUnreservedResource,
             rc, clusterResource);
             rc, clusterResource);
     if (idToUnreserve == null) {
     if (idToUnreserve == null) {
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java

@@ -23,12 +23,13 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 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.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 
@@ -54,7 +55,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
 
 
   @Override
   @Override
   public synchronized void reserveResource(
   public synchronized void reserveResource(
-      SchedulerApplicationAttempt application, Priority priority,
+      SchedulerApplicationAttempt application, SchedulerRequestKey priority,
       RMContainer container) {
       RMContainer container) {
     // Check if it's already reserved
     // Check if it's already reserved
     RMContainer reservedContainer = getReservedContainer();
     RMContainer reservedContainer = getReservedContainer();

+ 119 - 94
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@@ -97,8 +98,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * at the current allowed level and the time since the last container
    * at the current allowed level and the time since the last container
    * was scheduled. Currently we use only the former.
    * was scheduled. Currently we use only the former.
    */
    */
-  private final Map<Priority, NodeType> allowedLocalityLevel =
-      new HashMap<Priority, NodeType>();
+  private final Map<SchedulerRequestKey, NodeType> allowedLocalityLevel =
+      new HashMap<>();
 
 
   public FSAppAttempt(FairScheduler scheduler,
   public FSAppAttempt(FairScheduler scheduler,
       ApplicationAttemptId applicationAttemptId, String user, FSLeafQueue queue,
       ApplicationAttemptId applicationAttemptId, String user, FSLeafQueue queue,
@@ -163,23 +164,23 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   }
   }
 
 
   private synchronized void unreserveInternal(
   private synchronized void unreserveInternal(
-      Priority priority, FSSchedulerNode node) {
+      SchedulerRequestKey schedulerKey, FSSchedulerNode node) {
     Map<NodeId, RMContainer> reservedContainers = 
     Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(priority);
+        this.reservedContainers.get(schedulerKey);
     RMContainer reservedContainer = reservedContainers.remove(node.getNodeID());
     RMContainer reservedContainer = reservedContainers.remove(node.getNodeID());
     if (reservedContainers.isEmpty()) {
     if (reservedContainers.isEmpty()) {
-      this.reservedContainers.remove(priority);
+      this.reservedContainers.remove(schedulerKey);
     }
     }
     
     
     // Reset the re-reservation count
     // Reset the re-reservation count
-    resetReReservations(priority);
+    resetReReservations(schedulerKey);
 
 
     Resource resource = reservedContainer.getContainer().getResource();
     Resource resource = reservedContainer.getContainer().getResource();
     this.attemptResourceUsage.decReserved(resource);
     this.attemptResourceUsage.decReserved(resource);
 
 
     LOG.info("Application " + getApplicationId() + " unreserved " + " on node "
     LOG.info("Application " + getApplicationId() + " unreserved " + " on node "
         + node + ", currently has " + reservedContainers.size()
         + node + ", currently has " + reservedContainers.size()
-        + " at priority " + priority + "; currentReservation "
+        + " at priority " + schedulerKey.getPriority() + "; currentReservation "
         + this.attemptResourceUsage.getReserved());
         + this.attemptResourceUsage.getReserved());
   }
   }
 
 
@@ -239,10 +240,10 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   }
   }
 
 
   public synchronized float getLocalityWaitFactor(
   public synchronized float getLocalityWaitFactor(
-      Priority priority, int clusterNodes) {
+      SchedulerRequestKey schedulerKey, int clusterNodes) {
     // Estimate: Required unique resources (i.e. hosts + racks)
     // Estimate: Required unique resources (i.e. hosts + racks)
     int requiredResources = 
     int requiredResources = 
-        Math.max(this.getResourceRequests(priority).size() - 1, 0);
+        Math.max(this.getResourceRequests(schedulerKey).size() - 1, 0);
     
     
     // waitFactor can't be more than '1' 
     // waitFactor can't be more than '1' 
     // i.e. no point skipping more than clustersize opportunities
     // i.e. no point skipping more than clustersize opportunities
@@ -254,9 +255,15 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * current size of the cluster and thresholds indicating how many nodes to
    * current size of the cluster and thresholds indicating how many nodes to
    * fail at (as a fraction of cluster size) before relaxing scheduling
    * fail at (as a fraction of cluster size) before relaxing scheduling
    * constraints.
    * constraints.
+   * @param schedulerKey SchedulerRequestKey
+   * @param numNodes Num Nodes
+   * @param nodeLocalityThreshold nodeLocalityThreshold
+   * @param rackLocalityThreshold rackLocalityThreshold
+   * @return NodeType
    */
    */
-  public synchronized NodeType getAllowedLocalityLevel(Priority priority,
-      int numNodes, double nodeLocalityThreshold, double rackLocalityThreshold) {
+  public synchronized NodeType getAllowedLocalityLevel(
+      SchedulerRequestKey schedulerKey, int numNodes,
+      double nodeLocalityThreshold, double rackLocalityThreshold) {
     // upper limit on threshold
     // upper limit on threshold
     if (nodeLocalityThreshold > 1.0) { nodeLocalityThreshold = 1.0; }
     if (nodeLocalityThreshold > 1.0) { nodeLocalityThreshold = 1.0; }
     if (rackLocalityThreshold > 1.0) { rackLocalityThreshold = 1.0; }
     if (rackLocalityThreshold > 1.0) { rackLocalityThreshold = 1.0; }
@@ -267,12 +274,12 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     }
     }
 
 
     // Default level is NODE_LOCAL
     // Default level is NODE_LOCAL
-    if (!allowedLocalityLevel.containsKey(priority)) {
-      allowedLocalityLevel.put(priority, NodeType.NODE_LOCAL);
+    if (!allowedLocalityLevel.containsKey(schedulerKey)) {
+      allowedLocalityLevel.put(schedulerKey, NodeType.NODE_LOCAL);
       return NodeType.NODE_LOCAL;
       return NodeType.NODE_LOCAL;
     }
     }
 
 
-    NodeType allowed = allowedLocalityLevel.get(priority);
+    NodeType allowed = allowedLocalityLevel.get(schedulerKey);
 
 
     // If level is already most liberal, we're done
     // If level is already most liberal, we're done
     if (allowed.equals(NodeType.OFF_SWITCH)) return NodeType.OFF_SWITCH;
     if (allowed.equals(NodeType.OFF_SWITCH)) return NodeType.OFF_SWITCH;
@@ -281,27 +288,32 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       rackLocalityThreshold;
       rackLocalityThreshold;
 
 
     // Relax locality constraints once we've surpassed threshold.
     // Relax locality constraints once we've surpassed threshold.
-    if (getSchedulingOpportunities(priority) > (numNodes * threshold)) {
+    if (getSchedulingOpportunities(schedulerKey) > (numNodes * threshold)) {
       if (allowed.equals(NodeType.NODE_LOCAL)) {
       if (allowed.equals(NodeType.NODE_LOCAL)) {
-        allowedLocalityLevel.put(priority, NodeType.RACK_LOCAL);
-        resetSchedulingOpportunities(priority);
+        allowedLocalityLevel.put(schedulerKey, NodeType.RACK_LOCAL);
+        resetSchedulingOpportunities(schedulerKey);
       }
       }
       else if (allowed.equals(NodeType.RACK_LOCAL)) {
       else if (allowed.equals(NodeType.RACK_LOCAL)) {
-        allowedLocalityLevel.put(priority, NodeType.OFF_SWITCH);
-        resetSchedulingOpportunities(priority);
+        allowedLocalityLevel.put(schedulerKey, NodeType.OFF_SWITCH);
+        resetSchedulingOpportunities(schedulerKey);
       }
       }
     }
     }
-    return allowedLocalityLevel.get(priority);
+    return allowedLocalityLevel.get(schedulerKey);
   }
   }
 
 
   /**
   /**
    * Return the level at which we are allowed to schedule containers.
    * Return the level at which we are allowed to schedule containers.
    * Given the thresholds indicating how much time passed before relaxing
    * Given the thresholds indicating how much time passed before relaxing
    * scheduling constraints.
    * scheduling constraints.
+   * @param schedulerKey SchedulerRequestKey
+   * @param nodeLocalityDelayMs nodeLocalityThreshold
+   * @param rackLocalityDelayMs nodeLocalityDelayMs
+   * @param currentTimeMs currentTimeMs
+   * @return NodeType
    */
    */
-  public synchronized NodeType getAllowedLocalityLevelByTime(Priority priority,
-          long nodeLocalityDelayMs, long rackLocalityDelayMs,
-          long currentTimeMs) {
+  public synchronized NodeType getAllowedLocalityLevelByTime(
+      SchedulerRequestKey schedulerKey, long nodeLocalityDelayMs,
+      long rackLocalityDelayMs, long currentTimeMs) {
 
 
     // if not being used, can schedule anywhere
     // if not being used, can schedule anywhere
     if (nodeLocalityDelayMs < 0 || rackLocalityDelayMs < 0) {
     if (nodeLocalityDelayMs < 0 || rackLocalityDelayMs < 0) {
@@ -309,19 +321,19 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     }
     }
 
 
     // default level is NODE_LOCAL
     // default level is NODE_LOCAL
-    if (!allowedLocalityLevel.containsKey(priority)) {
+    if (!allowedLocalityLevel.containsKey(schedulerKey)) {
       // add the initial time of priority to prevent comparing with FsApp
       // add the initial time of priority to prevent comparing with FsApp
       // startTime and allowedLocalityLevel degrade
       // startTime and allowedLocalityLevel degrade
-      lastScheduledContainer.put(priority, currentTimeMs);
+      lastScheduledContainer.put(schedulerKey, currentTimeMs);
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Init the lastScheduledContainer time, priority: " + priority
-            + ", time: " + currentTimeMs);
+        LOG.debug("Init the lastScheduledContainer time, priority: "
+            + schedulerKey.getPriority() + ", time: " + currentTimeMs);
       }
       }
-      allowedLocalityLevel.put(priority, NodeType.NODE_LOCAL);
+      allowedLocalityLevel.put(schedulerKey, NodeType.NODE_LOCAL);
       return NodeType.NODE_LOCAL;
       return NodeType.NODE_LOCAL;
     }
     }
 
 
-    NodeType allowed = allowedLocalityLevel.get(priority);
+    NodeType allowed = allowedLocalityLevel.get(schedulerKey);
 
 
     // if level is already most liberal, we're done
     // if level is already most liberal, we're done
     if (allowed.equals(NodeType.OFF_SWITCH)) {
     if (allowed.equals(NodeType.OFF_SWITCH)) {
@@ -330,8 +342,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
 
     // check waiting time
     // check waiting time
     long waitTime = currentTimeMs;
     long waitTime = currentTimeMs;
-    if (lastScheduledContainer.containsKey(priority)) {
-      waitTime -= lastScheduledContainer.get(priority);
+    if (lastScheduledContainer.containsKey(schedulerKey)) {
+      waitTime -= lastScheduledContainer.get(schedulerKey);
     } else {
     } else {
       waitTime -= getStartTime();
       waitTime -= getStartTime();
     }
     }
@@ -341,43 +353,43 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
 
     if (waitTime > thresholdTime) {
     if (waitTime > thresholdTime) {
       if (allowed.equals(NodeType.NODE_LOCAL)) {
       if (allowed.equals(NodeType.NODE_LOCAL)) {
-        allowedLocalityLevel.put(priority, NodeType.RACK_LOCAL);
-        resetSchedulingOpportunities(priority, currentTimeMs);
+        allowedLocalityLevel.put(schedulerKey, NodeType.RACK_LOCAL);
+        resetSchedulingOpportunities(schedulerKey, currentTimeMs);
       } else if (allowed.equals(NodeType.RACK_LOCAL)) {
       } else if (allowed.equals(NodeType.RACK_LOCAL)) {
-        allowedLocalityLevel.put(priority, NodeType.OFF_SWITCH);
-        resetSchedulingOpportunities(priority, currentTimeMs);
+        allowedLocalityLevel.put(schedulerKey, NodeType.OFF_SWITCH);
+        resetSchedulingOpportunities(schedulerKey, currentTimeMs);
       }
       }
     }
     }
-    return allowedLocalityLevel.get(priority);
+    return allowedLocalityLevel.get(schedulerKey);
   }
   }
 
 
   synchronized public RMContainer allocate(NodeType type, FSSchedulerNode node,
   synchronized public RMContainer allocate(NodeType type, FSSchedulerNode node,
-      Priority priority, ResourceRequest request,
+      SchedulerRequestKey schedulerKey, ResourceRequest request,
       Container reservedContainer) {
       Container reservedContainer) {
     // Update allowed locality level
     // Update allowed locality level
-    NodeType allowed = allowedLocalityLevel.get(priority);
+    NodeType allowed = allowedLocalityLevel.get(schedulerKey);
     if (allowed != null) {
     if (allowed != null) {
       if (allowed.equals(NodeType.OFF_SWITCH) &&
       if (allowed.equals(NodeType.OFF_SWITCH) &&
           (type.equals(NodeType.NODE_LOCAL) ||
           (type.equals(NodeType.NODE_LOCAL) ||
               type.equals(NodeType.RACK_LOCAL))) {
               type.equals(NodeType.RACK_LOCAL))) {
-        this.resetAllowedLocalityLevel(priority, type);
+        this.resetAllowedLocalityLevel(schedulerKey, type);
       }
       }
       else if (allowed.equals(NodeType.RACK_LOCAL) &&
       else if (allowed.equals(NodeType.RACK_LOCAL) &&
           type.equals(NodeType.NODE_LOCAL)) {
           type.equals(NodeType.NODE_LOCAL)) {
-        this.resetAllowedLocalityLevel(priority, type);
+        this.resetAllowedLocalityLevel(schedulerKey, type);
       }
       }
     }
     }
 
 
     // Required sanity check - AM can call 'allocate' to update resource 
     // Required sanity check - AM can call 'allocate' to update resource 
     // request without locking the scheduler, hence we need to check
     // request without locking the scheduler, hence we need to check
-    if (getTotalRequiredResources(priority) <= 0) {
+    if (getTotalRequiredResources(schedulerKey) <= 0) {
       return null;
       return null;
     }
     }
 
 
     Container container = reservedContainer;
     Container container = reservedContainer;
     if (container == null) {
     if (container == null) {
       container =
       container =
-          createContainer(node, request.getCapability(), request.getPriority());
+          createContainer(node, request.getCapability(), schedulerKey);
     }
     }
     
     
     // Create RMContainer
     // Create RMContainer
@@ -392,7 +404,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
 
     // Update consumption and track allocations
     // Update consumption and track allocations
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
-        type, node, priority, request, container);
+        type, node, schedulerKey, request, container);
     this.attemptResourceUsage.incUsed(container.getResource());
     this.attemptResourceUsage.incUsed(container.getResource());
 
 
     // Update resource requests related to "request" and store in RMContainer
     // Update resource requests related to "request" and store in RMContainer
@@ -419,13 +431,15 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * Should be called when the scheduler assigns a container at a higher
    * Should be called when the scheduler assigns a container at a higher
    * degree of locality than the current threshold. Reset the allowed locality
    * degree of locality than the current threshold. Reset the allowed locality
    * level to a higher degree of locality.
    * level to a higher degree of locality.
+   * @param schedulerKey Scheduler Key
+   * @param level NodeType
    */
    */
-  public synchronized void resetAllowedLocalityLevel(Priority priority,
-      NodeType level) {
-    NodeType old = allowedLocalityLevel.get(priority);
+  public synchronized void resetAllowedLocalityLevel(
+      SchedulerRequestKey schedulerKey, NodeType level) {
+    NodeType old = allowedLocalityLevel.get(schedulerKey);
     LOG.info("Raising locality level from " + old + " to " + level + " at " +
     LOG.info("Raising locality level from " + old + " to " + level + " at " +
-        " priority " + priority);
-    allowedLocalityLevel.put(priority, level);
+        " priority " + schedulerKey.getPriority());
+    allowedLocalityLevel.put(schedulerKey, level);
   }
   }
 
 
   // related methods
   // related methods
@@ -468,9 +482,13 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * Create and return a container object reflecting an allocation for the
    * Create and return a container object reflecting an allocation for the
    * given appliction on the given node with the given capability and
    * given appliction on the given node with the given capability and
    * priority.
    * priority.
+   * @param node Node
+   * @param capability Capability
+   * @param schedulerKey Scheduler Key
+   * @return Container
    */
    */
-  public Container createContainer(
-      FSSchedulerNode node, Resource capability, Priority priority) {
+  public Container createContainer(FSSchedulerNode node, Resource capability,
+      SchedulerRequestKey schedulerKey) {
 
 
     NodeId nodeId = node.getRMNode().getNodeID();
     NodeId nodeId = node.getRMNode().getNodeID();
     ContainerId containerId = BuilderUtils.newContainerId(
     ContainerId containerId = BuilderUtils.newContainerId(
@@ -479,7 +497,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     // Create the container
     // Create the container
     Container container =
     Container container =
         BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
         BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
-            .getHttpAddress(), capability, priority, null);
+            .getHttpAddress(), capability, schedulerKey.getPriority(), null);
 
 
     return container;
     return container;
   }
   }
@@ -492,26 +510,26 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * return whether reservation was possible with the current threshold limits
    * return whether reservation was possible with the current threshold limits
    */
    */
   private boolean reserve(ResourceRequest request, FSSchedulerNode node,
   private boolean reserve(ResourceRequest request, FSSchedulerNode node,
-      Container reservedContainer, NodeType type) {
+      Container reservedContainer, NodeType type,
+      SchedulerRequestKey schedulerKey) {
 
 
-    Priority priority = request.getPriority();
     if (!reservationExceedsThreshold(node, type)) {
     if (!reservationExceedsThreshold(node, type)) {
       LOG.info("Making reservation: node=" + node.getNodeName() +
       LOG.info("Making reservation: node=" + node.getNodeName() +
               " app_id=" + getApplicationId());
               " app_id=" + getApplicationId());
       if (reservedContainer == null) {
       if (reservedContainer == null) {
         reservedContainer =
         reservedContainer =
             createContainer(node, request.getCapability(),
             createContainer(node, request.getCapability(),
-              request.getPriority());
+              schedulerKey);
         getMetrics().reserveResource(getUser(),
         getMetrics().reserveResource(getUser(),
             reservedContainer.getResource());
             reservedContainer.getResource());
         RMContainer rmContainer =
         RMContainer rmContainer =
-                super.reserve(node, priority, null, reservedContainer);
-        node.reserveResource(this, priority, rmContainer);
+                super.reserve(node, schedulerKey, null, reservedContainer);
+        node.reserveResource(this, schedulerKey, rmContainer);
         setReservation(node);
         setReservation(node);
       } else {
       } else {
         RMContainer rmContainer = node.getReservedContainer();
         RMContainer rmContainer = node.getReservedContainer();
-        super.reserve(node, priority, rmContainer, reservedContainer);
-        node.reserveResource(this, priority, rmContainer);
+        super.reserve(node, schedulerKey, rmContainer, reservedContainer);
+        node.reserveResource(this, schedulerKey, rmContainer);
         setReservation(node);
         setReservation(node);
       }
       }
       return true;
       return true;
@@ -548,13 +566,17 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     }
     }
     return false;
     return false;
   }
   }
+
   /**
   /**
-   * Remove the reservation on {@code node} at the given {@link Priority}.
+   * Remove the reservation on {@code node} at the given SchedulerRequestKey.
    * This dispatches SchedulerNode handlers as well.
    * This dispatches SchedulerNode handlers as well.
+   * @param schedulerKey Scheduler Key
+   * @param node Node
    */
    */
-  public void unreserve(Priority priority, FSSchedulerNode node) {
+  public void unreserve(SchedulerRequestKey schedulerKey,
+      FSSchedulerNode node) {
     RMContainer rmContainer = node.getReservedContainer();
     RMContainer rmContainer = node.getReservedContainer();
-    unreserveInternal(priority, node);
+    unreserveInternal(schedulerKey, node);
     node.unreserveResource(this);
     node.unreserveResource(this);
     clearReservation(node);
     clearReservation(node);
     getMetrics().unreserveResource(
     getMetrics().unreserveResource(
@@ -618,7 +640,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    */
    */
   private Resource assignContainer(
   private Resource assignContainer(
       FSSchedulerNode node, ResourceRequest request, NodeType type,
       FSSchedulerNode node, ResourceRequest request, NodeType type,
-      boolean reserved) {
+      boolean reserved, SchedulerRequestKey schedulerKey) {
 
 
     // How much does this request need?
     // How much does this request need?
     Resource capability = request.getCapability();
     Resource capability = request.getCapability();
@@ -635,19 +657,19 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     if (Resources.fitsIn(capability, available)) {
     if (Resources.fitsIn(capability, available)) {
       // Inform the application of the new container for this request
       // Inform the application of the new container for this request
       RMContainer allocatedContainer =
       RMContainer allocatedContainer =
-          allocate(type, node, request.getPriority(), request,
+          allocate(type, node, schedulerKey, request,
               reservedContainer);
               reservedContainer);
       if (allocatedContainer == null) {
       if (allocatedContainer == null) {
         // Did the application need this resource?
         // Did the application need this resource?
         if (reserved) {
         if (reserved) {
-          unreserve(request.getPriority(), node);
+          unreserve(schedulerKey, node);
         }
         }
         return Resources.none();
         return Resources.none();
       }
       }
 
 
       // If we had previously made a reservation, delete it
       // If we had previously made a reservation, delete it
       if (reserved) {
       if (reserved) {
-        unreserve(request.getPriority(), node);
+        unreserve(schedulerKey, node);
       }
       }
 
 
       // Inform the node
       // Inform the node
@@ -667,7 +689,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
 
     // The desired container won't fit here, so reserve
     // The desired container won't fit here, so reserve
     if (isReservable(capability) &&
     if (isReservable(capability) &&
-        reserve(request, node, reservedContainer, type)) {
+        reserve(request, node, reservedContainer, type, schedulerKey)) {
       return FairScheduler.CONTAINER_RESERVED;
       return FairScheduler.CONTAINER_RESERVED;
     } else {
     } else {
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
@@ -683,8 +705,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
         getQueue().getPolicy().getResourceCalculator(), capacity);
         getQueue().getPolicy().getResourceCalculator(), capacity);
   }
   }
 
 
-  private boolean hasNodeOrRackLocalRequests(Priority priority) {
-    return getResourceRequests(priority).size() > 1;
+  private boolean hasNodeOrRackLocalRequests(SchedulerRequestKey schedulerKey) {
+    return getResourceRequests(schedulerKey).size() > 1;
   }
   }
 
 
   /**
   /**
@@ -707,26 +729,26 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       LOG.debug("Node offered to app: " + getName() + " reserved: " + reserved);
       LOG.debug("Node offered to app: " + getName() + " reserved: " + reserved);
     }
     }
 
 
-    Collection<Priority> prioritiesToTry = (reserved) ?
-        Arrays.asList(node.getReservedContainer().getReservedPriority()) :
-        getPriorities();
+    Collection<SchedulerRequestKey> keysToTry = (reserved) ?
+        Arrays.asList(node.getReservedContainer().getReservedSchedulerKey()) :
+        getSchedulerKeys();
 
 
     // For each priority, see if we can schedule a node local, rack local
     // For each priority, see if we can schedule a node local, rack local
     // or off-switch request. Rack of off-switch requests may be delayed
     // or off-switch request. Rack of off-switch requests may be delayed
     // (not scheduled) in order to promote better locality.
     // (not scheduled) in order to promote better locality.
     synchronized (this) {
     synchronized (this) {
-      for (Priority priority : prioritiesToTry) {
+      for (SchedulerRequestKey schedulerKey : keysToTry) {
         // Skip it for reserved container, since
         // Skip it for reserved container, since
         // we already check it in isValidReservation.
         // we already check it in isValidReservation.
-        if (!reserved && !hasContainerForNode(priority, node)) {
+        if (!reserved && !hasContainerForNode(schedulerKey, node)) {
           continue;
           continue;
         }
         }
 
 
-        addSchedulingOpportunity(priority);
+        addSchedulingOpportunity(schedulerKey);
 
 
-        ResourceRequest rackLocalRequest = getResourceRequest(priority,
+        ResourceRequest rackLocalRequest = getResourceRequest(schedulerKey,
             node.getRackName());
             node.getRackName());
-        ResourceRequest localRequest = getResourceRequest(priority,
+        ResourceRequest localRequest = getResourceRequest(schedulerKey,
             node.getNodeName());
             node.getNodeName());
 
 
         if (localRequest != null && !localRequest.getRelaxLocality()) {
         if (localRequest != null && !localRequest.getRelaxLocality()) {
@@ -736,12 +758,12 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
 
         NodeType allowedLocality;
         NodeType allowedLocality;
         if (scheduler.isContinuousSchedulingEnabled()) {
         if (scheduler.isContinuousSchedulingEnabled()) {
-          allowedLocality = getAllowedLocalityLevelByTime(priority,
+          allowedLocality = getAllowedLocalityLevelByTime(schedulerKey,
               scheduler.getNodeLocalityDelayMs(),
               scheduler.getNodeLocalityDelayMs(),
               scheduler.getRackLocalityDelayMs(),
               scheduler.getRackLocalityDelayMs(),
               scheduler.getClock().getTime());
               scheduler.getClock().getTime());
         } else {
         } else {
-          allowedLocality = getAllowedLocalityLevel(priority,
+          allowedLocality = getAllowedLocalityLevel(schedulerKey,
               scheduler.getNumClusterNodes(),
               scheduler.getNumClusterNodes(),
               scheduler.getNodeLocalityThreshold(),
               scheduler.getNodeLocalityThreshold(),
               scheduler.getRackLocalityThreshold());
               scheduler.getRackLocalityThreshold());
@@ -750,7 +772,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
             && localRequest != null && localRequest.getNumContainers() != 0) {
             && localRequest != null && localRequest.getNumContainers() != 0) {
           return assignContainer(node, localRequest,
           return assignContainer(node, localRequest,
-              NodeType.NODE_LOCAL, reserved);
+              NodeType.NODE_LOCAL, reserved, schedulerKey);
         }
         }
 
 
         if (rackLocalRequest != null && !rackLocalRequest.getRelaxLocality()) {
         if (rackLocalRequest != null && !rackLocalRequest.getRelaxLocality()) {
@@ -761,21 +783,22 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
             && (allowedLocality.equals(NodeType.RACK_LOCAL) ||
             && (allowedLocality.equals(NodeType.RACK_LOCAL) ||
             allowedLocality.equals(NodeType.OFF_SWITCH))) {
             allowedLocality.equals(NodeType.OFF_SWITCH))) {
           return assignContainer(node, rackLocalRequest,
           return assignContainer(node, rackLocalRequest,
-              NodeType.RACK_LOCAL, reserved);
+              NodeType.RACK_LOCAL, reserved, schedulerKey);
         }
         }
 
 
         ResourceRequest offSwitchRequest =
         ResourceRequest offSwitchRequest =
-            getResourceRequest(priority, ResourceRequest.ANY);
+            getResourceRequest(schedulerKey, ResourceRequest.ANY);
         if (offSwitchRequest != null && !offSwitchRequest.getRelaxLocality()) {
         if (offSwitchRequest != null && !offSwitchRequest.getRelaxLocality()) {
           continue;
           continue;
         }
         }
 
 
         if (offSwitchRequest != null &&
         if (offSwitchRequest != null &&
             offSwitchRequest.getNumContainers() != 0) {
             offSwitchRequest.getNumContainers() != 0) {
-          if (!hasNodeOrRackLocalRequests(priority) ||
+          if (!hasNodeOrRackLocalRequests(schedulerKey) ||
               allowedLocality.equals(NodeType.OFF_SWITCH)) {
               allowedLocality.equals(NodeType.OFF_SWITCH)) {
             return assignContainer(
             return assignContainer(
-                node, offSwitchRequest, NodeType.OFF_SWITCH, reserved);
+                node, offSwitchRequest, NodeType.OFF_SWITCH, reserved,
+                schedulerKey);
           }
           }
         }
         }
       }
       }
@@ -787,10 +810,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * Whether this app has containers requests that could be satisfied on the
    * Whether this app has containers requests that could be satisfied on the
    * given node, if the node had full space.
    * given node, if the node had full space.
    */
    */
-  private boolean hasContainerForNode(Priority prio, FSSchedulerNode node) {
-    ResourceRequest anyRequest = getResourceRequest(prio, ResourceRequest.ANY);
-    ResourceRequest rackRequest = getResourceRequest(prio, node.getRackName());
-    ResourceRequest nodeRequest = getResourceRequest(prio, node.getNodeName());
+  private boolean hasContainerForNode(SchedulerRequestKey key,
+      FSSchedulerNode node) {
+    ResourceRequest anyRequest = getResourceRequest(key, ResourceRequest.ANY);
+    ResourceRequest rackRequest = getResourceRequest(key, node.getRackName());
+    ResourceRequest nodeRequest = getResourceRequest(key, node.getNodeName());
 
 
     return
     return
         // There must be outstanding requests at the given priority:
         // There must be outstanding requests at the given priority:
@@ -812,9 +836,9 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   }
   }
 
 
   private boolean isValidReservation(FSSchedulerNode node) {
   private boolean isValidReservation(FSSchedulerNode node) {
-    Priority reservedPriority = node.getReservedContainer().
-        getReservedPriority();
-    return hasContainerForNode(reservedPriority, node) &&
+    SchedulerRequestKey schedulerKey = node.getReservedContainer().
+        getReservedSchedulerKey();
+    return hasContainerForNode(schedulerKey, node) &&
         !isOverAMShareLimit();
         !isOverAMShareLimit();
   }
   }
 
 
@@ -830,13 +854,14 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    */
    */
   public boolean assignReservedContainer(FSSchedulerNode node) {
   public boolean assignReservedContainer(FSSchedulerNode node) {
     RMContainer rmContainer = node.getReservedContainer();
     RMContainer rmContainer = node.getReservedContainer();
-    Priority reservedPriority = rmContainer.getReservedPriority();
+    SchedulerRequestKey reservedSchedulerKey =
+        rmContainer.getReservedSchedulerKey();
 
 
     if (!isValidReservation(node)) {
     if (!isValidReservation(node)) {
       // Don't hold the reservation if app can no longer use it
       // Don't hold the reservation if app can no longer use it
       LOG.info("Releasing reservation that cannot be satisfied for " +
       LOG.info("Releasing reservation that cannot be satisfied for " +
           "application " + getApplicationAttemptId() + " on node " + node);
           "application " + getApplicationAttemptId() + " on node " + node);
-      unreserve(reservedPriority, node);
+      unreserve(reservedSchedulerKey, node);
       return false;
       return false;
     }
     }
 
 
@@ -938,8 +963,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
 
     // Add up outstanding resource requests
     // Add up outstanding resource requests
     synchronized (this) {
     synchronized (this) {
-      for (Priority p : getPriorities()) {
-        ResourceRequest r = getResourceRequest(p, ResourceRequest.ANY);
+      for (SchedulerRequestKey k : getSchedulerKeys()) {
+        ResourceRequest r = getResourceRequest(k, ResourceRequest.ANY);
         if (r != null) {
         if (r != null) {
           Resources.multiplyAndAddTo(demand,
           Resources.multiplyAndAddTo(demand,
               r.getCapability(), r.getNumContainers());
               r.getCapability(), r.getNumContainers());

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

@@ -23,10 +23,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 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.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 
 
 @Private
 @Private
@@ -43,7 +43,7 @@ public class FSSchedulerNode extends SchedulerNode {
 
 
   @Override
   @Override
   public synchronized void reserveResource(
   public synchronized void reserveResource(
-      SchedulerApplicationAttempt application, Priority priority,
+      SchedulerApplicationAttempt application, SchedulerRequestKey schedulerKey,
       RMContainer container) {
       RMContainer container) {
     // Check if it's already reserved
     // Check if it's already reserved
     RMContainer reservedContainer = getReservedContainer();
     RMContainer reservedContainer = getReservedContainer();
@@ -102,4 +102,5 @@ public class FSSchedulerNode extends SchedulerNode {
   public synchronized FSAppAttempt getReservedAppSchedulable() {
   public synchronized FSAppAttempt getReservedAppSchedulable() {
     return reservedAppSchedulable;
     return reservedAppSchedulable;
   }
   }
+
 }
 }

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

@@ -866,7 +866,7 @@ public class FairScheduler extends
     FSSchedulerNode node = getFSSchedulerNode(container.getNodeId());
     FSSchedulerNode node = getFSSchedulerNode(container.getNodeId());
 
 
     if (rmContainer.getState() == RMContainerState.RESERVED) {
     if (rmContainer.getState() == RMContainerState.RESERVED) {
-      application.unreserve(rmContainer.getReservedPriority(), node);
+      application.unreserve(rmContainer.getReservedSchedulerKey(), node);
     } else {
     } else {
       application.containerCompleted(rmContainer, containerStatus, event);
       application.containerCompleted(rmContainer, containerStatus, event);
       node.releaseContainer(container);
       node.releaseContainer(container);

+ 46 - 41
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

@@ -81,6 +81,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerCha
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
@@ -514,14 +515,15 @@ public class FifoScheduler extends
           continue;
           continue;
         }
         }
 
 
-        for (Priority priority : application.getPriorities()) {
-          int maxContainers = 
-            getMaxAllocatableContainers(application, priority, node, 
-                NodeType.OFF_SWITCH); 
+        for (SchedulerRequestKey schedulerKey :
+            application.getSchedulerKeys()) {
+          int maxContainers =
+              getMaxAllocatableContainers(application, schedulerKey, node,
+                  NodeType.OFF_SWITCH);
           // Ensure the application needs containers of this priority
           // Ensure the application needs containers of this priority
           if (maxContainers > 0) {
           if (maxContainers > 0) {
-            int assignedContainers = 
-              assignContainersOnNode(node, application, priority);
+            int assignedContainers =
+                assignContainersOnNode(node, application, schedulerKey);
             // Do not assign out of order w.r.t priorities
             // Do not assign out of order w.r.t priorities
             if (assignedContainers == 0) {
             if (assignedContainers == 0) {
               break;
               break;
@@ -553,11 +555,11 @@ public class FifoScheduler extends
   }
   }
 
 
   private int getMaxAllocatableContainers(FiCaSchedulerApp application,
   private int getMaxAllocatableContainers(FiCaSchedulerApp application,
-      Priority priority, FiCaSchedulerNode node, NodeType type) {
+      SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, NodeType type) {
     int maxContainers = 0;
     int maxContainers = 0;
-    
-    ResourceRequest offSwitchRequest = 
-      application.getResourceRequest(priority, ResourceRequest.ANY);
+
+    ResourceRequest offSwitchRequest =
+        application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
     if (offSwitchRequest != null) {
     if (offSwitchRequest != null) {
       maxContainers = offSwitchRequest.getNumContainers();
       maxContainers = offSwitchRequest.getNumContainers();
     }
     }
@@ -567,8 +569,9 @@ public class FifoScheduler extends
     }
     }
 
 
     if (type == NodeType.RACK_LOCAL) {
     if (type == NodeType.RACK_LOCAL) {
-      ResourceRequest rackLocalRequest = 
-        application.getResourceRequest(priority, node.getRMNode().getRackName());
+      ResourceRequest rackLocalRequest =
+          application.getResourceRequest(schedulerKey, node.getRMNode()
+              .getRackName());
       if (rackLocalRequest == null) {
       if (rackLocalRequest == null) {
         return maxContainers;
         return maxContainers;
       }
       }
@@ -577,8 +580,9 @@ public class FifoScheduler extends
     }
     }
 
 
     if (type == NodeType.NODE_LOCAL) {
     if (type == NodeType.NODE_LOCAL) {
-      ResourceRequest nodeLocalRequest = 
-        application.getResourceRequest(priority, node.getRMNode().getNodeAddress());
+      ResourceRequest nodeLocalRequest =
+          application.getResourceRequest(schedulerKey, node.getRMNode()
+              .getNodeAddress());
       if (nodeLocalRequest != null) {
       if (nodeLocalRequest != null) {
         maxContainers = Math.min(maxContainers, nodeLocalRequest.getNumContainers());
         maxContainers = Math.min(maxContainers, nodeLocalRequest.getNumContainers());
       }
       }
@@ -589,25 +593,25 @@ public class FifoScheduler extends
 
 
 
 
   private int assignContainersOnNode(FiCaSchedulerNode node, 
   private int assignContainersOnNode(FiCaSchedulerNode node, 
-      FiCaSchedulerApp application, Priority priority 
+      FiCaSchedulerApp application, SchedulerRequestKey schedulerKey
   ) {
   ) {
     // Data-local
     // Data-local
-    int nodeLocalContainers = 
-      assignNodeLocalContainers(node, application, priority); 
+    int nodeLocalContainers =
+        assignNodeLocalContainers(node, application, schedulerKey);
 
 
     // Rack-local
     // Rack-local
-    int rackLocalContainers = 
-      assignRackLocalContainers(node, application, priority);
+    int rackLocalContainers =
+        assignRackLocalContainers(node, application, schedulerKey);
 
 
     // Off-switch
     // Off-switch
     int offSwitchContainers =
     int offSwitchContainers =
-      assignOffSwitchContainers(node, application, priority);
+        assignOffSwitchContainers(node, application, schedulerKey);
 
 
 
 
     LOG.debug("assignContainersOnNode:" +
     LOG.debug("assignContainersOnNode:" +
         " node=" + node.getRMNode().getNodeAddress() + 
         " node=" + node.getRMNode().getNodeAddress() + 
         " application=" + application.getApplicationId().getId() +
         " application=" + application.getApplicationId().getId() +
-        " priority=" + priority.getPriority() + 
+        " priority=" + schedulerKey.getPriority() +
         " #assigned=" + 
         " #assigned=" + 
         (nodeLocalContainers + rackLocalContainers + offSwitchContainers));
         (nodeLocalContainers + rackLocalContainers + offSwitchContainers));
 
 
@@ -616,14 +620,14 @@ public class FifoScheduler extends
   }
   }
 
 
   private int assignNodeLocalContainers(FiCaSchedulerNode node, 
   private int assignNodeLocalContainers(FiCaSchedulerNode node, 
-      FiCaSchedulerApp application, Priority priority) {
+      FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) {
     int assignedContainers = 0;
     int assignedContainers = 0;
-    ResourceRequest request = 
-      application.getResourceRequest(priority, node.getNodeName());
+    ResourceRequest request =
+        application.getResourceRequest(schedulerKey, node.getNodeName());
     if (request != null) {
     if (request != null) {
       // Don't allocate on this node if we don't need containers on this rack
       // Don't allocate on this node if we don't need containers on this rack
       ResourceRequest rackRequest =
       ResourceRequest rackRequest =
-          application.getResourceRequest(priority, 
+          application.getResourceRequest(schedulerKey,
               node.getRMNode().getRackName());
               node.getRMNode().getRackName());
       if (rackRequest == null || rackRequest.getNumContainers() <= 0) {
       if (rackRequest == null || rackRequest.getNumContainers() <= 0) {
         return 0;
         return 0;
@@ -631,61 +635,62 @@ public class FifoScheduler extends
       
       
       int assignableContainers = 
       int assignableContainers = 
         Math.min(
         Math.min(
-            getMaxAllocatableContainers(application, priority, node, 
+            getMaxAllocatableContainers(application, schedulerKey, node,
                 NodeType.NODE_LOCAL), 
                 NodeType.NODE_LOCAL), 
                 request.getNumContainers());
                 request.getNumContainers());
       assignedContainers = 
       assignedContainers = 
-        assignContainer(node, application, priority, 
+        assignContainer(node, application, schedulerKey,
             assignableContainers, request, NodeType.NODE_LOCAL);
             assignableContainers, request, NodeType.NODE_LOCAL);
     }
     }
     return assignedContainers;
     return assignedContainers;
   }
   }
 
 
   private int assignRackLocalContainers(FiCaSchedulerNode node, 
   private int assignRackLocalContainers(FiCaSchedulerNode node, 
-      FiCaSchedulerApp application, Priority priority) {
+      FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) {
     int assignedContainers = 0;
     int assignedContainers = 0;
-    ResourceRequest request = 
-      application.getResourceRequest(priority, node.getRMNode().getRackName());
+    ResourceRequest request =
+        application.getResourceRequest(schedulerKey, node.getRMNode()
+            .getRackName());
     if (request != null) {
     if (request != null) {
       // Don't allocate on this rack if the application doens't need containers
       // Don't allocate on this rack if the application doens't need containers
       ResourceRequest offSwitchRequest =
       ResourceRequest offSwitchRequest =
-          application.getResourceRequest(priority, ResourceRequest.ANY);
+          application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
       if (offSwitchRequest.getNumContainers() <= 0) {
       if (offSwitchRequest.getNumContainers() <= 0) {
         return 0;
         return 0;
       }
       }
       
       
       int assignableContainers = 
       int assignableContainers = 
         Math.min(
         Math.min(
-            getMaxAllocatableContainers(application, priority, node, 
+            getMaxAllocatableContainers(application, schedulerKey, node,
                 NodeType.RACK_LOCAL), 
                 NodeType.RACK_LOCAL), 
                 request.getNumContainers());
                 request.getNumContainers());
       assignedContainers = 
       assignedContainers = 
-        assignContainer(node, application, priority, 
+        assignContainer(node, application, schedulerKey,
             assignableContainers, request, NodeType.RACK_LOCAL);
             assignableContainers, request, NodeType.RACK_LOCAL);
     }
     }
     return assignedContainers;
     return assignedContainers;
   }
   }
 
 
   private int assignOffSwitchContainers(FiCaSchedulerNode node, 
   private int assignOffSwitchContainers(FiCaSchedulerNode node, 
-      FiCaSchedulerApp application, Priority priority) {
+      FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) {
     int assignedContainers = 0;
     int assignedContainers = 0;
-    ResourceRequest request = 
-      application.getResourceRequest(priority, ResourceRequest.ANY);
+    ResourceRequest request =
+        application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
     if (request != null) {
     if (request != null) {
       assignedContainers = 
       assignedContainers = 
-        assignContainer(node, application, priority, 
+        assignContainer(node, application, schedulerKey,
             request.getNumContainers(), request, NodeType.OFF_SWITCH);
             request.getNumContainers(), request, NodeType.OFF_SWITCH);
     }
     }
     return assignedContainers;
     return assignedContainers;
   }
   }
 
 
   private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application,
   private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application,
-      Priority priority, int assignableContainers, 
+      SchedulerRequestKey schedulerKey, int assignableContainers,
       ResourceRequest request, NodeType type) {
       ResourceRequest request, NodeType type) {
     LOG.debug("assignContainers:" +
     LOG.debug("assignContainers:" +
         " node=" + node.getRMNode().getNodeAddress() + 
         " node=" + node.getRMNode().getNodeAddress() + 
         " application=" + application.getApplicationId().getId() + 
         " application=" + application.getApplicationId().getId() + 
-        " priority=" + priority.getPriority() + 
+        " priority=" + schedulerKey.getPriority().getPriority() +
         " assignableContainers=" + assignableContainers +
         " assignableContainers=" + assignableContainers +
         " request=" + request + " type=" + type);
         " request=" + request + " type=" + type);
     Resource capability = request.getCapability();
     Resource capability = request.getCapability();
@@ -707,13 +712,13 @@ public class FifoScheduler extends
         // Create the container
         // Create the container
         Container container =
         Container container =
             BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
             BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
-              .getHttpAddress(), capability, priority, null);
+              .getHttpAddress(), capability, schedulerKey.getPriority(), null);
         
         
         // Allocate!
         // Allocate!
         
         
         // Inform the application
         // Inform the application
         RMContainer rmContainer =
         RMContainer rmContainer =
-            application.allocate(type, node, priority, request, container);
+            application.allocate(type, node, schedulerKey, request, container);
         
         
         // Inform the node
         // Inform the node
         node.allocateContainer(rmContainer);
         node.allocateContainer(rmContainer);

+ 66 - 54
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java

@@ -60,6 +60,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 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.ResourceScheduler;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
+
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .TestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
@@ -78,25 +84,21 @@ public class Application {
   final private ApplicationAttemptId applicationAttemptId;
   final private ApplicationAttemptId applicationAttemptId;
   final private ResourceManager resourceManager;
   final private ResourceManager resourceManager;
   private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
   private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-  
-  final private Map<Priority, Resource> requestSpec = 
-    new TreeMap<Priority, Resource>(
-        new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
-  
-  final private Map<Priority, Map<String, ResourceRequest>> requests = 
-    new TreeMap<Priority, Map<String, ResourceRequest>>(
-        new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
-  
-  final Map<Priority, Set<Task>> tasks = 
-    new TreeMap<Priority, Set<Task>>(
-        new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
-  
-  final private Set<ResourceRequest> ask = 
-    new TreeSet<ResourceRequest>(
-        new org.apache.hadoop.yarn.api.records.ResourceRequest.ResourceRequestComparator());
 
 
-  final private Map<String, NodeManager> nodes = 
-    new HashMap<String, NodeManager>();
+  final private Map<SchedulerRequestKey, Resource> requestSpec =
+      new TreeMap<>();
+
+  final private Map<SchedulerRequestKey, Map<String, ResourceRequest>>
+      requests = new TreeMap<>();
+
+  final Map<SchedulerRequestKey, Set<Task>> tasks = new TreeMap<>();
+
+  final private Set<ResourceRequest> ask =
+      new TreeSet<>(
+          new org.apache.hadoop.yarn.api.records.ResourceRequest
+              .ResourceRequestComparator());
+
+  final private Map<String, NodeManager> nodes = new HashMap<>();
   
   
   Resource used = recordFactory.newRecordInstance(Resource.class);
   Resource used = recordFactory.newRecordInstance(Resource.class);
   
   
@@ -188,13 +190,19 @@ public class Application {
         new AppAttemptAddedSchedulerEvent(this.applicationAttemptId, false);
         new AppAttemptAddedSchedulerEvent(this.applicationAttemptId, false);
     scheduler.handle(addAttemptEvent);
     scheduler.handle(addAttemptEvent);
   }
   }
-  
+
   public synchronized void addResourceRequestSpec(
   public synchronized void addResourceRequestSpec(
       Priority priority, Resource capability) {
       Priority priority, Resource capability) {
-    Resource currentSpec = requestSpec.put(priority, capability);
+    addResourceRequestSpec(TestUtils.toSchedulerKey(priority.getPriority()),
+        capability);
+  }
+  public synchronized void addResourceRequestSpec(
+      SchedulerRequestKey schedulerKey, Resource capability) {
+    Resource currentSpec = requestSpec.put(schedulerKey, capability);
     if (currentSpec != null) {
     if (currentSpec != null) {
       throw new IllegalStateException("Resource spec already exists for " +
       throw new IllegalStateException("Resource spec already exists for " +
-      		"priority " + priority.getPriority() + " - " + currentSpec.getMemorySize());
+          "priority " + schedulerKey.getPriority().getPriority()
+          + " - " + currentSpec.getMemorySize());
     }
     }
   }
   }
   
   
@@ -208,29 +216,29 @@ public class Application {
   }
   }
   
   
   public synchronized void addTask(Task task) {
   public synchronized void addTask(Task task) {
-    Priority priority = task.getPriority();
-    Map<String, ResourceRequest> requests = this.requests.get(priority);
+    SchedulerRequestKey schedulerKey = task.getSchedulerKey();
+    Map<String, ResourceRequest> requests = this.requests.get(schedulerKey);
     if (requests == null) {
     if (requests == null) {
       requests = new HashMap<String, ResourceRequest>();
       requests = new HashMap<String, ResourceRequest>();
-      this.requests.put(priority, requests);
+      this.requests.put(schedulerKey, requests);
       if(LOG.isDebugEnabled()) {
       if(LOG.isDebugEnabled()) {
-        LOG.debug("Added priority=" + priority + " application="
-          + applicationId);
+        LOG.debug("Added priority=" + schedulerKey.getPriority()
+            + " application="+ applicationId);
       }
       }
     }
     }
     
     
-    final Resource capability = requestSpec.get(priority);
+    final Resource capability = requestSpec.get(schedulerKey);
     
     
     // Note down the task
     // Note down the task
-    Set<Task> tasks = this.tasks.get(priority);
+    Set<Task> tasks = this.tasks.get(schedulerKey);
     if (tasks == null) {
     if (tasks == null) {
       tasks = new HashSet<Task>();
       tasks = new HashSet<Task>();
-      this.tasks.put(priority, tasks);
+      this.tasks.put(schedulerKey, tasks);
     }
     }
     tasks.add(task);
     tasks.add(task);
     
     
     LOG.info("Added task " + task.getTaskId() + " to application " + 
     LOG.info("Added task " + task.getTaskId() + " to application " + 
-        applicationId + " at priority " + priority);
+        applicationId + " at priority " + schedulerKey.getPriority());
     
     
     if(LOG.isDebugEnabled()) {
     if(LOG.isDebugEnabled()) {
       LOG.debug("addTask: application=" + applicationId
       LOG.debug("addTask: application=" + applicationId
@@ -240,21 +248,21 @@ public class Application {
     // Create resource requests
     // Create resource requests
     for (String host : task.getHosts()) {
     for (String host : task.getHosts()) {
       // Data-local
       // Data-local
-      addResourceRequest(priority, requests, host, capability);
+      addResourceRequest(schedulerKey, requests, host, capability);
     }
     }
         
         
     // Rack-local
     // Rack-local
     for (String rack : task.getRacks()) {
     for (String rack : task.getRacks()) {
-      addResourceRequest(priority, requests, rack, capability);
+      addResourceRequest(schedulerKey, requests, rack, capability);
     }
     }
       
       
     // Off-switch
     // Off-switch
-    addResourceRequest(priority, requests, ResourceRequest.ANY, capability);
+    addResourceRequest(schedulerKey, requests, ResourceRequest.ANY, capability);
   }
   }
   
   
   public synchronized void finishTask(Task task) throws IOException,
   public synchronized void finishTask(Task task) throws IOException,
       YarnException {
       YarnException {
-    Set<Task> tasks = this.tasks.get(task.getPriority());
+    Set<Task> tasks = this.tasks.get(task.getSchedulerKey());
     if (!tasks.remove(task)) {
     if (!tasks.remove(task)) {
       throw new IllegalStateException(
       throw new IllegalStateException(
           "Finishing unknown task " + task.getTaskId() + 
           "Finishing unknown task " + task.getTaskId() + 
@@ -270,7 +278,7 @@ public class Application {
         StopContainersRequest.newInstance(containerIds);
         StopContainersRequest.newInstance(containerIds);
     nodeManager.stopContainers(stopRequest);
     nodeManager.stopContainers(stopRequest);
     
     
-    Resources.subtractFrom(used, requestSpec.get(task.getPriority()));
+    Resources.subtractFrom(used, requestSpec.get(task.getSchedulerKey()));
     
     
     LOG.info("Finished task " + task.getTaskId() + 
     LOG.info("Finished task " + task.getTaskId() + 
         " of application " + applicationId + 
         " of application " + applicationId + 
@@ -279,13 +287,13 @@ public class Application {
   }
   }
   
   
   private synchronized void addResourceRequest(
   private synchronized void addResourceRequest(
-      Priority priority, Map<String, ResourceRequest> requests, 
+      SchedulerRequestKey schedulerKey, Map<String, ResourceRequest> requests,
       String resourceName, Resource capability) {
       String resourceName, Resource capability) {
     ResourceRequest request = requests.get(resourceName);
     ResourceRequest request = requests.get(resourceName);
     if (request == null) {
     if (request == null) {
       request = 
       request = 
         org.apache.hadoop.yarn.server.utils.BuilderUtils.newResourceRequest(
         org.apache.hadoop.yarn.server.utils.BuilderUtils.newResourceRequest(
-            priority, resourceName, capability, 1);
+            schedulerKey.getPriority(), resourceName, capability, 1);
       requests.put(resourceName, request);
       requests.put(resourceName, request);
     } else {
     } else {
       request.setNumContainers(request.getNumContainers() + 1);
       request.setNumContainers(request.getNumContainers() + 1);
@@ -299,13 +307,13 @@ public class Application {
     ask.add(
     ask.add(
         org.apache.hadoop.yarn.server.utils.BuilderUtils.newResourceRequest(
         org.apache.hadoop.yarn.server.utils.BuilderUtils.newResourceRequest(
             request)); // clone to ensure the RM doesn't manipulate the same obj
             request)); // clone to ensure the RM doesn't manipulate the same obj
-    
-    if(LOG.isDebugEnabled()) {
+
+    if (LOG.isDebugEnabled()) {
       LOG.debug("addResourceRequest: applicationId=" + applicationId.getId()
       LOG.debug("addResourceRequest: applicationId=" + applicationId.getId()
-        + " priority=" + priority.getPriority()
-        + " resourceName=" + resourceName + " capability=" + capability
-        + " numContainers=" + request.getNumContainers()
-        + " #asks=" + ask.size());
+          + " priority=" + schedulerKey.getPriority().getPriority()
+          + " resourceName=" + resourceName + " capability=" + capability
+          + " numContainers=" + request.getNumContainers()
+          + " #asks=" + ask.size());
     }
     }
   }
   }
   
   
@@ -349,10 +357,10 @@ public class Application {
     
     
     int numContainers = containers.size();
     int numContainers = containers.size();
     // Schedule in priority order
     // Schedule in priority order
-    for (Priority priority : requests.keySet()) {
-      assign(priority, NodeType.NODE_LOCAL, containers);
-      assign(priority, NodeType.RACK_LOCAL, containers);
-      assign(priority, NodeType.OFF_SWITCH, containers);
+    for (SchedulerRequestKey schedulerKey: requests.keySet()) {
+      assign(schedulerKey, NodeType.NODE_LOCAL, containers);
+      assign(schedulerKey, NodeType.RACK_LOCAL, containers);
+      assign(schedulerKey, NodeType.OFF_SWITCH, containers);
 
 
       if (containers.isEmpty()) { 
       if (containers.isEmpty()) { 
         break;
         break;
@@ -368,15 +376,18 @@ public class Application {
     assign(getResources());
     assign(getResources());
   }
   }
   
   
-  private synchronized void assign(Priority priority, NodeType type, 
-      List<Container> containers) throws IOException, YarnException {
+  private synchronized void assign(SchedulerRequestKey schedulerKey,
+      NodeType type, List<Container> containers)
+      throws IOException, YarnException {
     for (Iterator<Container> i=containers.iterator(); i.hasNext();) {
     for (Iterator<Container> i=containers.iterator(); i.hasNext();) {
       Container container = i.next();
       Container container = i.next();
       String host = container.getNodeId().toString();
       String host = container.getNodeId().toString();
       
       
-      if (Resources.equals(requestSpec.get(priority), container.getResource())) { 
+      if (Resources.equals(requestSpec.get(schedulerKey),
+          container.getResource())) {
         // See which task can use this container
         // See which task can use this container
-        for (Iterator<Task> t=tasks.get(priority).iterator(); t.hasNext();) {
+        for (Iterator<Task> t=tasks.get(schedulerKey).iterator();
+             t.hasNext();) {
           Task task = t.next();
           Task task = t.next();
           if (task.getState() == State.PENDING && task.canSchedule(type, host)) {
           if (task.getState() == State.PENDING && task.canSchedule(type, host)) {
             NodeManager nodeManager = getNodeManager(host);
             NodeManager nodeManager = getNodeManager(host);
@@ -386,14 +397,15 @@ public class Application {
             
             
             // Track application resource usage
             // Track application resource usage
             Resources.addTo(used, container.getResource());
             Resources.addTo(used, container.getResource());
-            
+
             LOG.info("Assigned container (" + container + ") of type " + type +
             LOG.info("Assigned container (" + container + ") of type " + type +
-                " to task " + task.getTaskId() + " at priority " + priority + 
+                " to task " + task.getTaskId() + " at priority " +
+                schedulerKey.getPriority() +
                 " on node " + nodeManager.getHostName() +
                 " on node " + nodeManager.getHostName() +
                 ", currently using " + used + " resources");
                 ", currently using " + used + " resources");
 
 
             // Update resource requests
             // Update resource requests
-            updateResourceRequests(requests.get(priority), type, task);
+            updateResourceRequests(requests.get(schedulerKey), type, task);
 
 
             // Launch the container
             // Launch the container
             StartContainerRequest scRequest =
             StartContainerRequest scRequest =

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

@@ -31,6 +31,9 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .TestUtils;
 
 
 public class Task {
 public class Task {
   private static final Log LOG = LogFactory.getLog(Task.class);
   private static final Log LOG = LogFactory.getLog(Task.class);
@@ -40,6 +43,7 @@ public class Task {
   final private ApplicationId applicationId;
   final private ApplicationId applicationId;
   final private int taskId;
   final private int taskId;
   final private Priority priority;
   final private Priority priority;
+  final private SchedulerRequestKey schedulerKey;
   
   
   final private Set<String> hosts = new HashSet<String>();
   final private Set<String> hosts = new HashSet<String>();
   final private Set<String> racks = new HashSet<String>();
   final private Set<String> racks = new HashSet<String>();
@@ -48,7 +52,7 @@ public class Task {
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager nodeManager;
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager nodeManager;
   
   
   private State state;
   private State state;
-  
+
   public Task(Application application, Priority priority, String[] hosts) {
   public Task(Application application, Priority priority, String[] hosts) {
     this.applicationId = application.getApplicationId();
     this.applicationId = application.getApplicationId();
     this.priority = priority;
     this.priority = priority;
@@ -64,6 +68,7 @@ public class Task {
         this.racks.add(Application.resolve(host));
         this.racks.add(Application.resolve(host));
       }
       }
     }
     }
+    this.schedulerKey = TestUtils.toSchedulerKey(priority.getPriority());
     LOG.info("Task " + taskId + " added to application " + this.applicationId + 
     LOG.info("Task " + taskId + " added to application " + this.applicationId + 
         " with " + this.hosts.size() + " hosts, " + racks.size() + " racks");
         " with " + this.hosts.size() + " hosts, " + racks.size() + " racks");
   }
   }
@@ -75,6 +80,10 @@ public class Task {
   public Priority getPriority() {
   public Priority getPriority() {
     return priority;
     return priority;
   }
   }
+
+  public SchedulerRequestKey getSchedulerKey() {
+    return schedulerKey;
+  }
   
   
   public org.apache.hadoop.yarn.server.resourcemanager.NodeManager getNodeManager() {
   public org.apache.hadoop.yarn.server.resourcemanager.NodeManager getNodeManager() {
     return nodeManager;
     return nodeManager;

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java

@@ -36,6 +36,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 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.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
@@ -205,7 +207,9 @@ public class ProportionalCapacityPreemptionPolicyMockFramework {
         Container c = mock(Container.class);
         Container c = mock(Container.class);
         when(c.getResource()).thenReturn(res);
         when(c.getResource()).thenReturn(res);
         when(c.getPriority()).thenReturn(pri);
         when(c.getPriority()).thenReturn(pri);
+        SchedulerRequestKey sk = SchedulerRequestKey.extractFrom(c);
         RMContainerImpl rmc = mock(RMContainerImpl.class);
         RMContainerImpl rmc = mock(RMContainerImpl.class);
+        when(rmc.getAllocatedSchedulerKey()).thenReturn(sk);
         when(rmc.getAllocatedNode()).thenReturn(host);
         when(rmc.getAllocatedNode()).thenReturn(host);
         when(rmc.getNodeLabelExpression()).thenReturn(exp);
         when(rmc.getNodeLabelExpression()).thenReturn(exp);
         when(rmc.getAllocatedResource()).thenReturn(res);
         when(rmc.getAllocatedResource()).thenReturn(res);

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java

@@ -36,6 +36,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.resource.Priority;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 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.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
@@ -1318,8 +1320,10 @@ public class TestProportionalCapacityPreemptionPolicy {
     Container c = mock(Container.class);
     Container c = mock(Container.class);
     when(c.getResource()).thenReturn(r);
     when(c.getResource()).thenReturn(r);
     when(c.getPriority()).thenReturn(Priority.create(cpriority));
     when(c.getPriority()).thenReturn(Priority.create(cpriority));
+    SchedulerRequestKey sk = SchedulerRequestKey.extractFrom(c);
     RMContainer mC = mock(RMContainer.class);
     RMContainer mC = mock(RMContainer.class);
     when(mC.getContainerId()).thenReturn(cId);
     when(mC.getContainerId()).thenReturn(cId);
+    when(mC.getAllocatedSchedulerKey()).thenReturn(sk);
     when(mC.getContainer()).thenReturn(c);
     when(mC.getContainer()).thenReturn(c);
     when(mC.getApplicationAttemptId()).thenReturn(appAttId);
     when(mC.getApplicationAttemptId()).thenReturn(appAttId);
     when(mC.getAllocatedResource()).thenReturn(r);
     when(mC.getAllocatedResource()).thenReturn(r);

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

@@ -120,7 +120,8 @@ public class TestRMContainerImpl {
     assertEquals(RMContainerState.NEW, rmContainer.getState());
     assertEquals(RMContainerState.NEW, rmContainer.getState());
     assertEquals(resource, rmContainer.getAllocatedResource());
     assertEquals(resource, rmContainer.getAllocatedResource());
     assertEquals(nodeId, rmContainer.getAllocatedNode());
     assertEquals(nodeId, rmContainer.getAllocatedNode());
-    assertEquals(priority, rmContainer.getAllocatedPriority());
+    assertEquals(priority,
+        rmContainer.getAllocatedSchedulerKey().getPriority());
     verify(writer).containerStarted(any(RMContainer.class));
     verify(writer).containerStarted(any(RMContainer.class));
     verify(publisher).containerCreated(any(RMContainer.class), anyLong());
     verify(publisher).containerCreated(any(RMContainer.class), anyLong());
 
 
@@ -221,7 +222,8 @@ public class TestRMContainerImpl {
     assertEquals(RMContainerState.NEW, rmContainer.getState());
     assertEquals(RMContainerState.NEW, rmContainer.getState());
     assertEquals(resource, rmContainer.getAllocatedResource());
     assertEquals(resource, rmContainer.getAllocatedResource());
     assertEquals(nodeId, rmContainer.getAllocatedNode());
     assertEquals(nodeId, rmContainer.getAllocatedNode());
-    assertEquals(priority, rmContainer.getAllocatedPriority());
+    assertEquals(priority,
+        rmContainer.getAllocatedSchedulerKey().getPriority());
     verify(writer).containerStarted(any(RMContainer.class));
     verify(writer).containerStarted(any(RMContainer.class));
     verify(publisher).containerCreated(any(RMContainer.class), anyLong());
     verify(publisher).containerCreated(any(RMContainer.class), anyLong());
 
 

+ 19 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java

@@ -45,6 +45,8 @@ import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey;
+
 public class TestSchedulerApplicationAttempt {
 public class TestSchedulerApplicationAttempt {
 
 
   private static final NodeId nodeId = NodeId.newInstance("somehost", 5);
   private static final NodeId nodeId = NodeId.newInstance("somehost", 5);
@@ -88,7 +90,8 @@ public class TestSchedulerApplicationAttempt {
     RMContainer container1 = createRMContainer(appAttId, 1, requestedResource);
     RMContainer container1 = createRMContainer(appAttId, 1, requestedResource);
     app.liveContainers.put(container1.getContainerId(), container1);
     app.liveContainers.put(container1.getContainerId(), container1);
     SchedulerNode node = createNode();
     SchedulerNode node = createNode();
-    app.appSchedulingInfo.allocate(NodeType.OFF_SWITCH, node, requestedPriority,
+    app.appSchedulingInfo.allocate(NodeType.OFF_SWITCH, node,
+        toSchedulerKey(requestedPriority),
         request, container1.getContainer());
         request, container1.getContainer());
     
     
     // Reserved container
     // Reserved container
@@ -98,7 +101,7 @@ public class TestSchedulerApplicationAttempt {
         node.getNodeID(), prio1);
         node.getNodeID(), prio1);
     Map<NodeId, RMContainer> reservations = new HashMap<NodeId, RMContainer>();
     Map<NodeId, RMContainer> reservations = new HashMap<NodeId, RMContainer>();
     reservations.put(node.getNodeID(), container2);
     reservations.put(node.getNodeID(), container2);
-    app.reservedContainers.put(prio1, reservations);
+    app.reservedContainers.put(toSchedulerKey(prio1), reservations);
     oldMetrics.reserveResource(user, reservedResource);
     oldMetrics.reserveResource(user, reservedResource);
     
     
     checkQueueMetrics(oldMetrics, 1, 1, 1536, 2, 2048, 3, 3072, 4);
     checkQueueMetrics(oldMetrics, 1, 1, 1536, 2, 2048, 3, 3072, 4);
@@ -137,7 +140,8 @@ public class TestSchedulerApplicationAttempt {
       int id, Resource resource, NodeId nodeId, Priority reservedPriority) {
       int id, Resource resource, NodeId nodeId, Priority reservedPriority) {
     RMContainer container = createRMContainer(appAttId, id, resource);
     RMContainer container = createRMContainer(appAttId, id, resource);
     when(container.getReservedResource()).thenReturn(resource);
     when(container.getReservedResource()).thenReturn(resource);
-    when(container.getReservedPriority()).thenReturn(reservedPriority);
+    when(container.getReservedSchedulerKey())
+        .thenReturn(toSchedulerKey(reservedPriority));
     when(container.getReservedNode()).thenReturn(nodeId);
     when(container.getReservedNode()).thenReturn(nodeId);
     return container;
     return container;
   }
   }
@@ -260,16 +264,19 @@ public class TestSchedulerApplicationAttempt {
     SchedulerApplicationAttempt app = new SchedulerApplicationAttempt(
     SchedulerApplicationAttempt app = new SchedulerApplicationAttempt(
         attemptId, "user", queue, queue.getActiveUsersManager(), rmContext);
         attemptId, "user", queue, queue.getActiveUsersManager(), rmContext);
     Priority priority = Priority.newInstance(1);
     Priority priority = Priority.newInstance(1);
-    assertEquals(0, app.getSchedulingOpportunities(priority));
-    app.addSchedulingOpportunity(priority);
-    assertEquals(1, app.getSchedulingOpportunities(priority));
+    SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
+    assertEquals(0, app.getSchedulingOpportunities(schedulerKey));
+    app.addSchedulingOpportunity(schedulerKey);
+    assertEquals(1, app.getSchedulingOpportunities(schedulerKey));
     // verify the count is capped at MAX_VALUE and does not overflow
     // verify the count is capped at MAX_VALUE and does not overflow
-    app.setSchedulingOpportunities(priority, Integer.MAX_VALUE - 1);
+    app.setSchedulingOpportunities(schedulerKey, Integer.MAX_VALUE - 1);
     assertEquals(Integer.MAX_VALUE - 1,
     assertEquals(Integer.MAX_VALUE - 1,
-        app.getSchedulingOpportunities(priority));
-    app.addSchedulingOpportunity(priority);
-    assertEquals(Integer.MAX_VALUE, app.getSchedulingOpportunities(priority));
-    app.addSchedulingOpportunity(priority);
-    assertEquals(Integer.MAX_VALUE, app.getSchedulingOpportunities(priority));
+        app.getSchedulingOpportunities(schedulerKey));
+    app.addSchedulingOpportunity(schedulerKey);
+    assertEquals(Integer.MAX_VALUE,
+        app.getSchedulingOpportunities(schedulerKey));
+    app.addSchedulingOpportunity(schedulerKey);
+    assertEquals(Integer.MAX_VALUE,
+        app.getSchedulingOpportunities(schedulerKey));
   }
   }
 }
 }

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

@@ -116,6 +116,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
@@ -1244,7 +1246,7 @@ public class TestCapacityScheduler {
     rm1.stop();
     rm1.stop();
   }
   }
   
   
-  @Test(timeout = 30000)
+  @Test(timeout = 300000)
   public void testRecoverRequestAfterPreemption() throws Exception {
   public void testRecoverRequestAfterPreemption() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
@@ -1277,8 +1279,8 @@ public class TestCapacityScheduler {
 
 
       // Already the node local resource request is cleared from RM after
       // Already the node local resource request is cleared from RM after
       // allocation.
       // allocation.
-      Assert.assertNull(app.getResourceRequest(request.getPriority(),
-          request.getResourceName()));
+      Assert.assertNull(app.getResourceRequest(
+          SchedulerRequestKey.create(request), request.getResourceName()));
     }
     }
 
 
     // Call killContainer to preempt the container
     // Call killContainer to preempt the container
@@ -1290,7 +1292,7 @@ public class TestCapacityScheduler {
       // handling.
       // handling.
       Assert.assertEquals(
       Assert.assertEquals(
           1,
           1,
-          app.getResourceRequest(request.getPriority(),
+          app.getResourceRequest(SchedulerRequestKey.create(request),
               request.getResourceName()).getNumContainers());
               request.getResourceName()).getNumContainers());
     }
     }
 
 

+ 98 - 72
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java

@@ -68,6 +68,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
+
+
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
@@ -89,6 +92,8 @@ import org.junit.Test;
 import org.mockito.Matchers;
 import org.mockito.Matchers;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey;
+
 public class TestLeafQueue {  
 public class TestLeafQueue {  
   private final RecordFactory recordFactory = 
   private final RecordFactory recordFactory = 
       RecordFactoryProvider.getRecordFactory(null);
       RecordFactoryProvider.getRecordFactory(null);
@@ -731,6 +736,7 @@ public class TestLeafQueue {
             qb.getActiveUsersManager(), spyRMContext);
             qb.getActiveUsersManager(), spyRMContext);
     qb.submitApplicationAttempt(app_0, user_0);
     qb.submitApplicationAttempt(app_0, user_0);
     Priority u0Priority = TestUtils.createMockPriority(1);
     Priority u0Priority = TestUtils.createMockPriority(1);
+    SchedulerRequestKey u0SchedKey = toSchedulerKey(u0Priority);
     app_0.updateResourceRequests(Collections.singletonList(
     app_0.updateResourceRequests(Collections.singletonList(
         TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
         TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
             u0Priority, recordFactory)));
             u0Priority, recordFactory)));
@@ -753,6 +759,7 @@ public class TestLeafQueue {
         new FiCaSchedulerApp(appAttemptId_2, user_1, qb,
         new FiCaSchedulerApp(appAttemptId_2, user_1, qb,
             qb.getActiveUsersManager(), spyRMContext);
             qb.getActiveUsersManager(), spyRMContext);
     Priority u1Priority = TestUtils.createMockPriority(2);
     Priority u1Priority = TestUtils.createMockPriority(2);
+    SchedulerRequestKey u1SchedKey = toSchedulerKey(u1Priority);
     app_2.updateResourceRequests(Collections.singletonList(
     app_2.updateResourceRequests(Collections.singletonList(
         TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
         TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
             u1Priority, recordFactory)));
             u1Priority, recordFactory)));
@@ -773,9 +780,9 @@ public class TestLeafQueue {
     //test case 3
     //test case 3
     qb.finishApplication(app_0.getApplicationId(), user_0);
     qb.finishApplication(app_0.getApplicationId(), user_0);
     qb.finishApplication(app_2.getApplicationId(), user_1);
     qb.finishApplication(app_2.getApplicationId(), user_1);
-    qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority),
+    qb.releaseResource(clusterResource, app_0, app_0.getResource(u0SchedKey),
         null, null, false);
         null, null, false);
-    qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority),
+    qb.releaseResource(clusterResource, app_2, app_2.getResource(u1SchedKey),
         null, null, false);
         null, null, false);
 
 
     qb.setUserLimit(50);
     qb.setUserLimit(50);
@@ -1452,7 +1459,7 @@ public class TestLeafQueue {
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(4*GB, app_1.getCurrentReservation().getMemorySize());
     assertEquals(4*GB, app_1.getCurrentReservation().getMemorySize());
     assertEquals(1*GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(1*GB, node_0.getAllocatedResource().getMemorySize());
-    assertEquals(1, app_1.getReReservations(priority));
+    assertEquals(1, app_1.getReReservations(toSchedulerKey(priority)));
 
 
     // Re-reserve
     // Re-reserve
     a.assignContainers(clusterResource, node_0,
     a.assignContainers(clusterResource, node_0,
@@ -1462,7 +1469,7 @@ public class TestLeafQueue {
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     assertEquals(4*GB, app_1.getCurrentReservation().getMemorySize());
     assertEquals(4*GB, app_1.getCurrentReservation().getMemorySize());
     assertEquals(1*GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(1*GB, node_0.getAllocatedResource().getMemorySize());
-    assertEquals(2, app_1.getReReservations(priority));
+    assertEquals(2, app_1.getReReservations(toSchedulerKey(priority)));
     
     
     // Try to schedule on node_1 now, should *move* the reservation
     // Try to schedule on node_1 now, should *move* the reservation
     a.assignContainers(clusterResource, node_1,
     a.assignContainers(clusterResource, node_1,
@@ -1474,7 +1481,7 @@ public class TestLeafQueue {
     assertEquals(4*GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(4*GB, node_1.getAllocatedResource().getMemorySize());
     // Doesn't change yet... only when reservation is cancelled or a different
     // Doesn't change yet... only when reservation is cancelled or a different
     // container is reserved
     // container is reserved
-    assertEquals(2, app_1.getReReservations(priority)); 
+    assertEquals(2, app_1.getReReservations(toSchedulerKey(priority)));
     
     
     // Now finish another container from app_0 and see the reservation cancelled
     // Now finish another container from app_0 and see the reservation cancelled
     rmContainer = app_0.getLiveContainers().iterator().next();
     rmContainer = app_0.getLiveContainers().iterator().next();
@@ -1564,29 +1571,30 @@ public class TestLeafQueue {
 
 
     // Start testing...
     // Start testing...
     CSAssignment assignment = null;
     CSAssignment assignment = null;
-    
+
+    SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
     // Start with off switch, shouldn't allocate due to delay scheduling
     // Start with off switch, shouldn't allocate due to delay scheduling
     assignment = a.assignContainers(clusterResource, node_2,
     assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(1, app_0.getSchedulingOpportunities(priority));
-    assertEquals(3, app_0.getTotalRequiredResources(priority));
+    assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
 
 
     // Another off switch, shouldn't allocate due to delay scheduling
     // Another off switch, shouldn't allocate due to delay scheduling
     assignment = a.assignContainers(clusterResource, node_2,
     assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(2, app_0.getSchedulingOpportunities(priority));
-    assertEquals(3, app_0.getTotalRequiredResources(priority));
+    assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
     
     
     // Another off switch, shouldn't allocate due to delay scheduling
     // Another off switch, shouldn't allocate due to delay scheduling
     assignment = a.assignContainers(clusterResource, node_2,
     assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(3, app_0.getSchedulingOpportunities(priority));
-    assertEquals(3, app_0.getTotalRequiredResources(priority));
+    assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
     
     
     // Another off switch, now we should allocate 
     // Another off switch, now we should allocate 
@@ -1594,22 +1602,25 @@ public class TestLeafQueue {
     assignment = a.assignContainers(clusterResource, node_2,
     assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
     verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
-    assertEquals(4, app_0.getSchedulingOpportunities(priority)); // should NOT reset
-    assertEquals(2, app_0.getTotalRequiredResources(priority));
+    // should NOT reset
+    assertEquals(4, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(2, app_0.getTotalRequiredResources(schedulerKey));
     
     
     // NODE_LOCAL - node_0
     // NODE_LOCAL - node_0
     assignment = a.assignContainers(clusterResource, node_0,
     assignment = a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
-    assertEquals(1, app_0.getTotalRequiredResources(priority));
+    // should reset
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
     
     
     // NODE_LOCAL - node_1
     // NODE_LOCAL - node_1
     assignment = a.assignContainers(clusterResource, node_1,
     assignment = a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
-    assertEquals(0, app_0.getTotalRequiredResources(priority));
+    // should reset
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
     assertEquals(NodeType.NODE_LOCAL, assignment.getType());
     assertEquals(NodeType.NODE_LOCAL, assignment.getType());
     
     
     // Add 1 more request to check for RACK_LOCAL
     // Add 1 more request to check for RACK_LOCAL
@@ -1624,7 +1635,7 @@ public class TestLeafQueue {
         TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 4, // one extra
         TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 4, // one extra
             true, priority, recordFactory));
             true, priority, recordFactory));
     app_0.updateResourceRequests(app_0_requests_0);
     app_0.updateResourceRequests(app_0_requests_0);
-    assertEquals(4, app_0.getTotalRequiredResources(priority));
+    assertEquals(4, app_0.getTotalRequiredResources(schedulerKey));
     
     
     String host_3 = "127.0.0.4"; // on rack_1
     String host_3 = "127.0.0.4"; // on rack_1
     FiCaSchedulerNode node_3 = TestUtils.getMockNode(host_3, rack_1, 0, 8*GB);
     FiCaSchedulerNode node_3 = TestUtils.getMockNode(host_3, rack_1, 0, 8*GB);
@@ -1636,21 +1647,22 @@ public class TestLeafQueue {
     // Shouldn't assign RACK_LOCAL yet
     // Shouldn't assign RACK_LOCAL yet
     assignment = a.assignContainers(clusterResource, node_3,
     assignment = a.assignContainers(clusterResource, node_3,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
-    assertEquals(1, app_0.getSchedulingOpportunities(priority));
-    assertEquals(4, app_0.getTotalRequiredResources(priority));
+    assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(4, app_0.getTotalRequiredResources(schedulerKey));
 
 
     // Should assign RACK_LOCAL now
     // Should assign RACK_LOCAL now
     assignment = a.assignContainers(clusterResource, node_3,
     assignment = a.assignContainers(clusterResource, node_3,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
     verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
-    assertEquals(3, app_0.getTotalRequiredResources(priority));
+    // should reset
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
     
     
     // Shouldn't assign RACK_LOCAL because schedulingOpportunities should have gotten reset.
     // Shouldn't assign RACK_LOCAL because schedulingOpportunities should have gotten reset.
     assignment = a.assignContainers(clusterResource, node_3,
     assignment = a.assignContainers(clusterResource, node_3,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
-    assertEquals(1, app_0.getSchedulingOpportunities(priority));
-    assertEquals(3, app_0.getTotalRequiredResources(priority));
+    assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
 
 
     // Next time we schedule RACK_LOCAL, don't reset
     // Next time we schedule RACK_LOCAL, don't reset
     doReturn(false).when(a).getRackLocalityFullReset();
     doReturn(false).when(a).getRackLocalityFullReset();
@@ -1659,19 +1671,21 @@ public class TestLeafQueue {
     assignment = a.assignContainers(clusterResource, node_3,
     assignment = a.assignContainers(clusterResource, node_3,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
     verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
-    assertEquals(2, app_0.getSchedulingOpportunities(priority)); // should NOT reset
-    assertEquals(2, app_0.getTotalRequiredResources(priority));
+    // should NOT reset
+    assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(2, app_0.getTotalRequiredResources(schedulerKey));
 
 
     // Another RACK_LOCAL since schedulingOpportunities not reset
     // Another RACK_LOCAL since schedulingOpportunities not reset
     assignment = a.assignContainers(clusterResource, node_3,
     assignment = a.assignContainers(clusterResource, node_3,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
     verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
-    assertEquals(3, app_0.getSchedulingOpportunities(priority)); // should NOT reset
-    assertEquals(1, app_0.getTotalRequiredResources(priority));
+    // should NOT reset
+    assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
     
     
     // Add a request larger than cluster size to verify
     // Add a request larger than cluster size to verify
     // OFF_SWITCH delay is capped by cluster size
     // OFF_SWITCH delay is capped by cluster size
-    app_0.resetSchedulingOpportunities(priority);
+    app_0.resetSchedulingOpportunities(schedulerKey);
     app_0_requests_0.clear();
     app_0_requests_0.clear();
     app_0_requests_0.add(
     app_0_requests_0.add(
         TestUtils.createResourceRequest(host_0, 1*GB, 100,
         TestUtils.createResourceRequest(host_0, 1*GB, 100,
@@ -1690,13 +1704,13 @@ public class TestLeafQueue {
           a.assignContainers(clusterResource, node_2, new ResourceLimits(
           a.assignContainers(clusterResource, node_2, new ResourceLimits(
               clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
               clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
       verifyNoContainerAllocated(assignment);
       verifyNoContainerAllocated(assignment);
-      assertEquals(i+1, app_0.getSchedulingOpportunities(priority));
+      assertEquals(i+1, app_0.getSchedulingOpportunities(schedulerKey));
     }
     }
     // delay should be capped at numNodes so next one should allocate
     // delay should be capped at numNodes so next one should allocate
     assignment = a.assignContainers(clusterResource, node_2,
     assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
     verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
-    assertEquals(numNodes+1, app_0.getSchedulingOpportunities(priority));
+    assertEquals(numNodes+1, app_0.getSchedulingOpportunities(schedulerKey));
   }
   }
 
 
   @Test
   @Test
@@ -1738,6 +1752,7 @@ public class TestLeafQueue {
     
     
     // P1
     // P1
     Priority priority_1 = TestUtils.createMockPriority(1);
     Priority priority_1 = TestUtils.createMockPriority(1);
+    SchedulerRequestKey schedulerKey1 = toSchedulerKey(priority_1);
     app_0_requests_0.add(
     app_0_requests_0.add(
         TestUtils.createResourceRequest(host_0, 1*GB, 1, 
         TestUtils.createResourceRequest(host_0, 1*GB, 1, 
             true, priority_1, recordFactory));
             true, priority_1, recordFactory));
@@ -1756,6 +1771,7 @@ public class TestLeafQueue {
     
     
     // P2
     // P2
     Priority priority_2 = TestUtils.createMockPriority(2);
     Priority priority_2 = TestUtils.createMockPriority(2);
+    SchedulerRequestKey schedulerKey2 = toSchedulerKey(priority_2);
     app_0_requests_0.add(
     app_0_requests_0.add(
         TestUtils.createResourceRequest(host_2, 2*GB, 1, 
         TestUtils.createResourceRequest(host_2, 2*GB, 1, 
             true, priority_2, recordFactory));
             true, priority_2, recordFactory));
@@ -1775,47 +1791,47 @@ public class TestLeafQueue {
     CSAssignment assignment = a.assignContainers(clusterResource, node_2,
     CSAssignment assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(1, app_0.getSchedulingOpportunities(priority_1));
-    assertEquals(2, app_0.getTotalRequiredResources(priority_1));
-    assertEquals(0, app_0.getSchedulingOpportunities(priority_2));
-    assertEquals(1, app_0.getTotalRequiredResources(priority_2));
+    assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey1));
+    assertEquals(2, app_0.getTotalRequiredResources(schedulerKey1));
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
+    assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
 
 
     // Another off-switch, shouldn't allocate P1 due to delay scheduling
     // Another off-switch, shouldn't allocate P1 due to delay scheduling
     // thus, no P2 either!
     // thus, no P2 either!
     assignment = a.assignContainers(clusterResource, node_2,
     assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(2, app_0.getSchedulingOpportunities(priority_1));
-    assertEquals(2, app_0.getTotalRequiredResources(priority_1));
-    assertEquals(0, app_0.getSchedulingOpportunities(priority_2));
-    assertEquals(1, app_0.getTotalRequiredResources(priority_2));
+    assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey1));
+    assertEquals(2, app_0.getTotalRequiredResources(schedulerKey1));
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
+    assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
 
 
     // Another off-switch, shouldn't allocate OFF_SWITCH P1
     // Another off-switch, shouldn't allocate OFF_SWITCH P1
     assignment = a.assignContainers(clusterResource, node_2,
     assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
     verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
-    assertEquals(3, app_0.getSchedulingOpportunities(priority_1));
-    assertEquals(1, app_0.getTotalRequiredResources(priority_1));
-    assertEquals(0, app_0.getSchedulingOpportunities(priority_2));
-    assertEquals(1, app_0.getTotalRequiredResources(priority_2));
+    assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey1));
+    assertEquals(1, app_0.getTotalRequiredResources(schedulerKey1));
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
+    assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
 
 
     // Now, DATA_LOCAL for P1
     // Now, DATA_LOCAL for P1
     assignment = a.assignContainers(clusterResource, node_0,
     assignment = a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority_1));
-    assertEquals(0, app_0.getTotalRequiredResources(priority_1));
-    assertEquals(0, app_0.getSchedulingOpportunities(priority_2));
-    assertEquals(1, app_0.getTotalRequiredResources(priority_2));
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1));
+    assertEquals(0, app_0.getTotalRequiredResources(schedulerKey1));
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
+    assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
 
 
     // Now, OFF_SWITCH for P2
     // Now, OFF_SWITCH for P2
     assignment = a.assignContainers(clusterResource, node_1,
     assignment = a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
     verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority_1));
-    assertEquals(0, app_0.getTotalRequiredResources(priority_1));
-    assertEquals(1, app_0.getSchedulingOpportunities(priority_2));
-    assertEquals(0, app_0.getTotalRequiredResources(priority_2));
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1));
+    assertEquals(0, app_0.getTotalRequiredResources(schedulerKey1));
+    assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey2));
+    assertEquals(0, app_0.getTotalRequiredResources(schedulerKey2));
 
 
   }
   }
   
   
@@ -1855,6 +1871,7 @@ public class TestLeafQueue {
 
 
     // Setup resource-requests and submit
     // Setup resource-requests and submit
     Priority priority = TestUtils.createMockPriority(1);
     Priority priority = TestUtils.createMockPriority(1);
+    SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
     List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
     List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
     app_0_requests_0.add(
     app_0_requests_0.add(
         TestUtils.createResourceRequest(host_0_0, 1*GB, 1, 
         TestUtils.createResourceRequest(host_0_0, 1*GB, 1, 
@@ -1878,7 +1895,7 @@ public class TestLeafQueue {
     // Add one request
     // Add one request
     app_0_requests_0.clear();
     app_0_requests_0.clear();
     app_0_requests_0.add(
     app_0_requests_0.add(
-        TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, // only one
+        TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, // only 1
             true, priority, recordFactory));
             true, priority, recordFactory));
     app_0.updateResourceRequests(app_0_requests_0);
     app_0.updateResourceRequests(app_0_requests_0);
     
     
@@ -1886,17 +1903,19 @@ public class TestLeafQueue {
     CSAssignment assignment = a.assignContainers(clusterResource, node_0_0,
     CSAssignment assignment = a.assignContainers(clusterResource, node_0_0,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
-    assertEquals(0, app_0.getTotalRequiredResources(priority));
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
+    // should reset
+    assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
 
 
     // No allocation on node_1_0 even though it's node/rack local since
     // No allocation on node_1_0 even though it's node/rack local since
     // required(ANY) == 0
     // required(ANY) == 0
     assignment = a.assignContainers(clusterResource, node_1_0,
     assignment = a.assignContainers(clusterResource, node_1_0,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // Still zero
-                                                               // since #req=0
-    assertEquals(0, app_0.getTotalRequiredResources(priority));
+    // Still zero
+    // since #req=0
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
     
     
     // Add one request
     // Add one request
     app_0_requests_0.clear();
     app_0_requests_0.clear();
@@ -1910,15 +1929,16 @@ public class TestLeafQueue {
     assignment = a.assignContainers(clusterResource, node_0_1,
     assignment = a.assignContainers(clusterResource, node_0_1,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(1, app_0.getSchedulingOpportunities(priority)); 
-    assertEquals(1, app_0.getTotalRequiredResources(priority));
+    assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
     
     
     // NODE_LOCAL - node_1
     // NODE_LOCAL - node_1
     assignment = a.assignContainers(clusterResource, node_1_0,
     assignment = a.assignContainers(clusterResource, node_1_0,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
-    assertEquals(0, app_0.getTotalRequiredResources(priority));
+    // should reset
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
   }
   }
 
 
   @Test (timeout = 30000)
   @Test (timeout = 30000)
@@ -2142,6 +2162,7 @@ public class TestLeafQueue {
     // host_1_1: 8G
     // host_1_1: 8G
     // Blacklist: <host_0_0>
     // Blacklist: <host_0_0>
     Priority priority = TestUtils.createMockPriority(1);
     Priority priority = TestUtils.createMockPriority(1);
+    SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
     List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
     List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
     app_0_requests_0.add(
     app_0_requests_0.add(
         TestUtils.createResourceRequest(host_0_0, 1*GB, 1, 
         TestUtils.createResourceRequest(host_0_0, 1*GB, 1, 
@@ -2169,7 +2190,8 @@ public class TestLeafQueue {
         a.assignContainers(clusterResource, node_0_1, new ResourceLimits(
         a.assignContainers(clusterResource, node_0_1, new ResourceLimits(
             clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
             clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
+    // should be 0
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
     
     
     // resourceName: <priority, memory, #containers, relaxLocality>
     // resourceName: <priority, memory, #containers, relaxLocality>
     // host_0_0: < 1, 1GB, 1, true >
     // host_0_0: < 1, 1GB, 1, true >
@@ -2191,7 +2213,8 @@ public class TestLeafQueue {
     assignment = a.assignContainers(clusterResource, node_1_1, 
     assignment = a.assignContainers(clusterResource, node_1_1, 
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
+    // should be 0
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
     
     
     // Allow rack-locality for rack_1, but blacklist node_1_1
     // Allow rack-locality for rack_1, but blacklist node_1_1
     app_0_requests_0.add(
     app_0_requests_0.add(
@@ -2221,7 +2244,8 @@ public class TestLeafQueue {
     assignment = a.assignContainers(clusterResource, node_1_1, 
     assignment = a.assignContainers(clusterResource, node_1_1, 
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
+    // should be 0
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
 
 
     // Now, remove node_1_1 from blacklist, but add rack_1 to blacklist
     // Now, remove node_1_1 from blacklist, but add rack_1 to blacklist
     app_0.updateResourceRequests(app_0_requests_0);
     app_0.updateResourceRequests(app_0_requests_0);
@@ -2249,7 +2273,8 @@ public class TestLeafQueue {
     assignment = a.assignContainers(clusterResource, node_1_1, 
     assignment = a.assignContainers(clusterResource, node_1_1, 
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
+    // should be 0
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
     
     
     // Now remove rack_1 from blacklist
     // Now remove rack_1 from blacklist
     app_0.updateResourceRequests(app_0_requests_0);
     app_0.updateResourceRequests(app_0_requests_0);
@@ -2275,8 +2300,8 @@ public class TestLeafQueue {
     assignment = a.assignContainers(clusterResource, node_1_1, 
     assignment = a.assignContainers(clusterResource, node_1_1, 
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyNoContainerAllocated(assignment);
     verifyNoContainerAllocated(assignment);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); 
-    assertEquals(1, app_0.getTotalRequiredResources(priority));
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
 
 
     // Now sanity-check node_local
     // Now sanity-check node_local
     app_0_requests_0.add(
     app_0_requests_0.add(
@@ -2305,8 +2330,8 @@ public class TestLeafQueue {
     assignment = a.assignContainers(clusterResource, node_1_0, 
     assignment = a.assignContainers(clusterResource, node_1_0, 
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
-    assertEquals(0, app_0.getSchedulingOpportunities(priority)); 
-    assertEquals(0, app_0.getTotalRequiredResources(priority));
+    assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
 
 
   }
   }
   
   
@@ -2667,6 +2692,7 @@ public class TestLeafQueue {
     // App0 has node local request for host_0/host_1, and app1 has node local
     // App0 has node local request for host_0/host_1, and app1 has node local
     // request for host2.
     // request for host2.
     Priority priority = TestUtils.createMockPriority(1);
     Priority priority = TestUtils.createMockPriority(1);
+    SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
     List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
     List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
     app_0_requests_0.add(
     app_0_requests_0.add(
         TestUtils.createResourceRequest(host_0, 1*GB, 1, 
         TestUtils.createResourceRequest(host_0, 1*GB, 1, 
@@ -2706,8 +2732,8 @@ public class TestLeafQueue {
     assignment = a.assignContainers(clusterResource, node_2,
     assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
         new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
     verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
-    assertEquals(1, app_0.getSchedulingOpportunities(priority));
-    assertEquals(3, app_0.getTotalRequiredResources(priority));
+    assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
+    assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
     assertEquals(0, app_0.getLiveContainers().size());
     assertEquals(0, app_0.getLiveContainers().size());
     assertEquals(1, app_1.getLiveContainers().size());
     assertEquals(1, app_1.getLiveContainers().size());
   }
   }

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

@@ -32,7 +32,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
@@ -48,6 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 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.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
+
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@@ -549,7 +549,7 @@ public class TestNodeLabelContainerAllocation {
     FiCaSchedulerApp app = cs.getApplicationAttempt(attemptId);
     FiCaSchedulerApp app = cs.getApplicationAttempt(attemptId);
     ResourceRequest rr =
     ResourceRequest rr =
         app.getAppSchedulingInfo().getResourceRequest(
         app.getAppSchedulingInfo().getResourceRequest(
-            Priority.newInstance(priority), "*");
+            TestUtils.toSchedulerKey(priority), "*");
     Assert.assertEquals(memory,
     Assert.assertEquals(memory,
         rr.getCapability().getMemorySize() * rr.getNumContainers());
         rr.getCapability().getMemorySize() * rr.getNumContainers());
   }
   }

+ 36 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java

@@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
@@ -71,6 +73,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey;
+
 public class TestReservations {
 public class TestReservations {
 
 
   private static final Log LOG = LogFactory.getLog(TestReservations.class);
   private static final Log LOG = LogFactory.getLog(TestReservations.class);
@@ -300,7 +304,8 @@ public class TestReservations {
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
     assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
-    assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(2, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
 
 
     // try to assign reducer (5G on node 0 and should reserve)
     // try to assign reducer (5G on node 0 and should reserve)
     a.assignContainers(clusterResource, node_0,
     a.assignContainers(clusterResource, node_0,
@@ -316,7 +321,8 @@ public class TestReservations {
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
     assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
-    assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(2, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
 
 
     // assign reducer to node 2
     // assign reducer to node 2
     a.assignContainers(clusterResource, node_2,
     a.assignContainers(clusterResource, node_2,
@@ -332,7 +338,8 @@ public class TestReservations {
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
-    assertEquals(1, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(1, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
 
 
     // node_1 heartbeat and unreserves from node_0 in order to allocate
     // node_1 heartbeat and unreserves from node_0 in order to allocate
     // on node_1
     // on node_1
@@ -348,7 +355,8 @@ public class TestReservations {
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(8 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(8 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
-    assertEquals(0, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(0, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
   }
   }
 
 
   // Test that hitting a reservation limit and needing to unreserve
   // Test that hitting a reservation limit and needing to unreserve
@@ -597,7 +605,8 @@ public class TestReservations {
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
     assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
-    assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(2, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
 
 
     // try to assign reducer (5G on node 0 and should reserve)
     // try to assign reducer (5G on node 0 and should reserve)
     a.assignContainers(clusterResource, node_0,
     a.assignContainers(clusterResource, node_0,
@@ -613,7 +622,8 @@ public class TestReservations {
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
     assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
-    assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(2, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
 
 
     // assign reducer to node 2
     // assign reducer to node 2
     a.assignContainers(clusterResource, node_2,
     a.assignContainers(clusterResource, node_2,
@@ -629,7 +639,8 @@ public class TestReservations {
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
-    assertEquals(1, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(1, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
 
 
     // node_1 heartbeat and won't unreserve from node_0, potentially stuck
     // node_1 heartbeat and won't unreserve from node_0, potentially stuck
     // if AM doesn't handle
     // if AM doesn't handle
@@ -646,7 +657,8 @@ public class TestReservations {
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
-    assertEquals(1, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(1, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
   }
   }
 
 
   @Test
   @Test
@@ -754,7 +766,8 @@ public class TestReservations {
     assertEquals(null, node_0.getReservedContainer());
     assertEquals(null, node_0.getReservedContainer());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
-    assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(2, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
 
 
     // try to assign reducer (5G on node 0 and should reserve)
     // try to assign reducer (5G on node 0 and should reserve)
     a.assignContainers(clusterResource, node_0,
     a.assignContainers(clusterResource, node_0,
@@ -769,7 +782,8 @@ public class TestReservations {
         .getMemorySize());
         .getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
-    assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(2, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
 
 
     // could allocate but told need to unreserve first
     // could allocate but told need to unreserve first
     a.assignContainers(clusterResource, node_1,
     a.assignContainers(clusterResource, node_1,
@@ -783,7 +797,8 @@ public class TestReservations {
     assertEquals(null, node_0.getReservedContainer());
     assertEquals(null, node_0.getReservedContainer());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
     assertEquals(8 * GB, node_1.getAllocatedResource().getMemorySize());
     assertEquals(8 * GB, node_1.getAllocatedResource().getMemorySize());
-    assertEquals(1, app_0.getTotalRequiredResources(priorityReduce));
+    assertEquals(1, app_0.getTotalRequiredResources(
+        toSchedulerKey(priorityReduce)));
   }
   }
 
 
   @Test
   @Test
@@ -808,7 +823,8 @@ public class TestReservations {
     Resource clusterResource = Resources.createResource(2 * 8 * GB);
     Resource clusterResource = Resources.createResource(2 * 8 * GB);
 
 
     // Setup resource-requests
     // Setup resource-requests
-    Priority priorityMap = TestUtils.createMockPriority(5);
+    Priority p = TestUtils.createMockPriority(5);
+    SchedulerRequestKey priorityMap = toSchedulerKey(p);
     Resource capability = Resources.createResource(2*GB, 0);
     Resource capability = Resources.createResource(2*GB, 0);
 
 
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
@@ -826,12 +842,14 @@ public class TestReservations {
         app_0.getApplicationId(), 1);
         app_0.getApplicationId(), 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     Container container = TestUtils.getMockContainer(containerId,
     Container container = TestUtils.getMockContainer(containerId,
-        node_1.getNodeID(), Resources.createResource(2*GB), priorityMap);
+        node_1.getNodeID(), Resources.createResource(2*GB),
+        priorityMap.getPriority());
     RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
     RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
         node_1.getNodeID(), "user", rmContext);
         node_1.getNodeID(), "user", rmContext);
 
 
     Container container_1 = TestUtils.getMockContainer(containerId,
     Container container_1 = TestUtils.getMockContainer(containerId,
-        node_0.getNodeID(), Resources.createResource(1*GB), priorityMap);
+        node_0.getNodeID(), Resources.createResource(1*GB),
+        priorityMap.getPriority());
     RMContainer rmContainer_1 = new RMContainerImpl(container_1, appAttemptId,
     RMContainer rmContainer_1 = new RMContainerImpl(container_1, appAttemptId,
         node_0.getNodeID(), "user", rmContext);
         node_0.getNodeID(), "user", rmContext);
 
 
@@ -878,7 +896,8 @@ public class TestReservations {
         8 * GB);
         8 * GB);
 
 
     // Setup resource-requests
     // Setup resource-requests
-    Priority priorityMap = TestUtils.createMockPriority(5);
+    Priority p = TestUtils.createMockPriority(5);
+    SchedulerRequestKey priorityMap = toSchedulerKey(p);
     Resource capability = Resources.createResource(2 * GB, 0);
     Resource capability = Resources.createResource(2 * GB, 0);
 
 
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
@@ -896,7 +915,8 @@ public class TestReservations {
         app_0.getApplicationId(), 1);
         app_0.getApplicationId(), 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     Container container = TestUtils.getMockContainer(containerId,
     Container container = TestUtils.getMockContainer(containerId,
-        node_1.getNodeID(), Resources.createResource(2*GB), priorityMap);
+        node_1.getNodeID(), Resources.createResource(2*GB),
+        priorityMap.getPriority());
     RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
     RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
         node_1.getNodeID(), "user", rmContext);
         node_1.getNodeID(), "user", rmContext);
 
 

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

@@ -51,6 +51,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
@@ -400,4 +402,14 @@ public class TestUtils {
 
 
     return conf;
     return conf;
   }
   }
+
+  public static SchedulerRequestKey toSchedulerKey(Priority pri) {
+    return SchedulerRequestKey.create(
+        ResourceRequest.newInstance(pri, null, null, 0));
+  }
+
+  public static SchedulerRequestKey toSchedulerKey(int pri) {
+    return SchedulerRequestKey.create(ResourceRequest.newInstance(
+        Priority.newInstance(pri), null, null, 0));
+  }
 }
 }

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

@@ -38,7 +38,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .TestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
@@ -63,8 +66,9 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
   @Test
   @Test
   public void testDelayScheduling() {
   public void testDelayScheduling() {
     FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
     FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
-    Priority prio = Mockito.mock(Priority.class);
-    Mockito.when(prio.getPriority()).thenReturn(1);
+    Priority pri = Mockito.mock(Priority.class);
+    SchedulerRequestKey prio = TestUtils.toSchedulerKey(pri);
+    Mockito.when(pri.getPriority()).thenReturn(1);
     double nodeLocalityThreshold = .5;
     double nodeLocalityThreshold = .5;
     double rackLocalityThreshold = .6;
     double rackLocalityThreshold = .6;
 
 
@@ -122,8 +126,9 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
   public void testDelaySchedulingForContinuousScheduling()
   public void testDelaySchedulingForContinuousScheduling()
           throws InterruptedException {
           throws InterruptedException {
     FSLeafQueue queue = scheduler.getQueueManager().getLeafQueue("queue", true);
     FSLeafQueue queue = scheduler.getQueueManager().getLeafQueue("queue", true);
-    Priority prio = Mockito.mock(Priority.class);
-    Mockito.when(prio.getPriority()).thenReturn(1);
+    Priority pri = Mockito.mock(Priority.class);
+    SchedulerRequestKey prio = TestUtils.toSchedulerKey(pri);
+    Mockito.when(pri.getPriority()).thenReturn(1);
 
 
     ControlledClock clock = new ControlledClock();
     ControlledClock clock = new ControlledClock();
     scheduler.setClock(clock);
     scheduler.setClock(clock);
@@ -180,8 +185,9 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
    */
    */
   public void testLocalityLevelWithoutDelays() {
   public void testLocalityLevelWithoutDelays() {
     FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
     FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
-    Priority prio = Mockito.mock(Priority.class);
-    Mockito.when(prio.getPriority()).thenReturn(1);
+    Priority pri = Mockito.mock(Priority.class);
+    SchedulerRequestKey prio = TestUtils.toSchedulerKey(pri);
+    Mockito.when(pri.getPriority()).thenReturn(1);
 
 
     RMContext rmContext = resourceManager.getRMContext();
     RMContext rmContext = resourceManager.getRMContext();
     ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1);
     ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1);

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

@@ -92,8 +92,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdate
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .TestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
@@ -2316,7 +2321,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     assertEquals(1, app.getLiveContainers().size());
     assertEquals(1, app.getLiveContainers().size());
     // Reserved container should still be at lower priority
     // Reserved container should still be at lower priority
     for (RMContainer container : app.getReservedContainers()) {
     for (RMContainer container : app.getReservedContainers()) {
-      assertEquals(2, container.getReservedPriority().getPriority());
+      assertEquals(2,
+          container.getReservedSchedulerKey().getPriority().getPriority());
     }
     }
     
     
     // Complete container
     // Complete container
@@ -2817,7 +2823,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.handle(node1UpdateEvent);
     scheduler.handle(node1UpdateEvent);
     assertEquals(1, app.getLiveContainers().size());
     assertEquals(1, app.getLiveContainers().size());
   }
   }
-  
+
   @Test
   @Test
   public void testCancelStrictLocality() throws IOException {
   public void testCancelStrictLocality() throws IOException {
     scheduler.init(conf);
     scheduler.init(conf);
@@ -4485,9 +4491,10 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     // time
     // time
     clock.tickSec(DELAY_THRESHOLD_TIME_MS / 1000);
     clock.tickSec(DELAY_THRESHOLD_TIME_MS / 1000);
     scheduler.attemptScheduling(node);
     scheduler.attemptScheduling(node);
-    Map<Priority, Long> lastScheduledContainer =
+    Map<SchedulerRequestKey, Long> lastScheduledContainer =
         fsAppAttempt.getLastScheduledContainer();
         fsAppAttempt.getLastScheduledContainer();
-    long initSchedulerTime = lastScheduledContainer.get(priority);
+    long initSchedulerTime =
+        lastScheduledContainer.get(TestUtils.toSchedulerKey(priority));
     assertEquals(DELAY_THRESHOLD_TIME_MS, initSchedulerTime);
     assertEquals(DELAY_THRESHOLD_TIME_MS, initSchedulerTime);
   }
   }
 
 

+ 15 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java

@@ -22,7 +22,6 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -34,6 +33,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .TestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
@@ -375,13 +378,15 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     Set<RMContainer> set = new HashSet<RMContainer>();
     Set<RMContainer> set = new HashSet<RMContainer>();
     for (RMContainer container :
     for (RMContainer container :
             scheduler.getSchedulerApp(app2).getLiveContainers()) {
             scheduler.getSchedulerApp(app2).getLiveContainers()) {
-      if (container.getAllocatedPriority().getPriority() == 4) {
+      if (container.getAllocatedSchedulerKey().getPriority().getPriority() ==
+          4) {
         set.add(container);
         set.add(container);
       }
       }
     }
     }
     for (RMContainer container :
     for (RMContainer container :
             scheduler.getSchedulerApp(app4).getLiveContainers()) {
             scheduler.getSchedulerApp(app4).getLiveContainers()) {
-      if (container.getAllocatedPriority().getPriority() == 4) {
+      if (container.getAllocatedSchedulerKey().getPriority().getPriority() ==
+          4) {
         set.add(container);
         set.add(container);
       }
       }
     }
     }
@@ -1399,7 +1404,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     scheduler.start();
     scheduler.start();
     scheduler.reinitialize(conf, resourceManager.getRMContext());
     scheduler.reinitialize(conf, resourceManager.getRMContext());
 
 
-    Priority priority = Priority.newInstance(20);
+    SchedulerRequestKey schedulerKey = TestUtils.toSchedulerKey(20);
     String host = "127.0.0.1";
     String host = "127.0.0.1";
     int GB = 1024;
     int GB = 1024;
 
 
@@ -1412,11 +1417,12 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     // Create 3 container requests and place it in ask
     // Create 3 container requests and place it in ask
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     ResourceRequest nodeLocalRequest = createResourceRequest(GB, 1, host,
     ResourceRequest nodeLocalRequest = createResourceRequest(GB, 1, host,
-            priority.getPriority(), 1, true);
+            schedulerKey.getPriority().getPriority(), 1, true);
     ResourceRequest rackLocalRequest = createResourceRequest(GB, 1,
     ResourceRequest rackLocalRequest = createResourceRequest(GB, 1,
-            node.getRackName(), priority.getPriority(), 1, true);
+        node.getRackName(), schedulerKey.getPriority().getPriority(), 1,
+        true);
     ResourceRequest offRackRequest = createResourceRequest(GB, 1,
     ResourceRequest offRackRequest = createResourceRequest(GB, 1,
-            ResourceRequest.ANY, priority.getPriority(), 1, true);
+        ResourceRequest.ANY, schedulerKey.getPriority().getPriority(), 1, true);
     ask.add(nodeLocalRequest);
     ask.add(nodeLocalRequest);
     ask.add(rackLocalRequest);
     ask.add(rackLocalRequest);
     ask.add(offRackRequest);
     ask.add(offRackRequest);
@@ -1435,7 +1441,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     SchedulerApplicationAttempt app = scheduler.getSchedulerApp(appAttemptId);
     SchedulerApplicationAttempt app = scheduler.getSchedulerApp(appAttemptId);
 
 
     // ResourceRequest will be empty once NodeUpdate is completed
     // ResourceRequest will be empty once NodeUpdate is completed
-    Assert.assertNull(app.getResourceRequest(priority, host));
+    Assert.assertNull(app.getResourceRequest(schedulerKey, host));
 
 
     ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 1);
     ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 1);
     RMContainer rmContainer = app.getRMContainer(containerId1);
     RMContainer rmContainer = app.getRMContainer(containerId1);
@@ -1458,7 +1464,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     Assert.assertEquals(3, requests.size());
     Assert.assertEquals(3, requests.size());
     for (ResourceRequest request : requests) {
     for (ResourceRequest request : requests) {
       Assert.assertEquals(1,
       Assert.assertEquals(1,
-              app.getResourceRequest(priority, request.getResourceName())
+              app.getResourceRequest(schedulerKey, request.getResourceName())
                       .getNumContainers());
                       .getNumContainers());
     }
     }