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

This commit is contained in:
Arun Suresh 2016-07-21 20:57:44 -07:00
parent d2cf8b54c5
commit 5aace38b74
32 changed files with 925 additions and 623 deletions

View File

@ -23,7 +23,6 @@
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 @@ static void sortContainers(List<RMContainer> containers){
Collections.sort(containers, new Comparator<RMContainer>() {
@Override
public int compare(RMContainer a, RMContainer b) {
Comparator<Priority> c = new org.apache.hadoop.yarn.server
.resourcemanager.resource.Priority.Comparator();
int priorityComp = c.compare(b.getContainer().getPriority(),
a.getContainer().getPriority());
if (priorityComp != 0) {
return priorityComp;
int schedKeyComp = b.getAllocatedSchedulerKey()
.compareTo(a.getAllocatedSchedulerKey());
if (schedKeyComp != 0) {
return schedKeyComp;
}
return b.getContainerId().compareTo(a.getContainerId());
}

View File

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

View File

@ -31,7 +31,8 @@
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<RMContainerEvent> {
NodeId getReservedNode();
Priority getReservedPriority();
SchedulerRequestKey getReservedSchedulerKey();
Resource getAllocatedResource();
@ -63,6 +64,8 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
NodeId getAllocatedNode();
SchedulerRequestKey getAllocatedSchedulerKey();
Priority getAllocatedPriority();
long getCreationTime();

View File

@ -53,12 +53,12 @@
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 @@ RMContainerEventType.CHANGE_RESOURCE, new ChangeResourceTransition())
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 @@ RMContainerEventType.CHANGE_RESOURCE, new ChangeResourceTransition())
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 RMContainerImpl(Container container,
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 NodeId getReservedNode() {
}
@Override
public Priority getReservedPriority() {
return reservedPriority;
public SchedulerRequestKey getReservedSchedulerKey() {
return reservedSchedulerKey;
}
@Override
@ -325,6 +327,11 @@ public NodeId getAllocatedNode() {
return container.getNodeId();
}
@Override
public SchedulerRequestKey getAllocatedSchedulerKey() {
return allocatedSchedulerKey;
}
@Override
public Priority getAllocatedPriority() {
return container.getPriority();
@ -535,7 +542,7 @@ public void transition(RMContainerImpl container, RMContainerEvent event) {
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 ContainerReport createContainerReport() {
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());

View File

@ -20,8 +20,8 @@
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 NodeId getReservedNode() {
return reservedNode;
}
public Priority getReservedPriority() {
return reservedPriority;
public SchedulerRequestKey getReservedSchedulerKey() {
return reservedSchedulerKey;
}
}

View File

@ -20,7 +20,6 @@
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.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 @@
public class AppSchedulingInfo {
private static final Log LOG = LogFactory.getLog(AppSchedulingInfo.class);
private static final Comparator<Priority> COMPARATOR =
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator();
private static final int EPOCH_BIT_SHIFT = 40;
private final ApplicationId applicationId;
@ -82,10 +78,10 @@ public class AppSchedulingInfo {
private Set<String> requestedPartitions = new HashSet<>();
final Set<Priority> priorities = new TreeSet<>(COMPARATOR);
final Map<Priority, Map<String, ResourceRequest>> resourceRequestMap =
new ConcurrentHashMap<>();
final Map<NodeId, Map<Priority, Map<ContainerId,
final Set<SchedulerRequestKey> schedulerKeys = new TreeSet<>();
final Map<SchedulerRequestKey, Map<String, ResourceRequest>>
resourceRequestMap = new ConcurrentHashMap<>();
final Map<NodeId, Map<SchedulerRequestKey, Map<ContainerId,
SchedContainerChangeRequest>>> containerIncreaseRequestMap =
new ConcurrentHashMap<>();
@ -134,22 +130,23 @@ public Set<String> getRequestedPartitions() {
* 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<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
containerIncreaseRequestMap.get(nodeId);
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
return requestsOnNode == null ? false : requestsOnNode.size() > 0;
}
public synchronized Map<ContainerId, SchedContainerChangeRequest>
getIncreaseRequests(NodeId nodeId, Priority priority) {
Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
containerIncreaseRequestMap.get(nodeId);
return requestsOnNode == null ? null : requestsOnNode.get(priority);
getIncreaseRequests(NodeId nodeId, SchedulerRequestKey schedulerKey) {
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
return requestsOnNode == null ? null : requestsOnNode.get(
schedulerKey);
}
/**
@ -175,15 +172,17 @@ public synchronized boolean updateIncreaseRequests(
}
NodeId nodeId = r.getRMContainer().getAllocatedNode();
Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
containerIncreaseRequestMap.get(nodeId);
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
if (null == requestsOnNode) {
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 synchronized boolean updateIncreaseRequests(
}
// 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 synchronized boolean updateIncreaseRequests(
*/
private void insertIncreaseRequest(SchedContainerChangeRequest request) {
NodeId nodeId = request.getNodeId();
Priority priority = request.getPriority();
SchedulerRequestKey schedulerKey =
request.getRMContainer().getAllocatedSchedulerKey();
ContainerId containerId = request.getContainerId();
Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
containerIncreaseRequestMap.get(nodeId);
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
if (null == requestsOnNode) {
requestsOnNode = new HashMap<>();
containerIncreaseRequestMap.put(nodeId, requestsOnNode);
}
Map<ContainerId, SchedContainerChangeRequest> 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 @@ private void insertIncreaseRequest(SchedContainerChangeRequest request) {
+ " delta=" + delta);
}
// update priorities
priorities.add(priority);
// update Scheduler Keys
schedulerKeys.add(schedulerKey);
}
public synchronized boolean removeIncreaseRequest(NodeId nodeId, Priority priority,
ContainerId containerId) {
Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
containerIncreaseRequestMap.get(nodeId);
public synchronized boolean removeIncreaseRequest(NodeId nodeId,
SchedulerRequestKey schedulerKey, ContainerId containerId) {
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
if (null == requestsOnNode) {
return false;
}
Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
requestsOnNode.get(priority);
requestsOnNode.get(schedulerKey);
if (null == requestsOnNodeWithPriority) {
return false;
}
@ -272,7 +273,7 @@ public synchronized boolean removeIncreaseRequest(NodeId nodeId, Priority priori
// 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 synchronized boolean removeIncreaseRequest(NodeId nodeId, Priority priori
}
public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
Priority priority, ContainerId containerId) {
Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
containerIncreaseRequestMap.get(nodeId);
SchedulerRequestKey schedulerKey, ContainerId containerId) {
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
if (null == requestsOnNode) {
return null;
}
Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
requestsOnNode.get(priority);
requestsOnNode.get(schedulerKey);
return requestsOnNodeWithPriority == null ? null
: requestsOnNodeWithPriority.get(containerId);
}
@ -328,17 +329,18 @@ public synchronized boolean updateResourceRequests(
// 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<String, ResourceRequest> asks = this.resourceRequestMap.get(priority);
Map<String, ResourceRequest> 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 @@ private void updatePendingResources(ResourceRequest lastRequest,
}
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 @@ private void updateNodeLabels(ResourceRequest request) {
if ((null == previousAnyRequest)
|| hasRequestLabelChanged(previousAnyRequest, request)) {
Map<String, ResourceRequest> resourceRequest =
getResourceRequests(priority);
getResourceRequests(schedulerKey);
if (resourceRequest != null) {
for (ResourceRequest r : resourceRequest.values()) {
if (!r.getResourceName().equals(ResourceRequest.ANY)) {
@ -428,7 +430,7 @@ private void updateNodeLabels(ResourceRequest request) {
}
} else {
ResourceRequest anyRequest =
getResourceRequest(priority, ResourceRequest.ANY);
getResourceRequest(schedulerKey, ResourceRequest.ANY);
if (anyRequest != null) {
request.setNodeLabelExpression(anyRequest.getNodeLabelExpression());
}
@ -501,13 +503,13 @@ public boolean getAndResetBlacklistChanged() {
return userBlacklistChanged.getAndSet(false);
}
public synchronized Collection<Priority> getPriorities() {
return priorities;
public synchronized Collection<SchedulerRequestKey> getSchedulerKeys() {
return schedulerKeys;
}
public synchronized Map<String, ResourceRequest> getResourceRequests(
Priority priority) {
return resourceRequestMap.get(priority);
SchedulerRequestKey schedulerKey) {
return resourceRequestMap.get(schedulerKey);
}
public synchronized List<ResourceRequest> getAllResourceRequests() {
@ -518,14 +520,16 @@ public synchronized List<ResourceRequest> getAllResourceRequests() {
return ret;
}
public synchronized ResourceRequest getResourceRequest(Priority priority,
String resourceName) {
Map<String, ResourceRequest> nodeRequests = resourceRequestMap.get(priority);
public synchronized ResourceRequest getResourceRequest(
SchedulerRequestKey schedulerKey, String resourceName) {
Map<String, ResourceRequest> nodeRequests =
resourceRequestMap.get(schedulerKey);
return (nodeRequests == null) ? null : nodeRequests.get(resourceName);
}
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 boolean isPlaceBlacklisted(String resourceName,
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 synchronized void increaseContainer(
// 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);
}
@ -595,15 +600,21 @@ public synchronized void decreaseContainer(
/**
* 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<ResourceRequest> allocate(NodeType type,
SchedulerNode node, Priority priority, ResourceRequest request,
Container containerAllocated) {
SchedulerNode node, SchedulerRequestKey schedulerKey,
ResourceRequest request, Container containerAllocated) {
List<ResourceRequest> resourceRequests = new ArrayList<>();
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 synchronized List<ResourceRequest> allocate(NodeType type,
* application.
*/
private synchronized void allocateNodeLocal(SchedulerNode node,
Priority priority, ResourceRequest nodeLocalRequest,
SchedulerRequestKey schedulerKey, ResourceRequest nodeLocalRequest,
List<ResourceRequest> 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 @@ private synchronized void allocateNodeLocal(SchedulerNode node,
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 @@ private void decResourceRequest(String resourceName, Priority priority,
* application.
*/
private synchronized void allocateRackLocal(SchedulerNode node,
Priority priority, ResourceRequest rackLocalRequest,
SchedulerRequestKey schedulerKey, ResourceRequest rackLocalRequest,
List<ResourceRequest> 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 @@ private synchronized void decrementOutstanding(
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;

View File

@ -98,10 +98,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
protected ApplicationAttemptId attemptId;
protected Map<ContainerId, RMContainer> liveContainers =
new HashMap<ContainerId, RMContainer>();
protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers =
new HashMap<Priority, Map<NodeId, RMContainer>>();
protected final Map<SchedulerRequestKey, Map<NodeId, RMContainer>>
reservedContainers = new HashMap<>();
private final Multiset<Priority> reReservations = HashMultiset.create();
private final Multiset<SchedulerRequestKey> reReservations =
HashMultiset.create();
private Resource resourceLimit = Resource.newInstance(0, 0);
private 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<Priority> schedulingOpportunities = HashMultiset.create();
Multiset<SchedulerRequestKey> 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<Priority> missedNonPartitionedRequestSchedulingOpportunity =
Multiset<SchedulerRequestKey> missedNonPartitionedReqSchedulingOpportunity =
HashMultiset.create();
// Time of the last container scheduled at the current allowed level
protected Map<Priority, Long> lastScheduledContainer =
new HashMap<Priority, Long>();
protected Map<SchedulerRequestKey, Long> lastScheduledContainer =
new HashMap<>();
protected Queue queue;
protected boolean isStopped = false;
@ -225,8 +226,9 @@ public String getUser() {
return appSchedulingInfo.getUser();
}
public Map<String, ResourceRequest> getResourceRequests(Priority priority) {
return appSchedulingInfo.getResourceRequests(priority);
public Map<String, ResourceRequest> getResourceRequests(
SchedulerRequestKey schedulerKey) {
return appSchedulingInfo.getResourceRequests(schedulerKey);
}
public Set<ContainerId> getPendingRelease() {
@ -237,22 +239,24 @@ public long getNewContainerId() {
return appSchedulingInfo.getNewContainerId();
}
public Collection<Priority> getPriorities() {
return appSchedulingInfo.getPriorities();
public Collection<SchedulerRequestKey> getSchedulerKeys() {
return appSchedulingInfo.getSchedulerKeys();
}
public synchronized ResourceRequest getResourceRequest(Priority priority,
String resourceName) {
return this.appSchedulingInfo.getResourceRequest(priority, resourceName);
public synchronized ResourceRequest getResourceRequest(
SchedulerRequestKey schedulerKey, String resourceName) {
return appSchedulingInfo.getResourceRequest(schedulerKey, resourceName);
}
public synchronized int getTotalRequiredResources(Priority priority) {
ResourceRequest request = getResourceRequest(priority, ResourceRequest.ANY);
public synchronized int getTotalRequiredResources(
SchedulerRequestKey schedulerKey) {
ResourceRequest request =
getResourceRequest(schedulerKey, ResourceRequest.ANY);
return request == null ? 0 : request.getNumContainers();
}
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 synchronized void removeRMContainer(ContainerId containerId) {
}
}
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 synchronized boolean isStopped() {
*/
public synchronized List<RMContainer> getReservedContainers() {
List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
for (Map.Entry<Priority, Map<NodeId, RMContainer>> e :
for (Map.Entry<SchedulerRequestKey, Map<NodeId, RMContainer>> e :
this.reservedContainers.entrySet()) {
reservedContainers.addAll(e.getValue().values());
}
@ -374,8 +380,9 @@ public synchronized List<RMContainer> getReservedContainers() {
}
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 synchronized boolean reserveIncreasedContainer(SchedulerNode node,
}
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 @@ private synchronized boolean commonReserve(SchedulerNode node,
}
Map<NodeId, RMContainer> reservedContainers =
this.reservedContainers.get(priority);
this.reservedContainers.get(schedulerKey);
if (reservedContainers == null) {
reservedContainers = new HashMap<NodeId, RMContainer>();
this.reservedContainers.put(priority, reservedContainers);
this.reservedContainers.put(schedulerKey, reservedContainers);
}
reservedContainers.put(node.getNodeID(), rmContainer);
@ -408,7 +416,7 @@ private synchronized boolean commonReserve(SchedulerNode node,
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 @@ private synchronized boolean commonReserve(SchedulerNode node,
}
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 synchronized RMContainer reserve(SchedulerNode node,
((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 synchronized RMContainer reserve(SchedulerNode node,
* Has the application reserved the given <code>node</code> at the
* given <code>priority</code>?
* @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<NodeId, RMContainer> reservedContainers =
this.reservedContainers.get(priority);
this.reservedContainers.get(schedulerKey);
if (reservedContainers != null) {
return reservedContainers.containsKey(node.getNodeID());
}
@ -471,9 +481,10 @@ public synchronized Resource getHeadroom() {
return resourceLimit;
}
public synchronized int getNumReservedContainers(Priority priority) {
public synchronized int getNumReservedContainers(
SchedulerRequestKey schedulerKey) {
Map<NodeId, RMContainer> reservedContainers =
this.reservedContainers.get(priority);
this.reservedContainers.get(schedulerKey);
return (reservedContainers == null) ? 0 : reservedContainers.size();
}
@ -495,8 +506,9 @@ public synchronized void containerLaunchedOnNode(ContainerId containerId,
public synchronized void showRequests() {
if (LOG.isDebugEnabled()) {
for (Priority priority : getPriorities()) {
Map<String, ResourceRequest> requests = getResourceRequests(priority);
for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) {
Map<String, ResourceRequest> requests =
getResourceRequests(schedulerKey);
if (requests != null) {
LOG.debug("showRequests:" + " application=" + getApplicationId()
+ " headRoom=" + getHeadroom() + " currentConsumption="
@ -635,59 +647,66 @@ public boolean isPlaceBlacklisted(String resourceName) {
}
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 synchronized Resource getResourceLimit() {
return this.resourceLimit;
}
public synchronized Map<Priority, Long> getLastScheduledContainer() {
public synchronized Map<SchedulerRequestKey, Long>
getLastScheduledContainer() {
return this.lastScheduledContainer;
}
@ -892,8 +912,8 @@ public ResourceUsage getSchedulingResourceUsage() {
}
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);
}

View File

@ -31,7 +31,6 @@
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 @@ private synchronized void deductUnallocatedResource(Resource resource) {
/**
* 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.

View File

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

View File

@ -1265,7 +1265,8 @@ public void unreserveIncreasedContainer(Resource clusterResource,
}
if (null != priority) {
removed = app.unreserve(rmContainer.getContainer().getPriority(), node,
removed = app.unreserve(
rmContainer.getAllocatedSchedulerKey(), node,
rmContainer);
}
@ -1321,7 +1322,7 @@ public void completedContainer(Resource clusterResource,
// 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 void completedContainer(Resource clusterResource,
// 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 void decreaseContainer(Resource clusterResource,
// 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"

View File

@ -29,7 +29,6 @@
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.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 @@ private CSAssignment allocateIncreaseRequestFromReservedContainer(
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 @@ private CSAssignment allocateIncreaseRequest(FiCaSchedulerNode node,
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 CSAssignment assignContainers(Resource clusterResource,
* 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 CSAssignment assignContainers(Resource clusterResource,
* cannot be allocated.
*/
Map<ContainerId, SchedContainerChangeRequest> 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 CSAssignment assignContainers(Resource clusterResource,
// 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 CSAssignment assignContainers(Resource clusterResource,
// 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

View File

@ -23,7 +23,6 @@
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.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 @@ private boolean checkHeadroom(Resource clusterResource,
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 @@ private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
}
ResourceRequest anyRequest =
application.getResourceRequest(priority, ResourceRequest.ANY);
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
if (null == anyRequest) {
return ContainerAllocation.PRIORITY_SKIPPED;
}
@ -97,7 +100,7 @@ private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
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 @@ private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
}
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 @@ private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
}
// 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 @@ private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
if (anyRequest.getNodeLabelExpression()
.equals(RMNodeLabelsManager.NO_LABEL)) {
missedNonPartitionedRequestSchedulingOpportunity =
application
.addMissedNonPartitionedRequestSchedulingOpportunity(priority);
application.addMissedNonPartitionedRequestSchedulingOpportunity(
schedulerKey);
}
if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
@ -164,7 +167,7 @@ private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
.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 @@ private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
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 @@ ContainerAllocation preAllocation(Resource clusterResource,
// 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 @@ ContainerAllocation preAllocation(Resource clusterResource,
}
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 @@ private int getActualNodeLocalityDelay() {
.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 @@ private boolean canAssign(Priority priority, FiCaSchedulerNode node,
// '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 @@ private boolean canAssign(Priority priority, FiCaSchedulerNode node,
// 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 @@ private boolean canAssign(Priority priority, FiCaSchedulerNode node,
// 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 @@ private boolean canAssign(Priority priority, FiCaSchedulerNode node,
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 @@ private boolean canAssign(Priority priority, FiCaSchedulerNode node,
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 @@ private ContainerAllocation assignNodeLocalContainers(
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 @@ private ContainerAllocation assignRackLocalContainers(
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 @@ private ContainerAllocation assignOffSwitchContainers(
}
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 @@ private ContainerAllocation assignContainersOnNode(Resource clusterResource,
// 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 @@ private ContainerAllocation assignContainersOnNode(Resource clusterResource,
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 @@ private ContainerAllocation assignContainersOnNode(Resource clusterResource,
// 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 @@ private ContainerAllocation assignContainersOnNode(Resource clusterResource,
allocation =
assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
node, priority, reservedContainer, schedulingMode,
node, schedulerKey, reservedContainer, schedulingMode,
currentResoureLimits);
allocation.requestNodeType = requestType;
@ -403,21 +412,22 @@ private ContainerAllocation assignContainersOnNode(Resource clusterResource,
}
private ContainerAllocation assignContainer(Resource clusterResource,
FiCaSchedulerNode node, Priority priority, ResourceRequest request,
NodeType type, RMContainer rmContainer, SchedulingMode schedulingMode,
ResourceLimits currentResoureLimits) {
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
ResourceRequest request, NodeType type, RMContainer rmContainer,
SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
lastResourceRequest = request;
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 @@ private ContainerAllocation assignContainer(Resource clusterResource,
}
boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
priority, capability);
schedulerKey, capability);
// Can we allocate a container on this node?
long availableContainers =
@ -504,8 +514,8 @@ private ContainerAllocation assignContainer(Resource clusterResource,
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 @@ private ContainerAllocation assignContainer(Resource clusterResource,
}
}
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 @@ private ContainerAllocation assignContainer(Resource clusterResource,
}
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 @@ private Container createContainer(FiCaSchedulerNode node, Resource capability,
// 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 @@ private ContainerAllocation handleNewContainerAllocation(
}
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 @@ ContainerAllocation doAllocation(ContainerAllocation allocationResult,
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 @@ ContainerAllocation doAllocation(ContainerAllocation allocationResult,
// 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 @@ ContainerAllocation doAllocation(ContainerAllocation allocationResult,
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 CSAssignment assignContainers(Resource clusterResource,
}
// 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 CSAssignment assignContainers(Resource clusterResource,
} else {
ContainerAllocation result =
allocate(clusterResource, node, schedulingMode, resourceLimits,
reservedContainer.getReservedPriority(), reservedContainer);
reservedContainer.getReservedSchedulerKey(), reservedContainer);
return getCSAssignmentFromAllocateResult(clusterResource, result,
reservedContainer);
}

View File

@ -55,6 +55,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.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 synchronized boolean containerCompleted(RMContainer rmContainer,
}
public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
Priority priority, ResourceRequest request,
SchedulerRequestKey schedulerKey, ResourceRequest request,
Container container) {
if (isStopped) {
@ -190,7 +191,7 @@ public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
// 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;
}
@ -211,7 +212,7 @@ public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
// Update consumption and track allocations
List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
type, node, priority, request, container);
type, node, schedulerKey, request, container);
attemptResourceUsage.incUsed(node.getPartition(), container.getResource());
@ -235,13 +236,13 @@ public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
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 synchronized boolean unreserve(Priority priority,
return false;
}
private boolean internalUnreserve(FiCaSchedulerNode node, Priority priority) {
private boolean internalUnreserve(FiCaSchedulerNode node,
SchedulerRequestKey schedulerKey) {
Map<NodeId, RMContainer> 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 @@ private boolean internalUnreserve(FiCaSchedulerNode node, Priority priority) {
&& 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 @@ private boolean internalUnreserve(FiCaSchedulerNode node, Priority priority) {
}
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 synchronized Allocation getAllocation(ResourceCalculator rc,
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<NodeId, RMContainer> reservedContainers = this.reservedContainers
.get(priority);
.get(schedulerKey);
if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
for (Map.Entry<NodeId, RMContainer> entry : reservedContainers.entrySet()) {
@ -417,17 +421,17 @@ public synchronized void transferStateFromPreviousAttempt(
((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 boolean reserveIncreasedContainer(Priority priority,
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 void reserve(Priority priority,
}
// 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()) {

View File

@ -23,12 +23,13 @@
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 FiCaSchedulerNode(RMNode node, boolean usePortForNodeName) {
@Override
public synchronized void reserveResource(
SchedulerApplicationAttempt application, Priority priority,
SchedulerApplicationAttempt application, SchedulerRequestKey priority,
RMContainer container) {
// Check if it's already reserved
RMContainer reservedContainer = getReservedContainer();

View File

@ -55,6 +55,7 @@
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<Priority, NodeType> allowedLocalityLevel =
new HashMap<Priority, NodeType>();
private final Map<SchedulerRequestKey, NodeType> allowedLocalityLevel =
new HashMap<>();
public FSAppAttempt(FairScheduler scheduler,
ApplicationAttemptId applicationAttemptId, String user, FSLeafQueue queue,
@ -163,23 +164,23 @@ synchronized public void containerCompleted(RMContainer rmContainer,
}
private synchronized void unreserveInternal(
Priority priority, FSSchedulerNode node) {
SchedulerRequestKey schedulerKey, FSSchedulerNode node) {
Map<NodeId, RMContainer> 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 Resource getHeadroom() {
}
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 synchronized float getLocalityWaitFactor(
* 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 synchronized NodeType getAllowedLocalityLevel(Priority priority,
}
// 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 synchronized NodeType getAllowedLocalityLevel(Priority priority,
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 synchronized NodeType getAllowedLocalityLevelByTime(Priority priority,
}
// 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 synchronized NodeType getAllowedLocalityLevelByTime(Priority priority,
// 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 synchronized NodeType getAllowedLocalityLevelByTime(Priority priority,
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 @@ else if (allowed.equals(NodeType.RACK_LOCAL) &&
// Update consumption and track allocations
List<ResourceRequest> 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 @@ else if (allowed.equals(NodeType.RACK_LOCAL) &&
* 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 void clearPreemptedResources() {
* 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 Container createContainer(
// 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 Container createContainer(
* 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 @@ private boolean reservationExceedsThreshold(FSSchedulerNode node,
}
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 @@ int getNumReservations(String rackName, boolean isAny) {
*/
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 @@ private Resource assignContainer(
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 @@ private Resource assignContainer(
// 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 @@ private boolean isReservable(Resource capacity) {
getQueue().getPolicy().getResourceCalculator(), capacity);
}
private boolean hasNodeOrRackLocalRequests(Priority priority) {
return getResourceRequests(priority).size() > 1;
private boolean hasNodeOrRackLocalRequests(SchedulerRequestKey schedulerKey) {
return getResourceRequests(schedulerKey).size() > 1;
}
/**
@ -707,26 +729,26 @@ private Resource assignContainer(FSSchedulerNode node, boolean reserved) {
LOG.debug("Node offered to app: " + getName() + " reserved: " + reserved);
}
Collection<Priority> prioritiesToTry = (reserved) ?
Arrays.asList(node.getReservedContainer().getReservedPriority()) :
getPriorities();
Collection<SchedulerRequestKey> keysToTry = (reserved) ?
Arrays.asList(node.getReservedContainer().getReservedSchedulerKey()) :
getSchedulerKeys();
// For each priority, see if we can schedule a node local, rack local
// 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 @@ private Resource assignContainer(FSSchedulerNode node, boolean reserved) {
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 @@ private Resource assignContainer(FSSchedulerNode node, boolean reserved) {
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 @@ private Resource assignContainer(FSSchedulerNode node, boolean reserved) {
&& (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 @@ private Resource assignContainer(FSSchedulerNode node, boolean reserved) {
* 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 @@ private boolean hasContainerForNode(Priority prio, FSSchedulerNode node) {
}
private boolean isValidReservation(FSSchedulerNode node) {
Priority reservedPriority = node.getReservedContainer().
getReservedPriority();
return hasContainerForNode(reservedPriority, node) &&
SchedulerRequestKey schedulerKey = node.getReservedContainer().
getReservedSchedulerKey();
return hasContainerForNode(schedulerKey, node) &&
!isOverAMShareLimit();
}
@ -830,13 +854,14 @@ private boolean isValidReservation(FSSchedulerNode node) {
*/
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 void updateDemand() {
// 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());

View File

@ -23,10 +23,10 @@
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 FSSchedulerNode(RMNode node, boolean usePortForNodeName) {
@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 synchronized void unreserveResource(
public synchronized FSAppAttempt getReservedAppSchedulable() {
return reservedAppSchedulable;
}
}

View File

@ -866,7 +866,7 @@ protected synchronized void completedContainerInternal(
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);

View File

@ -81,6 +81,7 @@
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 @@ private void assignContainers(FiCaSchedulerNode node) {
continue;
}
for (Priority priority : application.getPriorities()) {
for (SchedulerRequestKey schedulerKey :
application.getSchedulerKeys()) {
int maxContainers =
getMaxAllocatableContainers(application, priority, node,
NodeType.OFF_SWITCH);
getMaxAllocatableContainers(application, schedulerKey, node,
NodeType.OFF_SWITCH);
// Ensure the application needs containers of this priority
if (maxContainers > 0) {
int assignedContainers =
assignContainersOnNode(node, application, priority);
assignContainersOnNode(node, application, schedulerKey);
// Do not assign out of order w.r.t priorities
if (assignedContainers == 0) {
break;
@ -553,11 +555,11 @@ private void assignContainers(FiCaSchedulerNode node) {
}
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);
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
if (offSwitchRequest != null) {
maxContainers = offSwitchRequest.getNumContainers();
}
@ -568,7 +570,8 @@ private int getMaxAllocatableContainers(FiCaSchedulerApp application,
if (type == NodeType.RACK_LOCAL) {
ResourceRequest rackLocalRequest =
application.getResourceRequest(priority, node.getRMNode().getRackName());
application.getResourceRequest(schedulerKey, node.getRMNode()
.getRackName());
if (rackLocalRequest == null) {
return maxContainers;
}
@ -578,7 +581,8 @@ private int getMaxAllocatableContainers(FiCaSchedulerApp application,
if (type == NodeType.NODE_LOCAL) {
ResourceRequest nodeLocalRequest =
application.getResourceRequest(priority, node.getRMNode().getNodeAddress());
application.getResourceRequest(schedulerKey, node.getRMNode()
.getNodeAddress());
if (nodeLocalRequest != null) {
maxContainers = Math.min(maxContainers, nodeLocalRequest.getNumContainers());
}
@ -589,25 +593,25 @@ private int getMaxAllocatableContainers(FiCaSchedulerApp application,
private int assignContainersOnNode(FiCaSchedulerNode node,
FiCaSchedulerApp application, Priority priority
FiCaSchedulerApp application, SchedulerRequestKey schedulerKey
) {
// Data-local
int nodeLocalContainers =
assignNodeLocalContainers(node, application, priority);
assignNodeLocalContainers(node, application, schedulerKey);
// Rack-local
int rackLocalContainers =
assignRackLocalContainers(node, application, priority);
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 @@ private int assignContainersOnNode(FiCaSchedulerNode node,
}
private int assignNodeLocalContainers(FiCaSchedulerNode node,
FiCaSchedulerApp application, Priority priority) {
FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) {
int assignedContainers = 0;
ResourceRequest request =
application.getResourceRequest(priority, node.getNodeName());
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 @@ private int assignNodeLocalContainers(FiCaSchedulerNode node,
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());
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);
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 @@ private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application
// 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);

View File

@ -60,6 +60,12 @@
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;
@ -79,24 +85,20 @@ public class Application {
final private ResourceManager resourceManager;
private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
final private Map<Priority, Resource> requestSpec =
new TreeMap<Priority, Resource>(
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
final private Map<SchedulerRequestKey, Resource> requestSpec =
new TreeMap<>();
final private Map<Priority, Map<String, ResourceRequest>> requests =
new TreeMap<Priority, Map<String, ResourceRequest>>(
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
final private Map<SchedulerRequestKey, Map<String, ResourceRequest>>
requests = new TreeMap<>();
final Map<Priority, Set<Task>> tasks =
new TreeMap<Priority, Set<Task>>(
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
final Map<SchedulerRequestKey, Set<Task>> tasks = new TreeMap<>();
final private Set<ResourceRequest> ask =
new TreeSet<ResourceRequest>(
new org.apache.hadoop.yarn.api.records.ResourceRequest.ResourceRequestComparator());
new TreeSet<>(
new org.apache.hadoop.yarn.api.records.ResourceRequest
.ResourceRequestComparator());
final private Map<String, NodeManager> nodes =
new HashMap<String, NodeManager>();
final private Map<String, NodeManager> nodes = new HashMap<>();
Resource used = recordFactory.newRecordInstance(Resource.class);
@ -191,10 +193,16 @@ public synchronized void submit() throws IOException, YarnException {
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 @@ private synchronized NodeManager getNodeManager(String host) {
}
public synchronized void addTask(Task task) {
Priority priority = task.getPriority();
Map<String, ResourceRequest> requests = this.requests.get(priority);
SchedulerRequestKey schedulerKey = task.getSchedulerKey();
Map<String, ResourceRequest> requests = this.requests.get(schedulerKey);
if (requests == null) {
requests = new HashMap<String, ResourceRequest>();
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<Task> tasks = this.tasks.get(priority);
Set<Task> tasks = this.tasks.get(schedulerKey);
if (tasks == null) {
tasks = new HashSet<Task>();
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 synchronized void addTask(Task task) {
// 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<Task> tasks = this.tasks.get(task.getPriority());
Set<Task> tasks = this.tasks.get(task.getSchedulerKey());
if (!tasks.remove(task)) {
throw new IllegalStateException(
"Finishing unknown task " + task.getTaskId() +
@ -270,7 +278,7 @@ public synchronized void finishTask(Task task) throws IOException,
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 synchronized void finishTask(Task task) throws IOException,
}
private synchronized void addResourceRequest(
Priority priority, Map<String, ResourceRequest> requests,
SchedulerRequestKey schedulerKey, Map<String, ResourceRequest> 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);
@ -300,12 +308,12 @@ private synchronized void addResourceRequest(
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 synchronized void assign(List<Container> containers)
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 synchronized void schedule() throws IOException, YarnException {
assign(getResources());
}
private synchronized void assign(Priority priority, NodeType type,
List<Container> containers) throws IOException, YarnException {
private synchronized void assign(SchedulerRequestKey schedulerKey,
NodeType type, List<Container> containers)
throws IOException, YarnException {
for (Iterator<Container> i=containers.iterator(); i.hasNext();) {
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<Task> t=tasks.get(priority).iterator(); t.hasNext();) {
for (Iterator<Task> t=tasks.get(schedulerKey).iterator();
t.hasNext();) {
Task task = t.next();
if (task.getState() == State.PENDING && task.canSchedule(type, host)) {
NodeManager nodeManager = getNodeManager(host);
@ -388,12 +399,13 @@ private synchronized void assign(Priority priority, NodeType type,
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 =

View File

@ -31,6 +31,9 @@
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 enum State {PENDING, ALLOCATED, RUNNING, COMPLETE};
final private ApplicationId applicationId;
final private int taskId;
final private Priority priority;
final private SchedulerRequestKey schedulerKey;
final private Set<String> hosts = new HashSet<String>();
final private Set<String> racks = new HashSet<String>();
@ -64,6 +68,7 @@ public Task(Application application, Priority priority, String[] hosts) {
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");
}
@ -76,6 +81,10 @@ public Priority getPriority() {
return priority;
}
public SchedulerRequestKey getSchedulerKey() {
return schedulerKey;
}
public org.apache.hadoop.yarn.server.resourcemanager.NodeManager getNodeManager() {
return nodeManager;
}

View File

@ -36,6 +36,8 @@
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 @@ private void mockContainers(String containersConfig, ApplicationAttemptId attemp
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);

View File

@ -36,6 +36,8 @@
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 @@ RMContainer mockContainer(ApplicationAttemptId appAttId, int id,
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);

View File

@ -120,7 +120,8 @@ public void testReleaseWhileRunning() {
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 void testExpireWhileRunning() {
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());

View File

@ -45,6 +45,8 @@
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 void testMove() {
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 void testMove() {
node.getNodeID(), prio1);
Map<NodeId, RMContainer> reservations = new HashMap<NodeId, RMContainer>();
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 @@ private RMContainer createReservedRMContainer(ApplicationAttemptId appAttId,
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 void testSchedulingOpportunityOverflow() throws Exception {
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));
}
}

View File

@ -116,6 +116,8 @@
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 void testPreemptionInfo() throws Exception {
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 void testRecoverRequestAfterPreemption() throws Exception {
// 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 void testRecoverRequestAfterPreemption() throws Exception {
// handling.
Assert.assertEquals(
1,
app.getResourceRequest(request.getPriority(),
app.getResourceRequest(SchedulerRequestKey.create(request),
request.getResourceName()).getNumContainers());
}

View File

@ -68,6 +68,9 @@
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.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 void testComputeUserLimitAndSetHeadroom(){
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 void testComputeUserLimitAndSetHeadroom(){
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 void testComputeUserLimitAndSetHeadroom(){
//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 void testReservationExchange() throws Exception {
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 void testReservationExchange() throws Exception {
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 void testReservationExchange() throws Exception {
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();
@ -1565,28 +1572,29 @@ public void testLocalityScheduling() throws Exception {
// 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 void testLocalityScheduling() throws Exception {
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 void testLocalityScheduling() throws Exception {
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 void testLocalityScheduling() throws Exception {
// 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 void testLocalityScheduling() throws Exception {
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 void testLocalityScheduling() throws Exception {
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 void testApplicationPriorityScheduling() throws Exception {
// 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 void testApplicationPriorityScheduling() throws Exception {
// 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 void testApplicationPriorityScheduling() throws Exception {
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 void testSchedulingConstraints() throws Exception {
// Setup resource-requests and submit
Priority priority = TestUtils.createMockPriority(1);
SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
app_0_requests_0.add(
TestUtils.createResourceRequest(host_0_0, 1*GB, 1,
@ -1878,7 +1895,7 @@ public void testSchedulingConstraints() throws Exception {
// 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 void testSchedulingConstraints() throws Exception {
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 void testSchedulingConstraints() throws Exception {
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 void testLocalityConstraints() throws Exception {
// host_1_1: 8G
// Blacklist: <host_0_0>
Priority priority = TestUtils.createMockPriority(1);
SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
app_0_requests_0.add(
TestUtils.createResourceRequest(host_0_0, 1*GB, 1,
@ -2169,7 +2190,8 @@ public void testLocalityConstraints() throws Exception {
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: <priority, memory, #containers, relaxLocality>
// host_0_0: < 1, 1GB, 1, true >
@ -2191,7 +2213,8 @@ public void testLocalityConstraints() throws Exception {
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 void testLocalityConstraints() throws Exception {
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 void testLocalityConstraints() throws Exception {
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 void testLocalityConstraints() throws Exception {
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 void testLocalityConstraints() throws Exception {
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 void testLocalityDelaySkipsApplication() throws Exception {
// 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<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
app_0_requests_0.add(
TestUtils.createResourceRequest(host_0, 1*GB, 1,
@ -2706,8 +2732,8 @@ public void testLocalityDelaySkipsApplication() throws Exception {
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());
}

View File

@ -32,7 +32,6 @@
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.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 @@ private void checkPendingResource(MockRM rm, int priority,
FiCaSchedulerApp app = cs.getApplicationAttempt(attemptId);
ResourceRequest rr =
app.getAppSchedulingInfo().getResourceRequest(
Priority.newInstance(priority), "*");
TestUtils.toSchedulerKey(priority), "*");
Assert.assertEquals(memory,
rr.getCapability().getMemorySize() * rr.getNumContainers());
}

View File

@ -56,6 +56,8 @@
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.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 void testReservation() throws Exception {
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 void testReservation() throws Exception {
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 void testReservation() throws Exception {
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 void testReservation() throws Exception {
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 void testReservationNoContinueLook() throws Exception {
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 void testReservationNoContinueLook() throws Exception {
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 void testReservationNoContinueLook() throws Exception {
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 void testReservationNoContinueLook() throws Exception {
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 void testAssignContainersNeedToUnreserve() throws Exception {
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 void testAssignContainersNeedToUnreserve() throws Exception {
.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 void testAssignContainersNeedToUnreserve() throws Exception {
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 void testGetAppToUnreserve() throws Exception {
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 void testGetAppToUnreserve() throws Exception {
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 void testFindNodeToUnreserve() throws Exception {
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 void testFindNodeToUnreserve() throws Exception {
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);

View File

@ -51,6 +51,8 @@
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 static FiCaSchedulerApp getFiCaSchedulerApp(MockRM rm,
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));
}
}

View File

@ -38,7 +38,10 @@
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 void setup() {
@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 void testDelayScheduling() {
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 void testDelaySchedulingForContinuousScheduling()
*/
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);

View File

@ -92,8 +92,13 @@
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 void testReservationWhileMultiplePriorities() throws IOException {
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
@ -4485,9 +4491,10 @@ public void testFairSchedulerContinuousSchedulingInitTime() throws Exception {
// time
clock.tickSec(DELAY_THRESHOLD_TIME_MS / 1000);
scheduler.attemptScheduling(node);
Map<Priority, Long> lastScheduledContainer =
Map<SchedulerRequestKey, Long> lastScheduledContainer =
fsAppAttempt.getLastScheduledContainer();
long initSchedulerTime = lastScheduledContainer.get(priority);
long initSchedulerTime =
lastScheduledContainer.get(TestUtils.toSchedulerKey(priority));
assertEquals(DELAY_THRESHOLD_TIME_MS, initSchedulerTime);
}

View File

@ -22,7 +22,6 @@
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.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 void testChoiceOfPreemptedContainers() throws Exception {
Set<RMContainer> set = new HashSet<RMContainer>();
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 void testRecoverRequestAfterPreemption() throws Exception {
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 void testRecoverRequestAfterPreemption() throws Exception {
// Create 3 container requests and place it in ask
List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
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 void testRecoverRequestAfterPreemption() throws Exception {
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 void testRecoverRequestAfterPreemption() throws Exception {
Assert.assertEquals(3, requests.size());
for (ResourceRequest request : requests) {
Assert.assertEquals(1,
app.getResourceRequest(priority, request.getResourceName())
app.getResourceRequest(schedulerKey, request.getResourceName())
.getNumContainers());
}