diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java index a8c62fd0577..9df395dd906 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java @@ -23,7 +23,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; @@ -343,12 +342,10 @@ public class FifoCandidatesSelector Collections.sort(containers, new Comparator() { @Override public int compare(RMContainer a, RMContainer b) { - Comparator 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()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java index 5060c4ce607..f0988063a84 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Priority.java +++ b/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); return priority; } - - public static class Comparator - implements java.util.Comparator { - @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(); - } - } - + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java index 504c9738bfb..e5d120846b0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java @@ -31,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.ResourceRequest; 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 @@ -55,7 +56,7 @@ public interface RMContainer extends EventHandler { NodeId getReservedNode(); - Priority getReservedPriority(); + SchedulerRequestKey getReservedSchedulerKey(); Resource getAllocatedResource(); @@ -63,6 +64,8 @@ public interface RMContainer extends EventHandler { NodeId getAllocatedNode(); + SchedulerRequestKey getAllocatedSchedulerKey(); + Priority getAllocatedPriority(); long getCreationTime(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java index ed819a093bf..706821ee565 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java @@ -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 .RMNodeDecreaseContainerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.state.InvalidStateTransitionException; import org.apache.hadoop.yarn.state.MultipleArcTransition; import org.apache.hadoop.yarn.state.SingleArcTransition; import org.apache.hadoop.yarn.state.StateMachine; 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.webapp.util.WebAppUtils; @@ -173,7 +173,7 @@ public class RMContainerImpl implements RMContainer, Comparable { private Resource reservedResource; private NodeId reservedNode; - private Priority reservedPriority; + private SchedulerRequestKey reservedSchedulerKey; private long creationTime; private long finishTime; private ContainerStatus finishedStatus; @@ -187,6 +187,7 @@ public class RMContainerImpl implements RMContainer, Comparable { private volatile String queueName; private boolean isExternallyAllocated; + private SchedulerRequestKey allocatedSchedulerKey; public RMContainerImpl(Container container, ApplicationAttemptId appAttemptId, NodeId nodeId, String user, @@ -226,6 +227,7 @@ public class RMContainerImpl implements RMContainer, Comparable { this.containerId = container.getId(); this.nodeId = nodeId; this.container = container; + this.allocatedSchedulerKey = SchedulerRequestKey.extractFrom(container); this.appAttemptId = appAttemptId; this.user = user; this.creationTime = creationTime; @@ -296,8 +298,8 @@ public class RMContainerImpl implements RMContainer, Comparable { } @Override - public Priority getReservedPriority() { - return reservedPriority; + public SchedulerRequestKey getReservedSchedulerKey() { + return reservedSchedulerKey; } @Override @@ -325,6 +327,11 @@ public class RMContainerImpl implements RMContainer, Comparable { return container.getNodeId(); } + @Override + public SchedulerRequestKey getAllocatedSchedulerKey() { + return allocatedSchedulerKey; + } + @Override public Priority getAllocatedPriority() { return container.getPriority(); @@ -535,7 +542,7 @@ public class RMContainerImpl implements RMContainer, Comparable { RMContainerReservedEvent e = (RMContainerReservedEvent)event; container.reservedResource = e.getReservedResource(); container.reservedNode = e.getReservedNode(); - container.reservedPriority = e.getReservedPriority(); + container.reservedSchedulerKey = e.getReservedSchedulerKey(); if (!EnumSet.of(RMContainerState.NEW, RMContainerState.RESERVED) .contains(container.getState())) { @@ -768,7 +775,7 @@ public class RMContainerImpl implements RMContainer, Comparable { try { containerReport = ContainerReport.newInstance(this.getContainerId(), this.getAllocatedResource(), this.getAllocatedNode(), - this.getAllocatedPriority(), this.getCreationTime(), + this.getAllocatedSchedulerKey().getPriority(), this.getCreationTime(), this.getFinishTime(), this.getDiagnosticsInfo(), this.getLogURL(), this.getContainerExitStatus(), this.getContainerState(), this.getNodeHttpAddress()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerReservedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerReservedEvent.java index 74e2dc4c939..d7d1e9463a4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerReservedEvent.java +++ b/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.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; 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. @@ -33,15 +33,15 @@ public class RMContainerReservedEvent extends RMContainerEvent { private final Resource reservedResource; private final NodeId reservedNode; - private final Priority reservedPriority; + private final SchedulerRequestKey reservedSchedulerKey; public RMContainerReservedEvent(ContainerId containerId, Resource reservedResource, NodeId reservedNode, - Priority reservedPriority) { + SchedulerRequestKey reservedSchedulerKey) { super(containerId, RMContainerEventType.RESERVED); this.reservedResource = reservedResource; this.reservedNode = reservedNode; - this.reservedPriority = reservedPriority; + this.reservedSchedulerKey = reservedSchedulerKey; } public Resource getReservedResource() { @@ -52,8 +52,8 @@ public class RMContainerReservedEvent extends RMContainerEvent { return reservedNode; } - public Priority getReservedPriority() { - return reservedPriority; + public SchedulerRequestKey getReservedSchedulerKey() { + return reservedSchedulerKey; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java index 8d42c977b8b..376466430af 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java @@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler; import java.util.ArrayList; import java.util.Collection; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; 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.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -60,8 +58,6 @@ import org.apache.hadoop.yarn.util.resource.Resources; public class AppSchedulingInfo { private static final Log LOG = LogFactory.getLog(AppSchedulingInfo.class); - private static final Comparator COMPARATOR = - new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator(); private static final int EPOCH_BIT_SHIFT = 40; private final ApplicationId applicationId; @@ -82,10 +78,10 @@ public class AppSchedulingInfo { private Set requestedPartitions = new HashSet<>(); - final Set priorities = new TreeSet<>(COMPARATOR); - final Map> resourceRequestMap = - new ConcurrentHashMap<>(); - final Map schedulerKeys = new TreeSet<>(); + final Map> + resourceRequestMap = new ConcurrentHashMap<>(); + final Map>> containerIncreaseRequestMap = new ConcurrentHashMap<>(); @@ -134,22 +130,23 @@ public class AppSchedulingInfo { * Clear any pending requests from this application. */ private synchronized void clearRequests() { - priorities.clear(); + schedulerKeys.clear(); resourceRequestMap.clear(); LOG.info("Application " + applicationId + " requests cleared"); } public synchronized boolean hasIncreaseRequest(NodeId nodeId) { - Map> requestsOnNode = - containerIncreaseRequestMap.get(nodeId); + Map> + requestsOnNode = containerIncreaseRequestMap.get(nodeId); return requestsOnNode == null ? false : requestsOnNode.size() > 0; } - + public synchronized Map - getIncreaseRequests(NodeId nodeId, Priority priority) { - Map> requestsOnNode = - containerIncreaseRequestMap.get(nodeId); - return requestsOnNode == null ? null : requestsOnNode.get(priority); + getIncreaseRequests(NodeId nodeId, SchedulerRequestKey schedulerKey) { + Map> + requestsOnNode = containerIncreaseRequestMap.get(nodeId); + return requestsOnNode == null ? null : requestsOnNode.get( + schedulerKey); } /** @@ -175,15 +172,17 @@ public class AppSchedulingInfo { } NodeId nodeId = r.getRMContainer().getAllocatedNode(); - Map> requestsOnNode = - containerIncreaseRequestMap.get(nodeId); + Map> + requestsOnNode = containerIncreaseRequestMap.get(nodeId); if (null == requestsOnNode) { requestsOnNode = new TreeMap<>(); containerIncreaseRequestMap.put(nodeId, requestsOnNode); } SchedContainerChangeRequest prevChangeRequest = - getIncreaseRequest(nodeId, r.getPriority(), r.getContainerId()); + getIncreaseRequest(nodeId, + r.getRMContainer().getAllocatedSchedulerKey(), + r.getContainerId()); if (null != prevChangeRequest) { if (Resources.equals(prevChangeRequest.getTargetCapacity(), r.getTargetCapacity())) { @@ -192,7 +191,8 @@ public class AppSchedulingInfo { } // remove the old one, as we will use the new one going forward - removeIncreaseRequest(nodeId, prevChangeRequest.getPriority(), + removeIncreaseRequest(nodeId, + prevChangeRequest.getRMContainer().getAllocatedSchedulerKey(), prevChangeRequest.getContainerId()); } @@ -219,21 +219,22 @@ public class AppSchedulingInfo { */ private void insertIncreaseRequest(SchedContainerChangeRequest request) { NodeId nodeId = request.getNodeId(); - Priority priority = request.getPriority(); + SchedulerRequestKey schedulerKey = + request.getRMContainer().getAllocatedSchedulerKey(); ContainerId containerId = request.getContainerId(); - Map> requestsOnNode = - containerIncreaseRequestMap.get(nodeId); + Map> + requestsOnNode = containerIncreaseRequestMap.get(nodeId); if (null == requestsOnNode) { requestsOnNode = new HashMap<>(); containerIncreaseRequestMap.put(nodeId, requestsOnNode); } Map requestsOnNodeWithPriority = - requestsOnNode.get(priority); + requestsOnNode.get(schedulerKey); if (null == requestsOnNodeWithPriority) { requestsOnNodeWithPriority = new TreeMap<>(); - requestsOnNode.put(priority, requestsOnNodeWithPriority); + requestsOnNode.put(schedulerKey, requestsOnNodeWithPriority); } requestsOnNodeWithPriority.put(containerId, request); @@ -249,20 +250,20 @@ public class AppSchedulingInfo { + " delta=" + delta); } - // update priorities - priorities.add(priority); + // update Scheduler Keys + schedulerKeys.add(schedulerKey); } - public synchronized boolean removeIncreaseRequest(NodeId nodeId, Priority priority, - ContainerId containerId) { - Map> requestsOnNode = - containerIncreaseRequestMap.get(nodeId); + public synchronized boolean removeIncreaseRequest(NodeId nodeId, + SchedulerRequestKey schedulerKey, ContainerId containerId) { + Map> + requestsOnNode = containerIncreaseRequestMap.get(nodeId); if (null == requestsOnNode) { return false; } Map requestsOnNodeWithPriority = - requestsOnNode.get(priority); + requestsOnNode.get(schedulerKey); if (null == requestsOnNodeWithPriority) { return false; } @@ -272,7 +273,7 @@ public class AppSchedulingInfo { // remove hierarchies if it becomes empty if (requestsOnNodeWithPriority.isEmpty()) { - requestsOnNode.remove(priority); + requestsOnNode.remove(schedulerKey); } if (requestsOnNode.isEmpty()) { containerIncreaseRequestMap.remove(nodeId); @@ -296,15 +297,15 @@ public class AppSchedulingInfo { } public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId, - Priority priority, ContainerId containerId) { - Map> requestsOnNode = - containerIncreaseRequestMap.get(nodeId); + SchedulerRequestKey schedulerKey, ContainerId containerId) { + Map> + requestsOnNode = containerIncreaseRequestMap.get(nodeId); if (null == requestsOnNode) { return null; } Map requestsOnNodeWithPriority = - requestsOnNode.get(priority); + requestsOnNode.get(schedulerKey); return requestsOnNodeWithPriority == null ? null : requestsOnNodeWithPriority.get(containerId); } @@ -328,17 +329,18 @@ public class AppSchedulingInfo { // Update resource requests for (ResourceRequest request : requests) { - Priority priority = request.getPriority(); + SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request); String resourceName = request.getResourceName(); // Update node labels if required updateNodeLabels(request); - Map asks = this.resourceRequestMap.get(priority); + Map asks = + this.resourceRequestMap.get(schedulerKey); if (asks == null) { 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 @@ -405,11 +407,11 @@ public class AppSchedulingInfo { } private void updateNodeLabels(ResourceRequest request) { - Priority priority = request.getPriority(); + SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request); String resourceName = request.getResourceName(); if (resourceName.equals(ResourceRequest.ANY)) { ResourceRequest previousAnyRequest = - getResourceRequest(priority, resourceName); + getResourceRequest(schedulerKey, resourceName); // When there is change in ANY request label expression, we should // update label for all resource requests already added of same @@ -417,7 +419,7 @@ public class AppSchedulingInfo { if ((null == previousAnyRequest) || hasRequestLabelChanged(previousAnyRequest, request)) { Map resourceRequest = - getResourceRequests(priority); + getResourceRequests(schedulerKey); if (resourceRequest != null) { for (ResourceRequest r : resourceRequest.values()) { if (!r.getResourceName().equals(ResourceRequest.ANY)) { @@ -428,7 +430,7 @@ public class AppSchedulingInfo { } } else { ResourceRequest anyRequest = - getResourceRequest(priority, ResourceRequest.ANY); + getResourceRequest(schedulerKey, ResourceRequest.ANY); if (anyRequest != null) { request.setNodeLabelExpression(anyRequest.getNodeLabelExpression()); } @@ -501,13 +503,13 @@ public class AppSchedulingInfo { return userBlacklistChanged.getAndSet(false); } - public synchronized Collection getPriorities() { - return priorities; + public synchronized Collection getSchedulerKeys() { + return schedulerKeys; } public synchronized Map getResourceRequests( - Priority priority) { - return resourceRequestMap.get(priority); + SchedulerRequestKey schedulerKey) { + return resourceRequestMap.get(schedulerKey); } public synchronized List getAllResourceRequests() { @@ -518,14 +520,16 @@ public class AppSchedulingInfo { return ret; } - public synchronized ResourceRequest getResourceRequest(Priority priority, - String resourceName) { - Map nodeRequests = resourceRequestMap.get(priority); + public synchronized ResourceRequest getResourceRequest( + SchedulerRequestKey schedulerKey, String resourceName) { + Map nodeRequests = + resourceRequestMap.get(schedulerKey); 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(); } @@ -555,7 +559,8 @@ public class AppSchedulingInfo { public synchronized void increaseContainer( SchedContainerChangeRequest increaseRequest) { NodeId nodeId = increaseRequest.getNodeId(); - Priority priority = increaseRequest.getPriority(); + SchedulerRequestKey schedulerKey = + increaseRequest.getRMContainer().getAllocatedSchedulerKey(); ContainerId containerId = increaseRequest.getContainerId(); Resource deltaCapacity = increaseRequest.getDeltaCapacity(); @@ -568,7 +573,7 @@ public class AppSchedulingInfo { // Set queue metrics queue.getMetrics().allocateResources(user, deltaCapacity); // remove the increase request from pending increase request map - removeIncreaseRequest(nodeId, priority, containerId); + removeIncreaseRequest(nodeId, schedulerKey, containerId); // update usage appResourceUsage.incUsed(increaseRequest.getNodePartition(), deltaCapacity); } @@ -591,19 +596,25 @@ public class AppSchedulingInfo { // update usage appResourceUsage.decUsed(decreaseRequest.getNodePartition(), absDelta); } - + /** * Resources have been allocated to this application by the resource * 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 allocate(NodeType type, - SchedulerNode node, Priority priority, ResourceRequest request, - Container containerAllocated) { + SchedulerNode node, SchedulerRequestKey schedulerKey, + ResourceRequest request, Container containerAllocated) { List resourceRequests = new ArrayList<>(); if (type == NodeType.NODE_LOCAL) { - allocateNodeLocal(node, priority, request, resourceRequests); + allocateNodeLocal(node, schedulerKey, request, resourceRequests); } else if (type == NodeType.RACK_LOCAL) { - allocateRackLocal(node, priority, request, resourceRequests); + allocateRackLocal(node, schedulerKey, request, resourceRequests); } else { allocateOffSwitch(request, resourceRequests); } @@ -633,16 +644,16 @@ public class AppSchedulingInfo { * application. */ private synchronized void allocateNodeLocal(SchedulerNode node, - Priority priority, ResourceRequest nodeLocalRequest, + SchedulerRequestKey schedulerKey, ResourceRequest nodeLocalRequest, List resourceRequests) { // 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()); - decResourceRequest(node.getRackName(), priority, rackLocalRequest); + decResourceRequest(node.getRackName(), schedulerKey, rackLocalRequest); - ResourceRequest offRackRequest = resourceRequestMap.get(priority).get( + ResourceRequest offRackRequest = resourceRequestMap.get(schedulerKey).get( ResourceRequest.ANY); decrementOutstanding(offRackRequest); @@ -652,11 +663,11 @@ public class AppSchedulingInfo { 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); if (request.getNumContainers() == 0) { - resourceRequestMap.get(priority).remove(resourceName); + resourceRequestMap.get(schedulerKey).remove(resourceName); } } @@ -665,12 +676,12 @@ public class AppSchedulingInfo { * application. */ private synchronized void allocateRackLocal(SchedulerNode node, - Priority priority, ResourceRequest rackLocalRequest, + SchedulerRequestKey schedulerKey, ResourceRequest rackLocalRequest, List resourceRequests) { // 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); decrementOutstanding(offRackRequest); @@ -712,8 +723,9 @@ public class AppSchedulingInfo { private synchronized void checkForDeactivation() { 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.getNumContainers() > 0) { deactivate = false; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java index b4a2639a1bf..c4b32a85992 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java @@ -98,10 +98,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { protected ApplicationAttemptId attemptId; protected Map liveContainers = new HashMap(); - protected final Map> reservedContainers = - new HashMap>(); + protected final Map> + reservedContainers = new HashMap<>(); - private final Multiset reReservations = HashMultiset.create(); + private final Multiset reReservations = + HashMultiset.create(); private Resource resourceLimit = Resource.newInstance(0, 0); 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 * is reset to 0. */ - Multiset schedulingOpportunities = HashMultiset.create(); + Multiset schedulingOpportunities = HashMultiset.create(); /** * 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, * it is reset to 0 when schedule any task at corresponding priority. */ - Multiset missedNonPartitionedRequestSchedulingOpportunity = + Multiset missedNonPartitionedReqSchedulingOpportunity = HashMultiset.create(); // Time of the last container scheduled at the current allowed level - protected Map lastScheduledContainer = - new HashMap(); + protected Map lastScheduledContainer = + new HashMap<>(); protected Queue queue; protected boolean isStopped = false; @@ -225,8 +226,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { return appSchedulingInfo.getUser(); } - public Map getResourceRequests(Priority priority) { - return appSchedulingInfo.getResourceRequests(priority); + public Map getResourceRequests( + SchedulerRequestKey schedulerKey) { + return appSchedulingInfo.getResourceRequests(schedulerKey); } public Set getPendingRelease() { @@ -237,22 +239,24 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { return appSchedulingInfo.getNewContainerId(); } - public Collection getPriorities() { - return appSchedulingInfo.getPriorities(); + public Collection 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(); } - public synchronized Resource getResource(Priority priority) { - return appSchedulingInfo.getResource(priority); + public synchronized Resource getResource(SchedulerRequestKey schedulerKey) { + return appSchedulingInfo.getResource(schedulerKey); } 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 getReservedContainers() { List reservedContainers = new ArrayList(); - for (Map.Entry> e : + for (Map.Entry> e : this.reservedContainers.entrySet()) { reservedContainers.addAll(e.getValue().values()); } @@ -374,8 +380,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { } 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(), reservedResource); // succeeded @@ -386,10 +393,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { } private synchronized boolean commonReserve(SchedulerNode node, - Priority priority, RMContainer rmContainer, Resource reservedResource) { + SchedulerRequestKey schedulerKey, RMContainer rmContainer, + Resource reservedResource) { try { rmContainer.handle(new RMContainerReservedEvent(rmContainer - .getContainerId(), reservedResource, node.getNodeID(), priority)); + .getContainerId(), reservedResource, node.getNodeID(), schedulerKey)); } catch (InvalidStateTransitionException e) { // We reach here could be caused by container already finished, return // false indicate it fails @@ -397,10 +405,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { } Map reservedContainers = - this.reservedContainers.get(priority); + this.reservedContainers.get(schedulerKey); if (reservedContainers == null) { reservedContainers = new HashMap(); - this.reservedContainers.put(priority, reservedContainers); + this.reservedContainers.put(schedulerKey, reservedContainers); } reservedContainers.put(node.getNodeID(), rmContainer); @@ -408,7 +416,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { LOG.debug("Application attempt " + getApplicationAttemptId() + " reserved container " + rmContainer + " on node " + node + ". This attempt currently has " + reservedContainers.size() - + " reserved containers at priority " + priority + + " reserved containers at priority " + schedulerKey.getPriority() + "; currentReservation " + reservedResource); } @@ -416,7 +424,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { } public synchronized RMContainer reserve(SchedulerNode node, - Priority priority, RMContainer rmContainer, Container container) { + SchedulerRequestKey schedulerKey, RMContainer rmContainer, + Container container) { // Create RMContainer if necessary if (rmContainer == null) { rmContainer = @@ -427,13 +436,13 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { ((RMContainerImpl)rmContainer).setQueueName(this.getQueueName()); // Reset the re-reservation count - resetReReservations(priority); + resetReReservations(schedulerKey); } else { // Note down the re-reservation - addReReservation(priority); + addReReservation(schedulerKey); } - commonReserve(node, priority, rmContainer, container.getResource()); + commonReserve(node, schedulerKey, rmContainer, container.getResource()); return rmContainer; } @@ -442,12 +451,13 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { * Has the application reserved the given node at the * given priority? * @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 */ - public synchronized boolean isReserved(SchedulerNode node, Priority priority) { + public synchronized boolean isReserved(SchedulerNode node, + SchedulerRequestKey schedulerKey) { Map reservedContainers = - this.reservedContainers.get(priority); + this.reservedContainers.get(schedulerKey); if (reservedContainers != null) { return reservedContainers.containsKey(node.getNodeID()); } @@ -471,9 +481,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { return resourceLimit; } - public synchronized int getNumReservedContainers(Priority priority) { + public synchronized int getNumReservedContainers( + SchedulerRequestKey schedulerKey) { Map reservedContainers = - this.reservedContainers.get(priority); + this.reservedContainers.get(schedulerKey); return (reservedContainers == null) ? 0 : reservedContainers.size(); } @@ -495,8 +506,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { public synchronized void showRequests() { if (LOG.isDebugEnabled()) { - for (Priority priority : getPriorities()) { - Map requests = getResourceRequests(priority); + for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) { + Map requests = + getResourceRequests(schedulerKey); if (requests != null) { LOG.debug("showRequests:" + " application=" + getApplicationId() + " headRoom=" + getHeadroom() + " currentConsumption=" @@ -635,59 +647,66 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { } 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 - 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) { - 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 * to schedule a task at the given priority since the last time it * 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 * - * @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 - 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 - void setSchedulingOpportunities(Priority priority, int count) { - schedulingOpportunities.setCount(priority, count); + void setSchedulingOpportunities(SchedulerRequestKey schedulerKey, int count) { + schedulingOpportunities.setCount(schedulerKey, count); } synchronized AggregateAppResourceUsage getRunningAggregateAppResourceUsage() { @@ -747,7 +766,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { return this.resourceLimit; } - public synchronized Map getLastScheduledContainer() { + public synchronized Map + getLastScheduledContainer() { return this.lastScheduledContainer; } @@ -892,8 +912,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { } 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); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java index 1f57e079d58..2efdbd02dcf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java @@ -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.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceUtilization; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -308,11 +307,11 @@ public abstract class SchedulerNode { /** * Reserve container for the attempt on this node. * @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. */ public abstract void reserveResource(SchedulerApplicationAttempt attempt, - Priority priority, RMContainer container); + SchedulerRequestKey schedulerKey, RMContainer container); /** * Unreserve resources on this node. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerRequestKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerRequestKey.java new file mode 100644 index 00000000000..b4988be9a80 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/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 { + + 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(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index 6dcafecb184..9aae9095c15 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -1265,7 +1265,8 @@ public class LeafQueue extends AbstractCSQueue { } if (null != priority) { - removed = app.unreserve(rmContainer.getContainer().getPriority(), node, + removed = app.unreserve( + rmContainer.getAllocatedSchedulerKey(), node, rmContainer); } @@ -1321,7 +1322,7 @@ public class LeafQueue extends AbstractCSQueue { // Remove container increase request if it exists application.removeIncreaseRequest(node.getNodeID(), - rmContainer.getAllocatedPriority(), rmContainer.getContainerId()); + rmContainer.getAllocatedSchedulerKey(), rmContainer.getContainerId()); boolean removed = false; @@ -1335,7 +1336,7 @@ public class LeafQueue extends AbstractCSQueue { // happen under scheduler's lock... // So, this is, in effect, a transaction across application & node if (rmContainer.getState() == RMContainerState.RESERVED) { - removed = application.unreserve(rmContainer.getReservedPriority(), + removed = application.unreserve(rmContainer.getReservedSchedulerKey(), node, rmContainer); } else { removed = @@ -1785,7 +1786,8 @@ public class LeafQueue extends AbstractCSQueue { // Do we have increase request for the same container? If so, remove it boolean hasIncreaseRequest = app.removeIncreaseRequest(decreaseRequest.getNodeId(), - decreaseRequest.getPriority(), decreaseRequest.getContainerId()); + decreaseRequest.getRMContainer().getAllocatedSchedulerKey(), + decreaseRequest.getContainerId()); if (hasIncreaseRequest) { if (LOG.isDebugEnabled()) { LOG.debug("While processing decrease requests, found an increase" diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java index 25e58241b85..4a2ae1839fa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/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.ContainerState; 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.server.resourcemanager.RMContext; 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.ResourceLimits; 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.capacity.CSAssignment; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; @@ -115,7 +116,8 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { node.getUnallocatedResource())) { // OK, we can allocate this increase request // Unreserve it first - application.unreserve(increaseRequest.getPriority(), + application.unreserve( + increaseRequest.getRMContainer().getAllocatedSchedulerKey(), (FiCaSchedulerNode) node, increaseRequest.getRMContainer()); // Notify application @@ -152,7 +154,8 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { return createSuccessfullyIncreasedCSAssignment(increaseRequest, false); } else { boolean reservationSucceeded = - application.reserveIncreasedContainer(increaseRequest.getPriority(), + application.reserveIncreasedContainer( + increaseRequest.getRMContainer().getAllocatedSchedulerKey(), node, increaseRequest.getRMContainer(), increaseRequest.getDeltaCapacity()); @@ -228,11 +231,11 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { * priority, but will skip increase request and move to next increase * request if queue-limit or user-limit aren't satisfied */ - for (Priority priority : application.getPriorities()) { + for (SchedulerRequestKey schedulerKey : application.getSchedulerKeys()) { if (LOG.isDebugEnabled()) { LOG.debug("Looking at increase request for application=" + application.getApplicationAttemptId() + " priority=" - + priority); + + schedulerKey.getPriority()); } /* @@ -242,14 +245,14 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { * cannot be allocated. */ Map increaseRequestMap = - sinfo.getIncreaseRequests(nodeId, priority); + sinfo.getIncreaseRequests(nodeId, schedulerKey); // We don't have more increase request on this priority, skip.. if (null == increaseRequestMap) { if (LOG.isDebugEnabled()) { LOG.debug("There's no increase request for " + application.getApplicationAttemptId() + " priority=" - + priority); + + schedulerKey.getPriority()); } continue; } @@ -318,7 +321,8 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { // Remove invalid in request requests if (!toBeRemovedRequests.isEmpty()) { for (SchedContainerChangeRequest req : toBeRemovedRequests) { - sinfo.removeIncreaseRequest(req.getNodeId(), req.getPriority(), + sinfo.removeIncreaseRequest(req.getNodeId(), + req.getRMContainer().getAllocatedSchedulerKey(), req.getContainerId()); } } @@ -337,8 +341,9 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { // We already reserved this increase container 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 // - Container finished diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java index aae5292795d..4bae5bedd66 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java @@ -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.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; 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.ResourceLimits; 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.capacity.CSAMContainerLaunchDiagnosticsConstants; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment; @@ -80,7 +83,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { private ContainerAllocation preCheckForNewContainer(Resource clusterResource, FiCaSchedulerNode node, SchedulingMode schedulingMode, - ResourceLimits resourceLimits, Priority priority) { + ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey) { if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) { application.updateAppSkipNodeDiagnostics( CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE); @@ -88,7 +91,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } ResourceRequest anyRequest = - application.getResourceRequest(priority, ResourceRequest.ANY); + application.getResourceRequest(schedulerKey, ResourceRequest.ANY); if (null == anyRequest) { return ContainerAllocation.PRIORITY_SKIPPED; } @@ -97,7 +100,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { Resource required = anyRequest.getCapability(); // Do we need containers at this 'priority'? - if (application.getTotalRequiredResources(priority) <= 0) { + if (application.getTotalRequiredResources(schedulerKey) <= 0) { return ContainerAllocation.PRIORITY_SKIPPED; } @@ -126,7 +129,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } if (!application.getCSLeafQueue().getReservationContinueLooking()) { - if (!shouldAllocOrReserveNewContainer(priority, required)) { + if (!shouldAllocOrReserveNewContainer(schedulerKey, required)) { if (LOG.isDebugEnabled()) { 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 - application.addSchedulingOpportunity(priority); + application.addSchedulingOpportunity(schedulerKey); // Increase missed-non-partitioned-resource-request-opportunity. // This is to make sure non-partitioned-resource-request will prefer @@ -152,8 +155,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { if (anyRequest.getNodeLabelExpression() .equals(RMNodeLabelsManager.NO_LABEL)) { missedNonPartitionedRequestSchedulingOpportunity = - application - .addMissedNonPartitionedRequestSchedulingOpportunity(priority); + application.addMissedNonPartitionedRequestSchedulingOpportunity( + schedulerKey); } if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) { @@ -164,7 +167,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { .getScheduler().getNumClusterNodes()) { if (LOG.isDebugEnabled()) { LOG.debug("Skip app_attempt=" + application.getApplicationAttemptId() - + " priority=" + priority + + " priority=" + schedulerKey.getPriority() + " because missed-non-partitioned-resource-request" + " opportunity under requred:" + " Now=" + missedNonPartitionedRequestSchedulingOpportunity + " required=" @@ -180,20 +183,20 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { ContainerAllocation preAllocation(Resource clusterResource, FiCaSchedulerNode node, SchedulingMode schedulingMode, - ResourceLimits resourceLimits, Priority priority, + ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey, RMContainer reservedContainer) { ContainerAllocation result; if (null == reservedContainer) { // pre-check when allocating new container result = preCheckForNewContainer(clusterResource, node, schedulingMode, - resourceLimits, priority); + resourceLimits, schedulerKey); if (null != result) { return result; } } else { // pre-check when allocating reserved container - if (application.getTotalRequiredResources(priority) == 0) { + if (application.getTotalRequiredResources(schedulerKey) == 0) { // Release return new ContainerAllocation(reservedContainer, null, AllocationState.QUEUE_SKIPPED); @@ -202,13 +205,13 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // Try to allocate containers on node result = - assignContainersOnNode(clusterResource, node, priority, + assignContainersOnNode(clusterResource, node, schedulerKey, reservedContainer, schedulingMode, resourceLimits); if (null == reservedContainer) { if (result.state == AllocationState.PRIORITY_SKIPPED) { // 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( - Priority priority, int clusterNodes) { + SchedulerRequestKey schedulerKey, int clusterNodes) { // Estimate: Required unique resources (i.e. hosts + racks) 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' // i.e. no point skipping more than clustersize opportunities @@ -231,8 +234,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { .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... if (type == NodeType.OFF_SWITCH) { @@ -242,15 +245,16 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // 'Delay' off-switch 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(); float localityWaitFactor = - getLocalityWaitFactor(priority, rmContext.getScheduler() + getLocalityWaitFactor(schedulerKey, rmContext.getScheduler() .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. return (Math.min(rmContext.getScheduler().getNumClusterNodes(), (requiredContainers * localityWaitFactor)) < missedOpportunities); @@ -258,7 +262,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // Check if we need containers on this rack ResourceRequest rackLocalRequest = - application.getResourceRequest(priority, node.getRackName()); + application.getResourceRequest(schedulerKey, node.getRackName()); if (rackLocalRequest == null || rackLocalRequest.getNumContainers() <= 0) { 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 (type == NodeType.RACK_LOCAL) { // 'Delay' rack-local just a little bit... - long missedOpportunities = application.getSchedulingOpportunities(priority); + long missedOpportunities = + application.getSchedulingOpportunities(schedulerKey); return getActualNodeLocalityDelay() < missedOpportunities; } @@ -274,7 +279,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { if (type == NodeType.NODE_LOCAL) { // Now check if we need containers on this host... ResourceRequest nodeLocalRequest = - application.getResourceRequest(priority, node.getNodeName()); + application.getResourceRequest(schedulerKey, node.getNodeName()); if (nodeLocalRequest != null) { return nodeLocalRequest.getNumContainers() > 0; } @@ -285,10 +290,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { private ContainerAllocation assignNodeLocalContainers( 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, schedulingMode, currentResoureLimits); } @@ -299,10 +305,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { private ContainerAllocation assignRackLocalContainers( 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, schedulingMode, currentResoureLimits); } @@ -313,10 +320,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { private ContainerAllocation assignOffSwitchContainers( 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, schedulingMode, currentResoureLimits); } @@ -327,20 +335,21 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } 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; NodeType requestType = null; // Data-local ResourceRequest nodeLocalResourceRequest = - application.getResourceRequest(priority, node.getNodeName()); + application.getResourceRequest(schedulerKey, node.getNodeName()); if (nodeLocalResourceRequest != null) { requestType = NodeType.NODE_LOCAL; allocation = assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest, - node, priority, reservedContainer, schedulingMode, + node, schedulerKey, reservedContainer, schedulingMode, currentResoureLimits); if (Resources.greaterThan(rc, clusterResource, allocation.getResourceToBeAllocated(), Resources.none())) { @@ -351,7 +360,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // Rack-local ResourceRequest rackLocalResourceRequest = - application.getResourceRequest(priority, node.getRackName()); + application.getResourceRequest(schedulerKey, node.getRackName()); if (rackLocalResourceRequest != null) { if (!rackLocalResourceRequest.getRelaxLocality()) { return ContainerAllocation.PRIORITY_SKIPPED; @@ -363,7 +372,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { allocation = assignRackLocalContainers(clusterResource, rackLocalResourceRequest, - node, priority, reservedContainer, schedulingMode, + node, schedulerKey, reservedContainer, schedulingMode, currentResoureLimits); if (Resources.greaterThan(rc, clusterResource, allocation.getResourceToBeAllocated(), Resources.none())) { @@ -374,7 +383,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // Off-switch ResourceRequest offSwitchResourceRequest = - application.getResourceRequest(priority, ResourceRequest.ANY); + application.getResourceRequest(schedulerKey, ResourceRequest.ANY); if (offSwitchResourceRequest != null) { if (!offSwitchResourceRequest.getRelaxLocality()) { return ContainerAllocation.PRIORITY_SKIPPED; @@ -386,7 +395,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { allocation = assignOffSwitchContainers(clusterResource, offSwitchResourceRequest, - node, priority, reservedContainer, schedulingMode, + node, schedulerKey, reservedContainer, schedulingMode, currentResoureLimits); allocation.requestNodeType = requestType; @@ -403,21 +412,22 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } 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; if (LOG.isDebugEnabled()) { 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 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 // to label not match. This can be caused by node label changed // We should un-reserve this container. @@ -439,7 +449,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer( - priority, capability); + schedulerKey, capability); // Can we allocate a container on this node? long availableContainers = @@ -504,8 +514,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { resourceNeedToUnReserve = capability; } unreservedContainer = - application.findNodeToUnreserve(clusterResource, node, priority, - resourceNeedToUnReserve); + application.findNodeToUnreserve(clusterResource, node, + schedulerKey, resourceNeedToUnReserve); // When (minimum-unreserved-resource > 0 OR we cannot allocate // new/reserved // 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; 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... // Protect against corner case where you need the whole node with // Math.min(nodeFactor, minimumAllocationFactor) starvation = - (int) ((application.getReReservations(priority) / + (int) ((application.getReReservations(schedulerKey) / (float) reservedContainers) * (1.0f - (Math.min( nodeFactor, application.getCSLeafQueue() .getMinimumAllocationFactor())))); if (LOG.isDebugEnabled()) { LOG.debug("needsContainers:" + " app.#re-reserve=" - + application.getReReservations(priority) + " reserved=" + + application.getReReservations(schedulerKey) + " reserved=" + reservedContainers + " nodeFactor=" + nodeFactor + " minAllocFactor=" + application.getCSLeafQueue().getMinimumAllocationFactor() @@ -585,13 +595,14 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } private Container getContainer(RMContainer rmContainer, - FiCaSchedulerNode node, Resource capability, Priority priority) { + FiCaSchedulerNode node, Resource capability, + SchedulerRequestKey schedulerKey) { return (rmContainer != null) ? rmContainer.getContainer() - : createContainer(node, capability, priority); + : createContainer(node, capability, schedulerKey); } private Container createContainer(FiCaSchedulerNode node, Resource capability, - Priority priority) { + SchedulerRequestKey schedulerKey) { NodeId nodeId = node.getRMNode().getNodeID(); ContainerId containerId = @@ -600,22 +611,23 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // Create the container return BuilderUtils.newContainer(containerId, nodeId, node.getRMNode() - .getHttpAddress(), capability, priority, null); + .getHttpAddress(), capability, schedulerKey.getPriority(), null); } private ContainerAllocation handleNewContainerAllocation( ContainerAllocation allocationResult, FiCaSchedulerNode node, - Priority priority, RMContainer reservedContainer, Container container) { + SchedulerRequestKey schedulerKey, RMContainer reservedContainer, + Container container) { // Handling container allocation // Did we previously reserve containers at this 'priority'? if (reservedContainer != null) { - application.unreserve(priority, node, reservedContainer); + application.unreserve(schedulerKey, node, reservedContainer); } // Inform the application RMContainer allocatedContainer = application.allocate(allocationResult.containerNodeType, node, - priority, lastResourceRequest, container); + schedulerKey, lastResourceRequest, container); // Does the application need this resource? if (allocatedContainer == null) { @@ -637,12 +649,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } ContainerAllocation doAllocation(ContainerAllocation allocationResult, - FiCaSchedulerNode node, Priority priority, + FiCaSchedulerNode node, SchedulerRequestKey schedulerKey, RMContainer reservedContainer) { // Create the container if necessary Container container = getContainer(reservedContainer, node, - allocationResult.getResourceToBeAllocated(), priority); + allocationResult.getResourceToBeAllocated(), schedulerKey); // something went wrong getting/creating the container if (container == null) { @@ -655,11 +667,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { if (allocationResult.getAllocationState() == AllocationState.ALLOCATED) { // When allocating container allocationResult = - handleNewContainerAllocation(allocationResult, node, priority, + handleNewContainerAllocation(allocationResult, node, schedulerKey, reservedContainer, container); } else { // When reserving container - application.reserve(priority, node, reservedContainer, container); + application.reserve(schedulerKey, node, reservedContainer, container); } allocationResult.updatedContainer = container; @@ -678,14 +690,15 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // RACK_LOCAL without delay. if (allocationResult.containerNodeType == NodeType.NODE_LOCAL || application.getCSLeafQueue().getRackLocalityFullReset()) { - application.resetSchedulingOpportunities(priority); + application.resetSchedulingOpportunities(schedulerKey); } } // Non-exclusive scheduling opportunity is different: we need reset // it every time to make sure non-labeled resource request will be // most likely allocated on non-labeled nodes first. - application.resetMissedNonPartitionedRequestSchedulingOpportunity(priority); + application.resetMissedNonPartitionedRequestSchedulingOpportunity( + schedulerKey); } return allocationResult; @@ -693,15 +706,15 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { private ContainerAllocation allocate(Resource clusterResource, FiCaSchedulerNode node, SchedulingMode schedulingMode, - ResourceLimits resourceLimits, Priority priority, + ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey, RMContainer reservedContainer) { ContainerAllocation result = preAllocation(clusterResource, node, schedulingMode, resourceLimits, - priority, reservedContainer); + schedulerKey, reservedContainer); if (AllocationState.ALLOCATED == result.state || AllocationState.RESERVED == result.state) { - result = doAllocation(result, node, priority, reservedContainer); + result = doAllocation(result, node, schedulerKey, reservedContainer); } return result; @@ -725,10 +738,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } // Schedule in priority order - for (Priority priority : application.getPriorities()) { + for (SchedulerRequestKey schedulerKey : application.getSchedulerKeys()) { ContainerAllocation result = allocate(clusterResource, node, schedulingMode, resourceLimits, - priority, null); + schedulerKey, null); AllocationState allocationState = result.getAllocationState(); if (allocationState == AllocationState.PRIORITY_SKIPPED) { @@ -744,7 +757,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } else { ContainerAllocation result = allocate(clusterResource, node, schedulingMode, resourceLimits, - reservedContainer.getReservedPriority(), reservedContainer); + reservedContainer.getReservedSchedulerKey(), reservedContainer); return getCSAssignmentFromAllocateResult(clusterResource, result, reservedContainer); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index 80095807f94..67d93a40a91 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -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.ResourceScheduler; 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.CSAssignment; 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, - Priority priority, ResourceRequest request, + SchedulerRequestKey schedulerKey, ResourceRequest request, Container container) { if (isStopped) { @@ -190,10 +191,10 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { // Required sanity check - AM can call 'allocate' to update resource // request without locking the scheduler, hence we need to check - if (getTotalRequiredResources(priority) <= 0) { + if (getTotalRequiredResources(schedulerKey) <= 0) { return null; } - + // Create RMContainer RMContainer rmContainer = new RMContainerImpl(container, this.getApplicationAttemptId(), @@ -211,7 +212,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { // Update consumption and track allocations List resourceRequestList = appSchedulingInfo.allocate( - type, node, priority, request, container); + type, node, schedulerKey, request, container); attemptResourceUsage.incUsed(node.getPartition(), container.getResource()); @@ -235,13 +236,13 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { return rmContainer; } - public synchronized boolean unreserve(Priority priority, + public synchronized boolean unreserve(SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, RMContainer rmContainer) { // Cancel increase request (if it has reserved increase request rmContainer.cancelIncreaseReservation(); // Done with the reservation? - if (internalUnreserve(node, priority)) { + if (internalUnreserve(node, schedulerKey)) { node.unreserveResource(this); // Update reserved metrics @@ -254,12 +255,14 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { return false; } - private boolean internalUnreserve(FiCaSchedulerNode node, Priority priority) { + private boolean internalUnreserve(FiCaSchedulerNode node, + SchedulerRequestKey schedulerKey) { Map reservedContainers = - this.reservedContainers.get(priority); + this.reservedContainers.get(schedulerKey); if (reservedContainers != null) { - RMContainer reservedContainer = reservedContainers.remove(node.getNodeID()); + RMContainer reservedContainer = + reservedContainers.remove(node.getNodeID()); // unreserve is now triggered in new scenarios (preemption) // as a consequence reservedcontainer might be null, adding NP-checks @@ -268,17 +271,18 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { && reservedContainer.getContainer().getResource() != null) { if (reservedContainers.isEmpty()) { - this.reservedContainers.remove(priority); + this.reservedContainers.remove(schedulerKey); } // Reset the re-reservation count - resetReReservations(priority); + resetReReservations(schedulerKey); Resource resource = reservedContainer.getReservedResource(); this.attemptResourceUsage.decReserved(node.getPartition(), resource); LOG.info("Application " + getApplicationId() + " unreserved " + " on node " + node + ", currently has " - + reservedContainers.size() + " at priority " + priority + + reservedContainers.size() + + " at priority " + schedulerKey.getPriority() + "; currentReservation " + this.attemptResourceUsage.getReserved() + " on node-label=" + node.getPartition()); return true; @@ -288,10 +292,10 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { } public synchronized float getLocalityWaitFactor( - Priority priority, int clusterNodes) { + SchedulerRequestKey schedulerKey, int clusterNodes) { // Estimate: Required unique resources (i.e. hosts + racks) 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' // i.e. no point skipping more than clustersize opportunities @@ -354,14 +358,14 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { 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 // reservation that has enough resources Map reservedContainers = this.reservedContainers - .get(priority); + .get(schedulerKey); if ((reservedContainers != null) && (!reservedContainers.isEmpty())) { for (Map.Entry entry : reservedContainers.entrySet()) { @@ -417,17 +421,17 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { ((FiCaSchedulerApp) appAttempt).getHeadroomProvider(); } - public boolean reserveIncreasedContainer(Priority priority, + public boolean reserveIncreasedContainer(SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, RMContainer rmContainer, Resource reservedResource) { // Inform the application - if (super.reserveIncreasedContainer(node, priority, rmContainer, + if (super.reserveIncreasedContainer(node, schedulerKey, rmContainer, reservedResource)) { queue.getMetrics().reserveResource(getUser(), reservedResource); // Update the node - node.reserveResource(this, priority, rmContainer); + node.reserveResource(this, schedulerKey, rmContainer); // Succeeded return true; @@ -436,7 +440,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { return false; } - public void reserve(Priority priority, + public void reserve(SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, RMContainer rmContainer, Container container) { // Update reserved metrics if this is the first reservation if (rmContainer == null) { @@ -445,19 +449,19 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { } // Inform the application - rmContainer = super.reserve(node, priority, rmContainer, container); + rmContainer = super.reserve(node, schedulerKey, rmContainer, container); // Update the node - node.reserveResource(this, priority, rmContainer); + node.reserveResource(this, schedulerKey, rmContainer); } @VisibleForTesting public RMContainer findNodeToUnreserve(Resource clusterResource, - FiCaSchedulerNode node, Priority priority, + FiCaSchedulerNode node, SchedulerRequestKey schedulerKey, Resource minimumUnreservedResource) { // need to unreserve some other container first NodeId idToUnreserve = - getNodeIdToUnreserve(priority, minimumUnreservedResource, + getNodeIdToUnreserve(schedulerKey, minimumUnreservedResource, rc, clusterResource); if (idToUnreserve == null) { if (LOG.isDebugEnabled()) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java index f90a53c269f..d79fcafad5c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/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.Container; 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.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.util.resource.Resources; @@ -54,7 +55,7 @@ public class FiCaSchedulerNode extends SchedulerNode { @Override public synchronized void reserveResource( - SchedulerApplicationAttempt application, Priority priority, + SchedulerApplicationAttempt application, SchedulerRequestKey priority, RMContainer container) { // Check if it's already reserved RMContainer reservedContainer = getReservedContainer(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java index 1eead9a11c8..8f074cdc42a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java @@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.utils.BuilderUtils; 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 * was scheduled. Currently we use only the former. */ - private final Map allowedLocalityLevel = - new HashMap(); + private final Map allowedLocalityLevel = + new HashMap<>(); public FSAppAttempt(FairScheduler scheduler, ApplicationAttemptId applicationAttemptId, String user, FSLeafQueue queue, @@ -163,23 +164,23 @@ public class FSAppAttempt extends SchedulerApplicationAttempt } private synchronized void unreserveInternal( - Priority priority, FSSchedulerNode node) { + SchedulerRequestKey schedulerKey, FSSchedulerNode node) { Map reservedContainers = - this.reservedContainers.get(priority); + this.reservedContainers.get(schedulerKey); RMContainer reservedContainer = reservedContainers.remove(node.getNodeID()); if (reservedContainers.isEmpty()) { - this.reservedContainers.remove(priority); + this.reservedContainers.remove(schedulerKey); } // Reset the re-reservation count - resetReReservations(priority); + resetReReservations(schedulerKey); Resource resource = reservedContainer.getContainer().getResource(); this.attemptResourceUsage.decReserved(resource); LOG.info("Application " + getApplicationId() + " unreserved " + " on node " + node + ", currently has " + reservedContainers.size() - + " at priority " + priority + "; currentReservation " + + " at priority " + schedulerKey.getPriority() + "; currentReservation " + this.attemptResourceUsage.getReserved()); } @@ -239,10 +240,10 @@ public class FSAppAttempt extends SchedulerApplicationAttempt } public synchronized float getLocalityWaitFactor( - Priority priority, int clusterNodes) { + SchedulerRequestKey schedulerKey, int clusterNodes) { // Estimate: Required unique resources (i.e. hosts + racks) 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' // 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 * fail at (as a fraction of cluster size) before relaxing scheduling * 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 if (nodeLocalityThreshold > 1.0) { nodeLocalityThreshold = 1.0; } if (rackLocalityThreshold > 1.0) { rackLocalityThreshold = 1.0; } @@ -267,12 +274,12 @@ public class FSAppAttempt extends SchedulerApplicationAttempt } // 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; } - NodeType allowed = allowedLocalityLevel.get(priority); + NodeType allowed = allowedLocalityLevel.get(schedulerKey); // If level is already most liberal, we're done if (allowed.equals(NodeType.OFF_SWITCH)) return NodeType.OFF_SWITCH; @@ -281,27 +288,32 @@ public class FSAppAttempt extends SchedulerApplicationAttempt rackLocalityThreshold; // Relax locality constraints once we've surpassed threshold. - if (getSchedulingOpportunities(priority) > (numNodes * threshold)) { + if (getSchedulingOpportunities(schedulerKey) > (numNodes * threshold)) { 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)) { - 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. * Given the thresholds indicating how much time passed before relaxing * 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 (nodeLocalityDelayMs < 0 || rackLocalityDelayMs < 0) { @@ -309,19 +321,19 @@ public class FSAppAttempt extends SchedulerApplicationAttempt } // default level is NODE_LOCAL - if (!allowedLocalityLevel.containsKey(priority)) { + if (!allowedLocalityLevel.containsKey(schedulerKey)) { // add the initial time of priority to prevent comparing with FsApp // startTime and allowedLocalityLevel degrade - lastScheduledContainer.put(priority, currentTimeMs); + lastScheduledContainer.put(schedulerKey, currentTimeMs); 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; } - NodeType allowed = allowedLocalityLevel.get(priority); + NodeType allowed = allowedLocalityLevel.get(schedulerKey); // if level is already most liberal, we're done if (allowed.equals(NodeType.OFF_SWITCH)) { @@ -330,8 +342,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt // check waiting time long waitTime = currentTimeMs; - if (lastScheduledContainer.containsKey(priority)) { - waitTime -= lastScheduledContainer.get(priority); + if (lastScheduledContainer.containsKey(schedulerKey)) { + waitTime -= lastScheduledContainer.get(schedulerKey); } else { waitTime -= getStartTime(); } @@ -341,43 +353,43 @@ public class FSAppAttempt extends SchedulerApplicationAttempt if (waitTime > thresholdTime) { 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)) { - 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, - Priority priority, ResourceRequest request, + SchedulerRequestKey schedulerKey, ResourceRequest request, Container reservedContainer) { // Update allowed locality level - NodeType allowed = allowedLocalityLevel.get(priority); + NodeType allowed = allowedLocalityLevel.get(schedulerKey); if (allowed != null) { if (allowed.equals(NodeType.OFF_SWITCH) && (type.equals(NodeType.NODE_LOCAL) || type.equals(NodeType.RACK_LOCAL))) { - this.resetAllowedLocalityLevel(priority, type); + this.resetAllowedLocalityLevel(schedulerKey, type); } else if (allowed.equals(NodeType.RACK_LOCAL) && type.equals(NodeType.NODE_LOCAL)) { - this.resetAllowedLocalityLevel(priority, type); + this.resetAllowedLocalityLevel(schedulerKey, type); } } // Required sanity check - AM can call 'allocate' to update resource // request without locking the scheduler, hence we need to check - if (getTotalRequiredResources(priority) <= 0) { + if (getTotalRequiredResources(schedulerKey) <= 0) { return null; } Container container = reservedContainer; if (container == null) { container = - createContainer(node, request.getCapability(), request.getPriority()); + createContainer(node, request.getCapability(), schedulerKey); } // Create RMContainer @@ -392,7 +404,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt // Update consumption and track allocations List resourceRequestList = appSchedulingInfo.allocate( - type, node, priority, request, container); + type, node, schedulerKey, request, container); this.attemptResourceUsage.incUsed(container.getResource()); // 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 * degree of locality than the current threshold. Reset the allowed 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 " + - " priority " + priority); - allowedLocalityLevel.put(priority, level); + " priority " + schedulerKey.getPriority()); + allowedLocalityLevel.put(schedulerKey, level); } // related methods @@ -468,9 +482,13 @@ public class FSAppAttempt extends SchedulerApplicationAttempt * Create and return a container object reflecting an allocation for the * given appliction on the given node with the given capability and * 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(); ContainerId containerId = BuilderUtils.newContainerId( @@ -479,7 +497,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt // Create the container Container container = BuilderUtils.newContainer(containerId, nodeId, node.getRMNode() - .getHttpAddress(), capability, priority, null); + .getHttpAddress(), capability, schedulerKey.getPriority(), null); return container; } @@ -492,26 +510,26 @@ public class FSAppAttempt extends SchedulerApplicationAttempt * return whether reservation was possible with the current threshold limits */ 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)) { LOG.info("Making reservation: node=" + node.getNodeName() + " app_id=" + getApplicationId()); if (reservedContainer == null) { reservedContainer = createContainer(node, request.getCapability(), - request.getPriority()); + schedulerKey); getMetrics().reserveResource(getUser(), reservedContainer.getResource()); 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); } else { 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); } return true; @@ -548,13 +566,17 @@ public class FSAppAttempt extends SchedulerApplicationAttempt } 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. + * @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(); - unreserveInternal(priority, node); + unreserveInternal(schedulerKey, node); node.unreserveResource(this); clearReservation(node); getMetrics().unreserveResource( @@ -618,7 +640,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt */ private Resource assignContainer( FSSchedulerNode node, ResourceRequest request, NodeType type, - boolean reserved) { + boolean reserved, SchedulerRequestKey schedulerKey) { // How much does this request need? Resource capability = request.getCapability(); @@ -635,19 +657,19 @@ public class FSAppAttempt extends SchedulerApplicationAttempt if (Resources.fitsIn(capability, available)) { // Inform the application of the new container for this request RMContainer allocatedContainer = - allocate(type, node, request.getPriority(), request, + allocate(type, node, schedulerKey, request, reservedContainer); if (allocatedContainer == null) { // Did the application need this resource? if (reserved) { - unreserve(request.getPriority(), node); + unreserve(schedulerKey, node); } return Resources.none(); } // If we had previously made a reservation, delete it if (reserved) { - unreserve(request.getPriority(), node); + unreserve(schedulerKey, node); } // Inform the node @@ -667,7 +689,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt // The desired container won't fit here, so reserve if (isReservable(capability) && - reserve(request, node, reservedContainer, type)) { + reserve(request, node, reservedContainer, type, schedulerKey)) { return FairScheduler.CONTAINER_RESERVED; } else { if (LOG.isDebugEnabled()) { @@ -683,8 +705,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt 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); } - Collection prioritiesToTry = (reserved) ? - Arrays.asList(node.getReservedContainer().getReservedPriority()) : - getPriorities(); + Collection keysToTry = (reserved) ? + Arrays.asList(node.getReservedContainer().getReservedSchedulerKey()) : + getSchedulerKeys(); // 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 // (not scheduled) in order to promote better locality. synchronized (this) { - for (Priority priority : prioritiesToTry) { + for (SchedulerRequestKey schedulerKey : keysToTry) { // Skip it for reserved container, since // we already check it in isValidReservation. - if (!reserved && !hasContainerForNode(priority, node)) { + if (!reserved && !hasContainerForNode(schedulerKey, node)) { continue; } - addSchedulingOpportunity(priority); + addSchedulingOpportunity(schedulerKey); - ResourceRequest rackLocalRequest = getResourceRequest(priority, + ResourceRequest rackLocalRequest = getResourceRequest(schedulerKey, node.getRackName()); - ResourceRequest localRequest = getResourceRequest(priority, + ResourceRequest localRequest = getResourceRequest(schedulerKey, node.getNodeName()); if (localRequest != null && !localRequest.getRelaxLocality()) { @@ -736,12 +758,12 @@ public class FSAppAttempt extends SchedulerApplicationAttempt NodeType allowedLocality; if (scheduler.isContinuousSchedulingEnabled()) { - allowedLocality = getAllowedLocalityLevelByTime(priority, + allowedLocality = getAllowedLocalityLevelByTime(schedulerKey, scheduler.getNodeLocalityDelayMs(), scheduler.getRackLocalityDelayMs(), scheduler.getClock().getTime()); } else { - allowedLocality = getAllowedLocalityLevel(priority, + allowedLocality = getAllowedLocalityLevel(schedulerKey, scheduler.getNumClusterNodes(), scheduler.getNodeLocalityThreshold(), scheduler.getRackLocalityThreshold()); @@ -750,7 +772,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0 && localRequest != null && localRequest.getNumContainers() != 0) { return assignContainer(node, localRequest, - NodeType.NODE_LOCAL, reserved); + NodeType.NODE_LOCAL, reserved, schedulerKey); } if (rackLocalRequest != null && !rackLocalRequest.getRelaxLocality()) { @@ -761,21 +783,22 @@ public class FSAppAttempt extends SchedulerApplicationAttempt && (allowedLocality.equals(NodeType.RACK_LOCAL) || allowedLocality.equals(NodeType.OFF_SWITCH))) { return assignContainer(node, rackLocalRequest, - NodeType.RACK_LOCAL, reserved); + NodeType.RACK_LOCAL, reserved, schedulerKey); } ResourceRequest offSwitchRequest = - getResourceRequest(priority, ResourceRequest.ANY); + getResourceRequest(schedulerKey, ResourceRequest.ANY); if (offSwitchRequest != null && !offSwitchRequest.getRelaxLocality()) { continue; } if (offSwitchRequest != null && offSwitchRequest.getNumContainers() != 0) { - if (!hasNodeOrRackLocalRequests(priority) || + if (!hasNodeOrRackLocalRequests(schedulerKey) || allowedLocality.equals(NodeType.OFF_SWITCH)) { 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 * 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 // There must be outstanding requests at the given priority: @@ -812,9 +836,9 @@ public class FSAppAttempt extends SchedulerApplicationAttempt } private boolean isValidReservation(FSSchedulerNode node) { - Priority reservedPriority = node.getReservedContainer(). - getReservedPriority(); - return hasContainerForNode(reservedPriority, node) && + SchedulerRequestKey schedulerKey = node.getReservedContainer(). + getReservedSchedulerKey(); + return hasContainerForNode(schedulerKey, node) && !isOverAMShareLimit(); } @@ -830,13 +854,14 @@ public class FSAppAttempt extends SchedulerApplicationAttempt */ public boolean assignReservedContainer(FSSchedulerNode node) { RMContainer rmContainer = node.getReservedContainer(); - Priority reservedPriority = rmContainer.getReservedPriority(); + SchedulerRequestKey reservedSchedulerKey = + rmContainer.getReservedSchedulerKey(); if (!isValidReservation(node)) { // Don't hold the reservation if app can no longer use it LOG.info("Releasing reservation that cannot be satisfied for " + "application " + getApplicationAttemptId() + " on node " + node); - unreserve(reservedPriority, node); + unreserve(reservedSchedulerKey, node); return false; } @@ -938,8 +963,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt // Add up outstanding resource requests 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) { Resources.multiplyAndAddTo(demand, r.getCapability(), r.getNumContainers()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java index c86201ab583..024ec670040 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java @@ -23,10 +23,10 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.rmnode.RMNode; 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; @Private @@ -43,7 +43,7 @@ public class FSSchedulerNode extends SchedulerNode { @Override public synchronized void reserveResource( - SchedulerApplicationAttempt application, Priority priority, + SchedulerApplicationAttempt application, SchedulerRequestKey schedulerKey, RMContainer container) { // Check if it's already reserved RMContainer reservedContainer = getReservedContainer(); @@ -102,4 +102,5 @@ public class FSSchedulerNode extends SchedulerNode { public synchronized FSAppAttempt getReservedAppSchedulable() { return reservedAppSchedulable; } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java index bc953ba3737..ac384a11048 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java @@ -866,7 +866,7 @@ public class FairScheduler extends FSSchedulerNode node = getFSSchedulerNode(container.getNodeId()); if (rmContainer.getState() == RMContainerState.RESERVED) { - application.unreserve(rmContainer.getReservedPriority(), node); + application.unreserve(rmContainer.getReservedSchedulerKey(), node); } else { application.containerCompleted(rmContainer, containerStatus, event); node.releaseContainer(container); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index eaab495a0d0..fe8d0afaae8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/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.SchedulerApplication; 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.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; @@ -514,14 +515,15 @@ public class FifoScheduler extends 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 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 if (assignedContainers == 0) { break; @@ -553,11 +555,11 @@ public class FifoScheduler extends } private int getMaxAllocatableContainers(FiCaSchedulerApp application, - Priority priority, FiCaSchedulerNode node, NodeType type) { + SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, NodeType type) { int maxContainers = 0; - - ResourceRequest offSwitchRequest = - application.getResourceRequest(priority, ResourceRequest.ANY); + + ResourceRequest offSwitchRequest = + application.getResourceRequest(schedulerKey, ResourceRequest.ANY); if (offSwitchRequest != null) { maxContainers = offSwitchRequest.getNumContainers(); } @@ -567,8 +569,9 @@ public class FifoScheduler extends } if (type == NodeType.RACK_LOCAL) { - ResourceRequest rackLocalRequest = - application.getResourceRequest(priority, node.getRMNode().getRackName()); + ResourceRequest rackLocalRequest = + application.getResourceRequest(schedulerKey, node.getRMNode() + .getRackName()); if (rackLocalRequest == null) { return maxContainers; } @@ -577,8 +580,9 @@ public class FifoScheduler extends } if (type == NodeType.NODE_LOCAL) { - ResourceRequest nodeLocalRequest = - application.getResourceRequest(priority, node.getRMNode().getNodeAddress()); + ResourceRequest nodeLocalRequest = + application.getResourceRequest(schedulerKey, node.getRMNode() + .getNodeAddress()); if (nodeLocalRequest != null) { maxContainers = Math.min(maxContainers, nodeLocalRequest.getNumContainers()); } @@ -589,25 +593,25 @@ public class FifoScheduler extends private int assignContainersOnNode(FiCaSchedulerNode node, - FiCaSchedulerApp application, Priority priority + FiCaSchedulerApp application, SchedulerRequestKey schedulerKey ) { // Data-local - int nodeLocalContainers = - assignNodeLocalContainers(node, application, priority); + int nodeLocalContainers = + assignNodeLocalContainers(node, application, schedulerKey); // Rack-local - int rackLocalContainers = - assignRackLocalContainers(node, application, priority); + int rackLocalContainers = + assignRackLocalContainers(node, application, schedulerKey); // Off-switch int offSwitchContainers = - assignOffSwitchContainers(node, application, priority); + assignOffSwitchContainers(node, application, schedulerKey); LOG.debug("assignContainersOnNode:" + " node=" + node.getRMNode().getNodeAddress() + " application=" + application.getApplicationId().getId() + - " priority=" + priority.getPriority() + + " priority=" + schedulerKey.getPriority() + " #assigned=" + (nodeLocalContainers + rackLocalContainers + offSwitchContainers)); @@ -616,14 +620,14 @@ public class FifoScheduler extends } private int assignNodeLocalContainers(FiCaSchedulerNode node, - FiCaSchedulerApp application, Priority priority) { + FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) { int assignedContainers = 0; - ResourceRequest request = - application.getResourceRequest(priority, node.getNodeName()); + ResourceRequest request = + application.getResourceRequest(schedulerKey, node.getNodeName()); if (request != null) { // Don't allocate on this node if we don't need containers on this rack ResourceRequest rackRequest = - application.getResourceRequest(priority, + application.getResourceRequest(schedulerKey, node.getRMNode().getRackName()); if (rackRequest == null || rackRequest.getNumContainers() <= 0) { return 0; @@ -631,61 +635,62 @@ public class FifoScheduler extends int assignableContainers = Math.min( - getMaxAllocatableContainers(application, priority, node, + getMaxAllocatableContainers(application, schedulerKey, node, NodeType.NODE_LOCAL), request.getNumContainers()); assignedContainers = - assignContainer(node, application, priority, + assignContainer(node, application, schedulerKey, assignableContainers, request, NodeType.NODE_LOCAL); } return assignedContainers; } private int assignRackLocalContainers(FiCaSchedulerNode node, - FiCaSchedulerApp application, Priority priority) { + FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) { int assignedContainers = 0; - ResourceRequest request = - application.getResourceRequest(priority, node.getRMNode().getRackName()); + ResourceRequest request = + application.getResourceRequest(schedulerKey, node.getRMNode() + .getRackName()); if (request != null) { // Don't allocate on this rack if the application doens't need containers ResourceRequest offSwitchRequest = - application.getResourceRequest(priority, ResourceRequest.ANY); + application.getResourceRequest(schedulerKey, ResourceRequest.ANY); if (offSwitchRequest.getNumContainers() <= 0) { return 0; } int assignableContainers = Math.min( - getMaxAllocatableContainers(application, priority, node, + getMaxAllocatableContainers(application, schedulerKey, node, NodeType.RACK_LOCAL), request.getNumContainers()); assignedContainers = - assignContainer(node, application, priority, + assignContainer(node, application, schedulerKey, assignableContainers, request, NodeType.RACK_LOCAL); } return assignedContainers; } private int assignOffSwitchContainers(FiCaSchedulerNode node, - FiCaSchedulerApp application, Priority priority) { + FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) { int assignedContainers = 0; - ResourceRequest request = - application.getResourceRequest(priority, ResourceRequest.ANY); + ResourceRequest request = + application.getResourceRequest(schedulerKey, ResourceRequest.ANY); if (request != null) { assignedContainers = - assignContainer(node, application, priority, + assignContainer(node, application, schedulerKey, request.getNumContainers(), request, NodeType.OFF_SWITCH); } return assignedContainers; } private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application, - Priority priority, int assignableContainers, + SchedulerRequestKey schedulerKey, int assignableContainers, ResourceRequest request, NodeType type) { LOG.debug("assignContainers:" + " node=" + node.getRMNode().getNodeAddress() + " application=" + application.getApplicationId().getId() + - " priority=" + priority.getPriority() + + " priority=" + schedulerKey.getPriority().getPriority() + " assignableContainers=" + assignableContainers + " request=" + request + " type=" + type); Resource capability = request.getCapability(); @@ -707,13 +712,13 @@ public class FifoScheduler extends // Create the container Container container = BuilderUtils.newContainer(containerId, nodeId, node.getRMNode() - .getHttpAddress(), capability, priority, null); + .getHttpAddress(), capability, schedulerKey.getPriority(), null); // Allocate! // Inform the application RMContainer rmContainer = - application.allocate(type, node, priority, request, container); + application.allocate(type, node, schedulerKey, request, container); // Inform the node node.allocateContainer(rmContainer); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java index 60233518148..e70c3e01d3f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java @@ -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.NodeType; 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.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.util.Records; @@ -78,25 +84,21 @@ public class Application { final private ApplicationAttemptId applicationAttemptId; final private ResourceManager resourceManager; private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); - - final private Map requestSpec = - new TreeMap( - new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator()); - - final private Map> requests = - new TreeMap>( - new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator()); - - final Map> tasks = - new TreeMap>( - new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator()); - - final private Set ask = - new TreeSet( - new org.apache.hadoop.yarn.api.records.ResourceRequest.ResourceRequestComparator()); - final private Map nodes = - new HashMap(); + final private Map requestSpec = + new TreeMap<>(); + + final private Map> + requests = new TreeMap<>(); + + final Map> tasks = new TreeMap<>(); + + final private Set ask = + new TreeSet<>( + new org.apache.hadoop.yarn.api.records.ResourceRequest + .ResourceRequestComparator()); + + final private Map nodes = new HashMap<>(); Resource used = recordFactory.newRecordInstance(Resource.class); @@ -188,13 +190,19 @@ public class Application { new AppAttemptAddedSchedulerEvent(this.applicationAttemptId, false); scheduler.handle(addAttemptEvent); } - + public synchronized void addResourceRequestSpec( 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) { 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) { - Priority priority = task.getPriority(); - Map requests = this.requests.get(priority); + SchedulerRequestKey schedulerKey = task.getSchedulerKey(); + Map requests = this.requests.get(schedulerKey); if (requests == null) { requests = new HashMap(); - this.requests.put(priority, requests); + this.requests.put(schedulerKey, requests); 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 - Set tasks = this.tasks.get(priority); + Set tasks = this.tasks.get(schedulerKey); if (tasks == null) { tasks = new HashSet(); - this.tasks.put(priority, tasks); + this.tasks.put(schedulerKey, tasks); } tasks.add(task); LOG.info("Added task " + task.getTaskId() + " to application " + - applicationId + " at priority " + priority); + applicationId + " at priority " + schedulerKey.getPriority()); if(LOG.isDebugEnabled()) { LOG.debug("addTask: application=" + applicationId @@ -240,21 +248,21 @@ public class Application { // Create resource requests for (String host : task.getHosts()) { // Data-local - addResourceRequest(priority, requests, host, capability); + addResourceRequest(schedulerKey, requests, host, capability); } // Rack-local for (String rack : task.getRacks()) { - addResourceRequest(priority, requests, rack, capability); + addResourceRequest(schedulerKey, requests, rack, capability); } // Off-switch - addResourceRequest(priority, requests, ResourceRequest.ANY, capability); + addResourceRequest(schedulerKey, requests, ResourceRequest.ANY, capability); } public synchronized void finishTask(Task task) throws IOException, YarnException { - Set tasks = this.tasks.get(task.getPriority()); + Set tasks = this.tasks.get(task.getSchedulerKey()); if (!tasks.remove(task)) { throw new IllegalStateException( "Finishing unknown task " + task.getTaskId() + @@ -270,7 +278,7 @@ public class Application { StopContainersRequest.newInstance(containerIds); nodeManager.stopContainers(stopRequest); - Resources.subtractFrom(used, requestSpec.get(task.getPriority())); + Resources.subtractFrom(used, requestSpec.get(task.getSchedulerKey())); LOG.info("Finished task " + task.getTaskId() + " of application " + applicationId + @@ -279,13 +287,13 @@ public class Application { } private synchronized void addResourceRequest( - Priority priority, Map requests, + SchedulerRequestKey schedulerKey, Map requests, String resourceName, Resource capability) { ResourceRequest request = requests.get(resourceName); if (request == null) { request = org.apache.hadoop.yarn.server.utils.BuilderUtils.newResourceRequest( - priority, resourceName, capability, 1); + schedulerKey.getPriority(), resourceName, capability, 1); requests.put(resourceName, request); } else { request.setNumContainers(request.getNumContainers() + 1); @@ -299,13 +307,13 @@ public class Application { ask.add( org.apache.hadoop.yarn.server.utils.BuilderUtils.newResourceRequest( request)); // clone to ensure the RM doesn't manipulate the same obj - - if(LOG.isDebugEnabled()) { + + if (LOG.isDebugEnabled()) { 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(); // 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()) { break; @@ -368,15 +376,18 @@ public class Application { assign(getResources()); } - private synchronized void assign(Priority priority, NodeType type, - List containers) throws IOException, YarnException { + private synchronized void assign(SchedulerRequestKey schedulerKey, + NodeType type, List containers) + throws IOException, YarnException { for (Iterator i=containers.iterator(); i.hasNext();) { Container container = i.next(); 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 - for (Iterator t=tasks.get(priority).iterator(); t.hasNext();) { + for (Iterator t=tasks.get(schedulerKey).iterator(); + t.hasNext();) { Task task = t.next(); if (task.getState() == State.PENDING && task.canSchedule(type, host)) { NodeManager nodeManager = getNodeManager(host); @@ -386,14 +397,15 @@ public class Application { // Track application resource usage Resources.addTo(used, container.getResource()); - + 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() + ", currently using " + used + " resources"); // Update resource requests - updateResourceRequests(requests.get(priority), type, task); + updateResourceRequests(requests.get(schedulerKey), type, task); // Launch the container StartContainerRequest scRequest = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Task.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Task.java index eebfa1da9e5..35218bdf119 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Task.java +++ b/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.ResourceRequest; 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 { private static final Log LOG = LogFactory.getLog(Task.class); @@ -40,6 +43,7 @@ public class Task { final private ApplicationId applicationId; final private int taskId; final private Priority priority; + final private SchedulerRequestKey schedulerKey; final private Set hosts = new HashSet(); final private Set racks = new HashSet(); @@ -48,7 +52,7 @@ public class Task { private org.apache.hadoop.yarn.server.resourcemanager.NodeManager nodeManager; private State state; - + public Task(Application application, Priority priority, String[] hosts) { this.applicationId = application.getApplicationId(); this.priority = priority; @@ -64,6 +68,7 @@ public class Task { this.racks.add(Application.resolve(host)); } } + this.schedulerKey = TestUtils.toSchedulerKey(priority.getPriority()); LOG.info("Task " + taskId + " added to application " + this.applicationId + " with " + this.hosts.size() + " hosts, " + racks.size() + " racks"); } @@ -75,6 +80,10 @@ public class Task { public Priority getPriority() { return priority; } + + public SchedulerRequestKey getSchedulerKey() { + return schedulerKey; + } public org.apache.hadoop.yarn.server.resourcemanager.NodeManager getNodeManager() { return nodeManager; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java index 7c8fb2ab222..3d3f1ea7e08 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java @@ -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.scheduler.ResourceUsage; 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.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; @@ -205,7 +207,9 @@ public class ProportionalCapacityPreemptionPolicyMockFramework { Container c = mock(Container.class); when(c.getResource()).thenReturn(res); when(c.getPriority()).thenReturn(pri); + SchedulerRequestKey sk = SchedulerRequestKey.extractFrom(c); RMContainerImpl rmc = mock(RMContainerImpl.class); + when(rmc.getAllocatedSchedulerKey()).thenReturn(sk); when(rmc.getAllocatedNode()).thenReturn(host); when(rmc.getNodeLabelExpression()).thenReturn(exp); when(rmc.getAllocatedResource()).thenReturn(res); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java index e3ef8c28fd4..a115aac5c75 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java @@ -36,6 +36,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.scheduler.ResourceUsage; 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.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; @@ -1318,8 +1320,10 @@ public class TestProportionalCapacityPreemptionPolicy { Container c = mock(Container.class); when(c.getResource()).thenReturn(r); when(c.getPriority()).thenReturn(Priority.create(cpriority)); + SchedulerRequestKey sk = SchedulerRequestKey.extractFrom(c); RMContainer mC = mock(RMContainer.class); when(mC.getContainerId()).thenReturn(cId); + when(mC.getAllocatedSchedulerKey()).thenReturn(sk); when(mC.getContainer()).thenReturn(c); when(mC.getApplicationAttemptId()).thenReturn(appAttId); when(mC.getAllocatedResource()).thenReturn(r); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java index ed8d56fa1e6..e737a845f01 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java @@ -120,7 +120,8 @@ public class TestRMContainerImpl { assertEquals(RMContainerState.NEW, rmContainer.getState()); assertEquals(resource, rmContainer.getAllocatedResource()); assertEquals(nodeId, rmContainer.getAllocatedNode()); - assertEquals(priority, rmContainer.getAllocatedPriority()); + assertEquals(priority, + rmContainer.getAllocatedSchedulerKey().getPriority()); verify(writer).containerStarted(any(RMContainer.class)); verify(publisher).containerCreated(any(RMContainer.class), anyLong()); @@ -221,7 +222,8 @@ public class TestRMContainerImpl { assertEquals(RMContainerState.NEW, rmContainer.getState()); assertEquals(resource, rmContainer.getAllocatedResource()); assertEquals(nodeId, rmContainer.getAllocatedNode()); - assertEquals(priority, rmContainer.getAllocatedPriority()); + assertEquals(priority, + rmContainer.getAllocatedSchedulerKey().getPriority()); verify(writer).containerStarted(any(RMContainer.class)); verify(publisher).containerCreated(any(RMContainer.class), anyLong()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java index 54166c0d622..3cb668c3151 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java +++ b/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.Test; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey; + public class TestSchedulerApplicationAttempt { private static final NodeId nodeId = NodeId.newInstance("somehost", 5); @@ -88,7 +90,8 @@ public class TestSchedulerApplicationAttempt { RMContainer container1 = createRMContainer(appAttId, 1, requestedResource); app.liveContainers.put(container1.getContainerId(), container1); SchedulerNode node = createNode(); - app.appSchedulingInfo.allocate(NodeType.OFF_SWITCH, node, requestedPriority, + app.appSchedulingInfo.allocate(NodeType.OFF_SWITCH, node, + toSchedulerKey(requestedPriority), request, container1.getContainer()); // Reserved container @@ -98,7 +101,7 @@ public class TestSchedulerApplicationAttempt { node.getNodeID(), prio1); Map reservations = new HashMap(); reservations.put(node.getNodeID(), container2); - app.reservedContainers.put(prio1, reservations); + app.reservedContainers.put(toSchedulerKey(prio1), reservations); oldMetrics.reserveResource(user, reservedResource); 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) { RMContainer container = createRMContainer(appAttId, id, resource); when(container.getReservedResource()).thenReturn(resource); - when(container.getReservedPriority()).thenReturn(reservedPriority); + when(container.getReservedSchedulerKey()) + .thenReturn(toSchedulerKey(reservedPriority)); when(container.getReservedNode()).thenReturn(nodeId); return container; } @@ -260,16 +264,19 @@ public class TestSchedulerApplicationAttempt { SchedulerApplicationAttempt app = new SchedulerApplicationAttempt( attemptId, "user", queue, queue.getActiveUsersManager(), rmContext); 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 - app.setSchedulingOpportunities(priority, Integer.MAX_VALUE - 1); + app.setSchedulingOpportunities(schedulerKey, 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)); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java index 7c3429241f4..fb021c0132a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java @@ -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.SchedulerApplication; 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.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils; @@ -1244,7 +1246,7 @@ public class TestCapacityScheduler { rm1.stop(); } - @Test(timeout = 30000) + @Test(timeout = 300000) public void testRecoverRequestAfterPreemption() throws Exception { Configuration conf = new Configuration(); 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 // allocation. - Assert.assertNull(app.getResourceRequest(request.getPriority(), - request.getResourceName())); + Assert.assertNull(app.getResourceRequest( + SchedulerRequestKey.create(request), request.getResourceName())); } // Call killContainer to preempt the container @@ -1290,7 +1292,7 @@ public class TestCapacityScheduler { // handling. Assert.assertEquals( 1, - app.getResourceRequest(request.getPriority(), + app.getResourceRequest(SchedulerRequestKey.create(request), request.getResourceName()).getNumContainers()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java index 48e6f0e1272..274c0631769 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java @@ -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.ResourceLimits; 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.common.fica.FiCaSchedulerApp; 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.Mockito; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey; + public class TestLeafQueue { private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); @@ -731,6 +736,7 @@ public class TestLeafQueue { qb.getActiveUsersManager(), spyRMContext); qb.submitApplicationAttempt(app_0, user_0); Priority u0Priority = TestUtils.createMockPriority(1); + SchedulerRequestKey u0SchedKey = toSchedulerKey(u0Priority); app_0.updateResourceRequests(Collections.singletonList( TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true, u0Priority, recordFactory))); @@ -753,6 +759,7 @@ public class TestLeafQueue { new FiCaSchedulerApp(appAttemptId_2, user_1, qb, qb.getActiveUsersManager(), spyRMContext); Priority u1Priority = TestUtils.createMockPriority(2); + SchedulerRequestKey u1SchedKey = toSchedulerKey(u1Priority); app_2.updateResourceRequests(Collections.singletonList( TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true, u1Priority, recordFactory))); @@ -773,9 +780,9 @@ public class TestLeafQueue { //test case 3 qb.finishApplication(app_0.getApplicationId(), user_0); qb.finishApplication(app_2.getApplicationId(), user_1); - qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority), + qb.releaseResource(clusterResource, app_0, app_0.getResource(u0SchedKey), null, null, false); - qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority), + qb.releaseResource(clusterResource, app_2, app_2.getResource(u1SchedKey), null, null, false); qb.setUserLimit(50); @@ -1452,7 +1459,7 @@ public class TestLeafQueue { assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); assertEquals(4*GB, app_1.getCurrentReservation().getMemorySize()); assertEquals(1*GB, node_0.getAllocatedResource().getMemorySize()); - assertEquals(1, app_1.getReReservations(priority)); + assertEquals(1, app_1.getReReservations(toSchedulerKey(priority))); // Re-reserve a.assignContainers(clusterResource, node_0, @@ -1462,7 +1469,7 @@ public class TestLeafQueue { assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); assertEquals(4*GB, app_1.getCurrentReservation().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 a.assignContainers(clusterResource, node_1, @@ -1474,7 +1481,7 @@ public class TestLeafQueue { assertEquals(4*GB, node_1.getAllocatedResource().getMemorySize()); // Doesn't change yet... only when reservation is cancelled or a different // 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 rmContainer = app_0.getLiveContainers().iterator().next(); @@ -1564,29 +1571,30 @@ public class TestLeafQueue { // Start testing... CSAssignment assignment = null; - + + SchedulerRequestKey schedulerKey = toSchedulerKey(priority); // Start with off switch, shouldn't allocate due to delay scheduling assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 // Another off switch, shouldn't allocate due to delay scheduling assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 // Another off switch, shouldn't allocate due to delay scheduling assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 // Another off switch, now we should allocate @@ -1594,22 +1602,25 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 assignment = a.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 assignment = a.assignContainers(clusterResource, node_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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()); // 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 true, priority, recordFactory)); 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 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 assignment = a.assignContainers(clusterResource, node_3, 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 assignment = a.assignContainers(clusterResource, node_3, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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. assignment = a.assignContainers(clusterResource, node_3, 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 doReturn(false).when(a).getRackLocalityFullReset(); @@ -1659,19 +1671,21 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_3, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 assignment = a.assignContainers(clusterResource, node_3, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 // 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.add( TestUtils.createResourceRequest(host_0, 1*GB, 100, @@ -1690,13 +1704,13 @@ public class TestLeafQueue { a.assignContainers(clusterResource, node_2, new ResourceLimits( clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); verifyContainerAllocated(assignment, NodeType.OFF_SWITCH); - assertEquals(numNodes+1, app_0.getSchedulingOpportunities(priority)); + assertEquals(numNodes+1, app_0.getSchedulingOpportunities(schedulerKey)); } @Test @@ -1738,6 +1752,7 @@ public class TestLeafQueue { // P1 Priority priority_1 = TestUtils.createMockPriority(1); + SchedulerRequestKey schedulerKey1 = toSchedulerKey(priority_1); app_0_requests_0.add( TestUtils.createResourceRequest(host_0, 1*GB, 1, true, priority_1, recordFactory)); @@ -1756,6 +1771,7 @@ public class TestLeafQueue { // P2 Priority priority_2 = TestUtils.createMockPriority(2); + SchedulerRequestKey schedulerKey2 = toSchedulerKey(priority_2); app_0_requests_0.add( TestUtils.createResourceRequest(host_2, 2*GB, 1, true, priority_2, recordFactory)); @@ -1775,47 +1791,47 @@ public class TestLeafQueue { CSAssignment assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 // thus, no P2 either! assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 assignment = a.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 assignment = a.assignContainers(clusterResource, node_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 Priority priority = TestUtils.createMockPriority(1); + SchedulerRequestKey schedulerKey = toSchedulerKey(priority); List app_0_requests_0 = new ArrayList(); app_0_requests_0.add( TestUtils.createResourceRequest(host_0_0, 1*GB, 1, @@ -1878,7 +1895,7 @@ public class TestLeafQueue { // Add one request app_0_requests_0.clear(); 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)); app_0.updateResourceRequests(app_0_requests_0); @@ -1886,17 +1903,19 @@ public class TestLeafQueue { CSAssignment assignment = a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 // required(ANY) == 0 assignment = a.assignContainers(clusterResource, node_1_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 app_0_requests_0.clear(); @@ -1910,15 +1929,16 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_0_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 assignment = a.assignContainers(clusterResource, node_1_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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) @@ -2142,6 +2162,7 @@ public class TestLeafQueue { // host_1_1: 8G // Blacklist: Priority priority = TestUtils.createMockPriority(1); + SchedulerRequestKey schedulerKey = toSchedulerKey(priority); List app_0_requests_0 = new ArrayList(); app_0_requests_0.add( TestUtils.createResourceRequest(host_0_0, 1*GB, 1, @@ -2169,7 +2190,8 @@ public class TestLeafQueue { a.assignContainers(clusterResource, node_0_1, new ResourceLimits( clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); verifyNoContainerAllocated(assignment); - assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0 + // should be 0 + assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); // resourceName: // host_0_0: < 1, 1GB, 1, true > @@ -2191,7 +2213,8 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_1_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 app_0_requests_0.add( @@ -2221,7 +2244,8 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_1_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 app_0.updateResourceRequests(app_0_requests_0); @@ -2249,7 +2273,8 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_1_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 app_0.updateResourceRequests(app_0_requests_0); @@ -2275,8 +2300,8 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_1_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 app_0_requests_0.add( @@ -2305,8 +2330,8 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_1_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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 // request for host2. Priority priority = TestUtils.createMockPriority(1); + SchedulerRequestKey schedulerKey = toSchedulerKey(priority); List app_0_requests_0 = new ArrayList(); app_0_requests_0.add( TestUtils.createResourceRequest(host_0, 1*GB, 1, @@ -2706,8 +2732,8 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); 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(1, app_1.getLiveContainers().size()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java index 47fd5348731..df2c9ffeff8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java @@ -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.NodeId; 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.conf.YarnConfiguration; 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.scheduler.ResourceScheduler; 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.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; @@ -549,7 +549,7 @@ public class TestNodeLabelContainerAllocation { FiCaSchedulerApp app = cs.getApplicationAttempt(attemptId); ResourceRequest rr = app.getAppSchedulingInfo().getResourceRequest( - Priority.newInstance(priority), "*"); + TestUtils.toSchedulerKey(priority), "*"); Assert.assertEquals(memory, rr.getCapability().getMemorySize() * rr.getNumContainers()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java index e8ac804557b..5e2007c0cbf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java @@ -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.ResourceUsage; 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.common.fica.FiCaSchedulerApp; 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.mockito.Mockito; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey; + public class TestReservations { private static final Log LOG = LogFactory.getLog(TestReservations.class); @@ -300,7 +304,8 @@ public class TestReservations { assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize()); assertEquals(3 * GB, node_1.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) a.assignContainers(clusterResource, node_0, @@ -316,7 +321,8 @@ public class TestReservations { assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize()); assertEquals(3 * GB, node_1.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 a.assignContainers(clusterResource, node_2, @@ -332,7 +338,8 @@ public class TestReservations { assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize()); assertEquals(3 * GB, node_1.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 // on node_1 @@ -348,7 +355,8 @@ public class TestReservations { assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize()); assertEquals(8 * GB, node_1.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 @@ -597,7 +605,8 @@ public class TestReservations { assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize()); assertEquals(3 * GB, node_1.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) a.assignContainers(clusterResource, node_0, @@ -613,7 +622,8 @@ public class TestReservations { assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize()); assertEquals(3 * GB, node_1.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 a.assignContainers(clusterResource, node_2, @@ -629,7 +639,8 @@ public class TestReservations { assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize()); assertEquals(3 * GB, node_1.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 // if AM doesn't handle @@ -646,7 +657,8 @@ public class TestReservations { assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize()); assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize()); assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize()); - assertEquals(1, app_0.getTotalRequiredResources(priorityReduce)); + assertEquals(1, app_0.getTotalRequiredResources( + toSchedulerKey(priorityReduce))); } @Test @@ -754,7 +766,8 @@ public class TestReservations { assertEquals(null, node_0.getReservedContainer()); assertEquals(5 * GB, node_0.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) a.assignContainers(clusterResource, node_0, @@ -769,7 +782,8 @@ public class TestReservations { .getMemorySize()); assertEquals(5 * GB, node_0.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 a.assignContainers(clusterResource, node_1, @@ -783,7 +797,8 @@ public class TestReservations { assertEquals(null, node_0.getReservedContainer()); assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize()); assertEquals(8 * GB, node_1.getAllocatedResource().getMemorySize()); - assertEquals(1, app_0.getTotalRequiredResources(priorityReduce)); + assertEquals(1, app_0.getTotalRequiredResources( + toSchedulerKey(priorityReduce))); } @Test @@ -808,7 +823,8 @@ public class TestReservations { Resource clusterResource = Resources.createResource(2 * 8 * GB); // Setup resource-requests - Priority priorityMap = TestUtils.createMockPriority(5); + Priority p = TestUtils.createMockPriority(5); + SchedulerRequestKey priorityMap = toSchedulerKey(p); Resource capability = Resources.createResource(2*GB, 0); RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); @@ -826,12 +842,14 @@ public class TestReservations { app_0.getApplicationId(), 1); ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1); 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, node_1.getNodeID(), "user", rmContext); 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, node_0.getNodeID(), "user", rmContext); @@ -878,7 +896,8 @@ public class TestReservations { 8 * GB); // Setup resource-requests - Priority priorityMap = TestUtils.createMockPriority(5); + Priority p = TestUtils.createMockPriority(5); + SchedulerRequestKey priorityMap = toSchedulerKey(p); Resource capability = Resources.createResource(2 * GB, 0); RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); @@ -896,7 +915,8 @@ public class TestReservations { app_0.getApplicationId(), 1); ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1); 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, node_1.getNodeID(), "user", rmContext); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java index 621c5c52d6c..c808b5a260c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java @@ -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.rmnode.RMNode; 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.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager; @@ -400,4 +402,14 @@ public class TestUtils { 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)); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java b/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 index af1dc62bae5..61c57439699 100644 --- a/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 +++ b/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.scheduler.NodeType; 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.capacity + .TestUtils; 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.FairSharePolicy; @@ -63,8 +66,9 @@ public class TestFSAppAttempt extends FairSchedulerTestBase { @Test public void testDelayScheduling() { 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 rackLocalityThreshold = .6; @@ -122,8 +126,9 @@ public class TestFSAppAttempt extends FairSchedulerTestBase { public void testDelaySchedulingForContinuousScheduling() throws InterruptedException { 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(); scheduler.setClock(clock); @@ -180,8 +185,9 @@ public class TestFSAppAttempt extends FairSchedulerTestBase { */ public void testLocalityLevelWithoutDelays() { 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(); ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/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 index f92af35e3ba..dab7312af4d 100644 --- a/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 +++ b/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.QueueMetrics; 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.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.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; @@ -2316,7 +2321,8 @@ public class TestFairScheduler extends FairSchedulerTestBase { assertEquals(1, app.getLiveContainers().size()); // Reserved container should still be at lower priority for (RMContainer container : app.getReservedContainers()) { - assertEquals(2, container.getReservedPriority().getPriority()); + assertEquals(2, + container.getReservedSchedulerKey().getPriority().getPriority()); } // Complete container @@ -2817,7 +2823,7 @@ public class TestFairScheduler extends FairSchedulerTestBase { scheduler.handle(node1UpdateEvent); assertEquals(1, app.getLiveContainers().size()); } - + @Test public void testCancelStrictLocality() throws IOException { scheduler.init(conf); @@ -4485,9 +4491,10 @@ public class TestFairScheduler extends FairSchedulerTestBase { // time clock.tickSec(DELAY_THRESHOLD_TIME_MS / 1000); scheduler.attemptScheduling(node); - Map lastScheduledContainer = + Map lastScheduledContainer = fsAppAttempt.getLastScheduledContainer(); - long initSchedulerTime = lastScheduledContainer.get(priority); + long initSchedulerTime = + lastScheduledContainer.get(TestUtils.toSchedulerKey(priority)); assertEquals(DELAY_THRESHOLD_TIME_MS, initSchedulerTime); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java b/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 index 07a2dcaf60d..79f4601e21c 100644 --- a/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 +++ b/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.Container; 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.ResourceRequest; 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.ResourceScheduler; 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.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; @@ -375,13 +378,15 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase { Set set = new HashSet(); for (RMContainer container : scheduler.getSchedulerApp(app2).getLiveContainers()) { - if (container.getAllocatedPriority().getPriority() == 4) { + if (container.getAllocatedSchedulerKey().getPriority().getPriority() == + 4) { set.add(container); } } for (RMContainer container : scheduler.getSchedulerApp(app4).getLiveContainers()) { - if (container.getAllocatedPriority().getPriority() == 4) { + if (container.getAllocatedSchedulerKey().getPriority().getPriority() == + 4) { set.add(container); } } @@ -1399,7 +1404,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase { scheduler.start(); scheduler.reinitialize(conf, resourceManager.getRMContext()); - Priority priority = Priority.newInstance(20); + SchedulerRequestKey schedulerKey = TestUtils.toSchedulerKey(20); String host = "127.0.0.1"; int GB = 1024; @@ -1412,11 +1417,12 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase { // Create 3 container requests and place it in ask List ask = new ArrayList(); ResourceRequest nodeLocalRequest = createResourceRequest(GB, 1, host, - priority.getPriority(), 1, true); + schedulerKey.getPriority().getPriority(), 1, true); ResourceRequest rackLocalRequest = createResourceRequest(GB, 1, - node.getRackName(), priority.getPriority(), 1, true); + node.getRackName(), schedulerKey.getPriority().getPriority(), 1, + true); ResourceRequest offRackRequest = createResourceRequest(GB, 1, - ResourceRequest.ANY, priority.getPriority(), 1, true); + ResourceRequest.ANY, schedulerKey.getPriority().getPriority(), 1, true); ask.add(nodeLocalRequest); ask.add(rackLocalRequest); ask.add(offRackRequest); @@ -1435,7 +1441,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase { SchedulerApplicationAttempt app = scheduler.getSchedulerApp(appAttemptId); // 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); RMContainer rmContainer = app.getRMContainer(containerId1); @@ -1458,7 +1464,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase { Assert.assertEquals(3, requests.size()); for (ResourceRequest request : requests) { Assert.assertEquals(1, - app.getResourceRequest(priority, request.getResourceName()) + app.getResourceRequest(schedulerKey, request.getResourceName()) .getNumContainers()); }