YARN-3361. CapacityScheduler side changes to support non-exclusive node labels. Contributed by Wangda Tan

This commit is contained in:
Jian He 2015-04-14 11:36:37 -07:00
parent b46ee1e7a3
commit 0fefda645b
25 changed files with 1913 additions and 953 deletions

View File

@ -72,6 +72,9 @@ Release 2.8.0 - UNRELEASED
YARN-3443. Create a 'ResourceHandler' subsystem to ease addition of support YARN-3443. Create a 'ResourceHandler' subsystem to ease addition of support
for new resource types on the NM. (Sidharta Seethana via junping_du) for new resource types on the NM. (Sidharta Seethana via junping_du)
YARN-3361. CapacityScheduler side changes to support non-exclusive node
labels. (Wangda Tan via jianhe)
IMPROVEMENTS IMPROVEMENTS
YARN-1880. Cleanup TestApplicationClientProtocolOnHA YARN-1880. Cleanup TestApplicationClientProtocolOnHA

View File

@ -313,6 +313,7 @@ public class BuilderUtils {
request.setResourceName(r.getResourceName()); request.setResourceName(r.getResourceName());
request.setCapability(r.getCapability()); request.setCapability(r.getCapability());
request.setNumContainers(r.getNumContainers()); request.setNumContainers(r.getNumContainers());
request.setNodeLabelExpression(r.getNodeLabelExpression());
return request; return request;
} }

View File

@ -146,7 +146,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private ConcurrentMap<NodeId, List<ContainerStatus>> private ConcurrentMap<NodeId, List<ContainerStatus>>
finishedContainersSentToAM = finishedContainersSentToAM =
new ConcurrentHashMap<NodeId, List<ContainerStatus>>(); new ConcurrentHashMap<NodeId, List<ContainerStatus>>();
private Container masterContainer; private volatile Container masterContainer;
private float progress = 0; private float progress = 0;
private String host = "N/A"; private String host = "N/A";
@ -762,13 +762,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
@Override @Override
public Container getMasterContainer() { public Container getMasterContainer() {
this.readLock.lock(); return this.masterContainer;
try {
return this.masterContainer;
} finally {
this.readLock.unlock();
}
} }
@InterfaceAudience.Private @InterfaceAudience.Private

View File

@ -73,10 +73,11 @@ public class AppSchedulingInfo {
/* Allocated by scheduler */ /* Allocated by scheduler */
boolean pending = true; // for app metrics boolean pending = true; // for app metrics
private ResourceUsage appResourceUsage;
public AppSchedulingInfo(ApplicationAttemptId appAttemptId, public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
String user, Queue queue, ActiveUsersManager activeUsersManager, String user, Queue queue, ActiveUsersManager activeUsersManager,
long epoch) { long epoch, ResourceUsage appResourceUsage) {
this.applicationAttemptId = appAttemptId; this.applicationAttemptId = appAttemptId;
this.applicationId = appAttemptId.getApplicationId(); this.applicationId = appAttemptId.getApplicationId();
this.queue = queue; this.queue = queue;
@ -84,6 +85,7 @@ public class AppSchedulingInfo {
this.user = user; this.user = user;
this.activeUsersManager = activeUsersManager; this.activeUsersManager = activeUsersManager;
this.containerIdCounter = new AtomicLong(epoch << EPOCH_BIT_SHIFT); this.containerIdCounter = new AtomicLong(epoch << EPOCH_BIT_SHIFT);
this.appResourceUsage = appResourceUsage;
} }
public ApplicationId getApplicationId() { public ApplicationId getApplicationId() {
@ -191,13 +193,19 @@ public class AppSchedulingInfo {
lastRequestCapability); lastRequestCapability);
// update queue: // update queue:
Resource increasedResource = Resources.multiply(request.getCapability(),
request.getNumContainers());
queue.incPendingResource( queue.incPendingResource(
request.getNodeLabelExpression(), request.getNodeLabelExpression(),
Resources.multiply(request.getCapability(), increasedResource);
request.getNumContainers())); appResourceUsage.incPending(request.getNodeLabelExpression(), increasedResource);
if (lastRequest != null) { if (lastRequest != null) {
Resource decreasedResource =
Resources.multiply(lastRequestCapability, lastRequestContainers);
queue.decPendingResource(lastRequest.getNodeLabelExpression(), queue.decPendingResource(lastRequest.getNodeLabelExpression(),
Resources.multiply(lastRequestCapability, lastRequestContainers)); decreasedResource);
appResourceUsage.decPending(lastRequest.getNodeLabelExpression(),
decreasedResource);
} }
} }
} }
@ -385,6 +393,8 @@ public class AppSchedulingInfo {
checkForDeactivation(); checkForDeactivation();
} }
appResourceUsage.decPending(offSwitchRequest.getNodeLabelExpression(),
offSwitchRequest.getCapability());
queue.decPendingResource(offSwitchRequest.getNodeLabelExpression(), queue.decPendingResource(offSwitchRequest.getNodeLabelExpression(),
offSwitchRequest.getCapability()); offSwitchRequest.getCapability());
} }
@ -492,9 +502,10 @@ public class AppSchedulingInfo {
} }
public ResourceRequest cloneResourceRequest(ResourceRequest request) { public ResourceRequest cloneResourceRequest(ResourceRequest request) {
ResourceRequest newRequest = ResourceRequest.newInstance( ResourceRequest newRequest =
request.getPriority(), request.getResourceName(), ResourceRequest.newInstance(request.getPriority(),
request.getCapability(), 1, request.getRelaxLocality()); request.getResourceName(), request.getCapability(), 1,
request.getRelaxLocality(), request.getNodeLabelExpression());
return newRequest; return newRequest;
} }
} }

View File

@ -27,6 +27,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
/** /**
@ -250,6 +251,10 @@ public class ResourceUsage {
} }
private Resource _get(String label, ResourceType type) { private Resource _get(String label, ResourceType type) {
if (label == null) {
label = RMNodeLabelsManager.NO_LABEL;
}
try { try {
readLock.lock(); readLock.lock();
UsageByLabel usage = usages.get(label); UsageByLabel usage = usages.get(label);
@ -263,6 +268,9 @@ public class ResourceUsage {
} }
private UsageByLabel getAndAddIfMissing(String label) { private UsageByLabel getAndAddIfMissing(String label) {
if (label == null) {
label = RMNodeLabelsManager.NO_LABEL;
}
if (!usages.containsKey(label)) { if (!usages.containsKey(label)) {
UsageByLabel u = new UsageByLabel(label); UsageByLabel u = new UsageByLabel(label);
usages.put(label, u); usages.put(label, u);

View File

@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -108,14 +110,24 @@ public class SchedulerApplicationAttempt {
private Set<ContainerId> pendingRelease = null; private Set<ContainerId> pendingRelease = null;
/** /**
* Count how many times the application has been given an opportunity * Count how many times the application has been given an opportunity to
* to schedule a task at each priority. Each time the scheduler * schedule a task at each priority. Each time the scheduler asks the
* asks the application for a task at this priority, it is incremented, * application for a task at this priority, it is incremented, and each time
* and each time the application successfully schedules a task, it * the application successfully schedules a task (at rack or node local), it
* is reset to 0. * is reset to 0.
*/ */
Multiset<Priority> schedulingOpportunities = HashMultiset.create(); Multiset<Priority> schedulingOpportunities = HashMultiset.create();
/**
* Count how many times the application has been given an opportunity to
* schedule a non-partitioned resource request at each priority. Each time the
* scheduler asks the application for a task at this priority, it is
* 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 =
HashMultiset.create();
// Time of the last container scheduled at the current allowed level // Time of the last container scheduled at the current allowed level
protected Map<Priority, Long> lastScheduledContainer = protected Map<Priority, Long> lastScheduledContainer =
new HashMap<Priority, Long>(); new HashMap<Priority, Long>();
@ -132,7 +144,7 @@ public class SchedulerApplicationAttempt {
this.rmContext = rmContext; this.rmContext = rmContext;
this.appSchedulingInfo = this.appSchedulingInfo =
new AppSchedulingInfo(applicationAttemptId, user, queue, new AppSchedulingInfo(applicationAttemptId, user, queue,
activeUsersManager, rmContext.getEpoch()); activeUsersManager, rmContext.getEpoch(), attemptResourceUsage);
this.queue = queue; this.queue = queue;
this.pendingRelease = new HashSet<ContainerId>(); this.pendingRelease = new HashSet<ContainerId>();
this.attemptId = applicationAttemptId; this.attemptId = applicationAttemptId;
@ -489,6 +501,18 @@ public class SchedulerApplicationAttempt {
return this.appSchedulingInfo.isBlacklisted(resourceName); return this.appSchedulingInfo.isBlacklisted(resourceName);
} }
public synchronized int addMissedNonPartitionedRequestSchedulingOpportunity(
Priority priority) {
missedNonPartitionedRequestSchedulingOpportunity.add(priority);
return missedNonPartitionedRequestSchedulingOpportunity.count(priority);
}
public synchronized void
resetMissedNonPartitionedRequestSchedulingOpportunity(Priority priority) {
missedNonPartitionedRequestSchedulingOpportunity.setCount(priority, 0);
}
public synchronized void addSchedulingOpportunity(Priority priority) { public synchronized void addSchedulingOpportunity(Priority priority) {
schedulingOpportunities.setCount(priority, schedulingOpportunities.setCount(priority,
schedulingOpportunities.count(priority) + 1); schedulingOpportunities.count(priority) + 1);
@ -518,6 +542,7 @@ public class SchedulerApplicationAttempt {
public synchronized void resetSchedulingOpportunities(Priority priority) { public synchronized void resetSchedulingOpportunities(Priority priority) {
resetSchedulingOpportunities(priority, System.currentTimeMillis()); resetSchedulingOpportunities(priority, System.currentTimeMillis());
} }
// used for continuous scheduling // used for continuous scheduling
public synchronized void resetSchedulingOpportunities(Priority priority, public synchronized void resetSchedulingOpportunities(Priority priority,
long currentTimeMs) { long currentTimeMs) {
@ -669,4 +694,13 @@ public class SchedulerApplicationAttempt {
public Set<String> getBlacklistedNodes() { public Set<String> getBlacklistedNodes() {
return this.appSchedulingInfo.getBlackListCopy(); return this.appSchedulingInfo.getBlackListCopy();
} }
@Private
public boolean hasPendingResourceRequest(ResourceCalculator rc,
String nodePartition, Resource cluster,
SchedulingMode schedulingMode) {
return SchedulerUtils.hasPendingResourceRequest(rc,
this.attemptResourceUsage, nodePartition, cluster,
schedulingMode);
}
} }

View File

@ -37,11 +37,10 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.AccessType; import org.apache.hadoop.yarn.security.AccessType;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.Sets;
/** /**
* Utilities shared by schedulers. * Utilities shared by schedulers.
*/ */
@ -235,9 +234,13 @@ public class SchedulerUtils {
if (labelExp == null && queueInfo != null if (labelExp == null && queueInfo != null
&& ResourceRequest.ANY.equals(resReq.getResourceName())) { && ResourceRequest.ANY.equals(resReq.getResourceName())) {
labelExp = queueInfo.getDefaultNodeLabelExpression(); labelExp = queueInfo.getDefaultNodeLabelExpression();
resReq.setNodeLabelExpression(labelExp);
} }
// If labelExp still equals to null, set it to be NO_LABEL
resReq
.setNodeLabelExpression(labelExp == null ? RMNodeLabelsManager.NO_LABEL
: labelExp);
// we don't allow specify label expression other than resourceName=ANY now // we don't allow specify label expression other than resourceName=ANY now
if (!ResourceRequest.ANY.equals(resReq.getResourceName()) if (!ResourceRequest.ANY.equals(resReq.getResourceName())
&& labelExp != null && !labelExp.trim().isEmpty()) { && labelExp != null && !labelExp.trim().isEmpty()) {
@ -273,25 +276,6 @@ public class SchedulerUtils {
} }
} }
public static boolean checkQueueAccessToNode(Set<String> queueLabels,
Set<String> nodeLabels) {
// if queue's label is *, it can access any node
if (queueLabels != null && queueLabels.contains(RMNodeLabelsManager.ANY)) {
return true;
}
// any queue can access to a node without label
if (nodeLabels == null || nodeLabels.isEmpty()) {
return true;
}
// a queue can access to a node only if it contains any label of the node
if (queueLabels != null
&& Sets.intersection(queueLabels, nodeLabels).size() > 0) {
return true;
}
// sorry, you cannot access
return false;
}
public static void checkIfLabelInClusterNodeLabels(RMNodeLabelsManager mgr, public static void checkIfLabelInClusterNodeLabels(RMNodeLabelsManager mgr,
Set<String> labels) throws IOException { Set<String> labels) throws IOException {
if (mgr == null) { if (mgr == null) {
@ -311,26 +295,6 @@ public class SchedulerUtils {
} }
} }
} }
public static boolean checkNodeLabelExpression(Set<String> nodeLabels,
String labelExpression) {
// empty label expression can only allocate on node with empty labels
if (labelExpression == null || labelExpression.trim().isEmpty()) {
if (!nodeLabels.isEmpty()) {
return false;
}
}
if (labelExpression != null) {
for (String str : labelExpression.split("&&")) {
if (!str.trim().isEmpty()
&& (nodeLabels == null || !nodeLabels.contains(str.trim()))) {
return false;
}
}
}
return true;
}
public static boolean checkQueueLabelExpression(Set<String> queueLabels, public static boolean checkQueueLabelExpression(Set<String> queueLabels,
String labelExpression) { String labelExpression) {
@ -360,4 +324,43 @@ public class SchedulerUtils {
} }
return null; return null;
} }
public static boolean checkResourceRequestMatchingNodePartition(
ResourceRequest offswitchResourceRequest, String nodePartition,
SchedulingMode schedulingMode) {
// We will only look at node label = nodeLabelToLookAt according to
// schedulingMode and partition of node.
String nodePartitionToLookAt = null;
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
nodePartitionToLookAt = nodePartition;
} else {
nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL;
}
String askedNodePartition = offswitchResourceRequest.getNodeLabelExpression();
if (null == askedNodePartition) {
askedNodePartition = RMNodeLabelsManager.NO_LABEL;
}
return askedNodePartition.equals(nodePartitionToLookAt);
}
private static boolean hasPendingResourceRequest(ResourceCalculator rc,
ResourceUsage usage, String partitionToLookAt, Resource cluster) {
if (Resources.greaterThan(rc, cluster,
usage.getPending(partitionToLookAt), Resources.none())) {
return true;
}
return false;
}
@Private
public static boolean hasPendingResourceRequest(ResourceCalculator rc,
ResourceUsage usage, String nodePartition, Resource cluster,
SchedulingMode schedulingMode) {
String partitionToLookAt = nodePartition;
if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
partitionToLookAt = RMNodeLabelsManager.NO_LABEL;
}
return hasPendingResourceRequest(rc, usage, partitionToLookAt, cluster);
}
} }

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -38,12 +37,12 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.security.AccessType; import org.apache.hadoop.yarn.security.AccessType;
import org.apache.hadoop.yarn.security.PrivilegedEntity; import org.apache.hadoop.yarn.security.PrivilegedEntity;
import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType; import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
import org.apache.hadoop.yarn.security.YarnAuthorizationProvider; import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
@ -56,6 +55,11 @@ import com.google.common.collect.Sets;
public abstract class AbstractCSQueue implements CSQueue { public abstract class AbstractCSQueue implements CSQueue {
private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class); private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class);
static final CSAssignment NULL_ASSIGNMENT =
new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
static final CSAssignment SKIP_ASSIGNMENT = new CSAssignment(true);
CSQueue parent; CSQueue parent;
final String queueName; final String queueName;
volatile int numContainers; volatile int numContainers;
@ -343,16 +347,8 @@ public abstract class AbstractCSQueue implements CSQueue {
} }
synchronized void allocateResource(Resource clusterResource, synchronized void allocateResource(Resource clusterResource,
Resource resource, Set<String> nodeLabels) { Resource resource, String nodePartition) {
queueUsage.incUsed(nodePartition, resource);
// Update usedResources by labels
if (nodeLabels == null || nodeLabels.isEmpty()) {
queueUsage.incUsed(resource);
} else {
for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
queueUsage.incUsed(label, resource);
}
}
++numContainers; ++numContainers;
CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(), CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
@ -360,15 +356,8 @@ public abstract class AbstractCSQueue implements CSQueue {
} }
protected synchronized void releaseResource(Resource clusterResource, protected synchronized void releaseResource(Resource clusterResource,
Resource resource, Set<String> nodeLabels) { Resource resource, String nodePartition) {
// Update usedResources by labels queueUsage.decUsed(nodePartition, resource);
if (null == nodeLabels || nodeLabels.isEmpty()) {
queueUsage.decUsed(resource);
} else {
for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
queueUsage.decUsed(label, resource);
}
}
CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(), CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
clusterResource, minimumAllocation); clusterResource, minimumAllocation);
@ -434,103 +423,108 @@ public abstract class AbstractCSQueue implements CSQueue {
parentQ.getPreemptionDisabled()); parentQ.getPreemptionDisabled());
} }
private Resource getCurrentLimitResource(String nodeLabel, private Resource getCurrentLimitResource(String nodePartition,
Resource clusterResource, ResourceLimits currentResourceLimits) { Resource clusterResource, ResourceLimits currentResourceLimits,
/* SchedulingMode schedulingMode) {
* Current limit resource: For labeled resource: limit = queue-max-resource if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
* (TODO, this part need update when we support labeled-limit) For /*
* non-labeled resource: limit = min(queue-max-resource, * Current limit resource: For labeled resource: limit = queue-max-resource
* limit-set-by-parent) * (TODO, this part need update when we support labeled-limit) For
*/ * non-labeled resource: limit = min(queue-max-resource,
Resource queueMaxResource = * limit-set-by-parent)
Resources.multiplyAndNormalizeDown(resourceCalculator, */
labelManager.getResourceByLabel(nodeLabel, clusterResource), Resource queueMaxResource =
queueCapacities.getAbsoluteMaximumCapacity(nodeLabel), minimumAllocation); Resources.multiplyAndNormalizeDown(resourceCalculator,
if (nodeLabel.equals(RMNodeLabelsManager.NO_LABEL)) { labelManager.getResourceByLabel(nodePartition, clusterResource),
return Resources.min(resourceCalculator, clusterResource, queueCapacities.getAbsoluteMaximumCapacity(nodePartition), minimumAllocation);
queueMaxResource, currentResourceLimits.getLimit()); if (nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
return Resources.min(resourceCalculator, clusterResource,
queueMaxResource, currentResourceLimits.getLimit());
}
return queueMaxResource;
} else if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
// When we doing non-exclusive resource allocation, maximum capacity of
// all queues on this label equals to total resource with the label.
return labelManager.getResourceByLabel(nodePartition, clusterResource);
} }
return queueMaxResource;
return Resources.none();
} }
synchronized boolean canAssignToThisQueue(Resource clusterResource, synchronized boolean canAssignToThisQueue(Resource clusterResource,
Set<String> nodeLabels, ResourceLimits currentResourceLimits, String nodePartition, ResourceLimits currentResourceLimits,
Resource nowRequired, Resource resourceCouldBeUnreserved) { Resource nowRequired, Resource resourceCouldBeUnreserved,
// Get label of this queue can access, it's (nodeLabel AND queueLabel) SchedulingMode schedulingMode) {
Set<String> labelCanAccess; // New total resource = used + required
if (null == nodeLabels || nodeLabels.isEmpty()) { Resource newTotalResource =
labelCanAccess = new HashSet<String>(); Resources.add(queueUsage.getUsed(nodePartition), nowRequired);
// Any queue can always access any node without label
labelCanAccess.add(RMNodeLabelsManager.NO_LABEL);
} else {
labelCanAccess = new HashSet<String>(
accessibleLabels.contains(CommonNodeLabelsManager.ANY) ? nodeLabels
: Sets.intersection(accessibleLabels, nodeLabels));
}
for (String label : labelCanAccess) {
// New total resource = used + required
Resource newTotalResource =
Resources.add(queueUsage.getUsed(label), nowRequired);
Resource currentLimitResource = // Get current limited resource:
getCurrentLimitResource(label, clusterResource, currentResourceLimits); // - When doing RESPECT_PARTITION_EXCLUSIVITY allocation, we will respect
// queues' max capacity.
// - When doing IGNORE_PARTITION_EXCLUSIVITY allocation, we will not respect
// queue's max capacity, queue's max capacity on the partition will be
// considered to be 100%. Which is a queue can use all resource in the
// partition.
// Doing this because: for non-exclusive allocation, we make sure there's
// idle resource on the partition, to avoid wastage, such resource will be
// leveraged as much as we can, and preemption policy will reclaim it back
// when partitoned-resource-request comes back.
Resource currentLimitResource =
getCurrentLimitResource(nodePartition, clusterResource,
currentResourceLimits, schedulingMode);
// if reservation continous looking enabled, check to see if could we // if reservation continous looking enabled, check to see if could we
// potentially use this node instead of a reserved node if the application // potentially use this node instead of a reserved node if the application
// has reserved containers. // has reserved containers.
// TODO, now only consider reservation cases when the node has no label // TODO, now only consider reservation cases when the node has no label
if (this.reservationsContinueLooking if (this.reservationsContinueLooking
&& label.equals(RMNodeLabelsManager.NO_LABEL) && nodePartition.equals(RMNodeLabelsManager.NO_LABEL)
&& Resources.greaterThan(resourceCalculator, clusterResource, && Resources.greaterThan(resourceCalculator, clusterResource,
resourceCouldBeUnreserved, Resources.none())) { resourceCouldBeUnreserved, Resources.none())) {
// resource-without-reserved = used - reserved // resource-without-reserved = used - reserved
Resource newTotalWithoutReservedResource = Resource newTotalWithoutReservedResource =
Resources.subtract(newTotalResource, resourceCouldBeUnreserved); Resources.subtract(newTotalResource, resourceCouldBeUnreserved);
// when total-used-without-reserved-resource < currentLimit, we still // when total-used-without-reserved-resource < currentLimit, we still
// have chance to allocate on this node by unreserving some containers // have chance to allocate on this node by unreserving some containers
if (Resources.lessThan(resourceCalculator, clusterResource, if (Resources.lessThan(resourceCalculator, clusterResource,
newTotalWithoutReservedResource, currentLimitResource)) { newTotalWithoutReservedResource, currentLimitResource)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("try to use reserved: " + getQueueName() LOG.debug("try to use reserved: " + getQueueName()
+ " usedResources: " + queueUsage.getUsed() + " usedResources: " + queueUsage.getUsed()
+ ", clusterResources: " + clusterResource + ", clusterResources: " + clusterResource
+ ", reservedResources: " + resourceCouldBeUnreserved + ", reservedResources: " + resourceCouldBeUnreserved
+ ", capacity-without-reserved: " + ", capacity-without-reserved: "
+ newTotalWithoutReservedResource + ", maxLimitCapacity: " + newTotalWithoutReservedResource + ", maxLimitCapacity: "
+ currentLimitResource); + currentLimitResource);
}
return true;
} }
return true;
} }
// Otherwise, if any of the label of this node beyond queue limit, we
// cannot allocate on this node. Consider a small epsilon here.
if (Resources.greaterThan(resourceCalculator, clusterResource,
newTotalResource, currentLimitResource)) {
return false;
}
if (LOG.isDebugEnabled()) {
LOG.debug(getQueueName()
+ "Check assign to queue, label=" + label
+ " usedResources: " + queueUsage.getUsed(label)
+ " clusterResources: " + clusterResource
+ " currentUsedCapacity "
+ Resources.divide(resourceCalculator, clusterResource,
queueUsage.getUsed(label),
labelManager.getResourceByLabel(label, clusterResource))
+ " max-capacity: "
+ queueCapacities.getAbsoluteMaximumCapacity(label)
+ ")");
}
return true;
} }
// Actually, this will not happen, since labelCanAccess will be always // Check if we over current-resource-limit computed.
// non-empty if (Resources.greaterThan(resourceCalculator, clusterResource,
return false; newTotalResource, currentLimitResource)) {
return false;
}
if (LOG.isDebugEnabled()) {
LOG.debug(getQueueName()
+ "Check assign to queue, nodePartition="
+ nodePartition
+ " usedResources: "
+ queueUsage.getUsed(nodePartition)
+ " clusterResources: "
+ clusterResource
+ " currentUsedCapacity "
+ Resources.divide(resourceCalculator, clusterResource,
queueUsage.getUsed(nodePartition),
labelManager.getResourceByLabel(nodePartition, clusterResource))
+ " max-capacity: "
+ queueCapacities.getAbsoluteMaximumCapacity(nodePartition) + ")");
}
return true;
} }
@Override @Override
@ -556,4 +550,33 @@ public abstract class AbstractCSQueue implements CSQueue {
parent.decPendingResource(nodeLabel, resourceToDec); parent.decPendingResource(nodeLabel, resourceToDec);
} }
} }
/**
* Return if the queue has pending resource on given nodePartition and
* schedulingMode.
*/
boolean hasPendingResourceRequest(String nodePartition,
Resource cluster, SchedulingMode schedulingMode) {
return SchedulerUtils.hasPendingResourceRequest(resourceCalculator,
queueUsage, nodePartition, cluster, schedulingMode);
}
boolean accessibleToPartition(String nodePartition) {
// if queue's label is *, it can access any node
if (accessibleLabels != null
&& accessibleLabels.contains(RMNodeLabelsManager.ANY)) {
return true;
}
// any queue can access to a node without label
if (nodePartition == null
|| nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
return true;
}
// a queue can access to a node only if it contains any label of the node
if (accessibleLabels != null && accessibleLabels.contains(nodePartition)) {
return true;
}
// sorry, you cannot access
return false;
}
} }

View File

@ -190,10 +190,13 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
* @param clusterResource the resource of the cluster. * @param clusterResource the resource of the cluster.
* @param node node on which resources are available * @param node node on which resources are available
* @param resourceLimits how much overall resource of this queue can use. * @param resourceLimits how much overall resource of this queue can use.
* @param schedulingMode Type of exclusive check when assign container on a
* NodeManager, see {@link SchedulingMode}.
* @return the assignment * @return the assignment
*/ */
public CSAssignment assignContainers(Resource clusterResource, public CSAssignment assignContainers(Resource clusterResource,
FiCaSchedulerNode node, ResourceLimits resourceLimits); FiCaSchedulerNode node, ResourceLimits resourceLimits,
SchedulingMode schedulingMode);
/** /**
* A container assigned to the queue has completed. * A container assigned to the queue has completed.

View File

@ -35,6 +35,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
@ -1114,28 +1115,30 @@ public class CapacityScheduler extends
if (reservedContainer != null) { if (reservedContainer != null) {
FiCaSchedulerApp reservedApplication = FiCaSchedulerApp reservedApplication =
getCurrentAttemptForContainer(reservedContainer.getContainerId()); getCurrentAttemptForContainer(reservedContainer.getContainerId());
// Try to fulfill the reservation // Try to fulfill the reservation
LOG.info("Trying to fulfill reservation for application " + LOG.info("Trying to fulfill reservation for application "
reservedApplication.getApplicationId() + " on node: " + + reservedApplication.getApplicationId() + " on node: "
node.getNodeID()); + node.getNodeID());
LeafQueue queue = ((LeafQueue)reservedApplication.getQueue()); LeafQueue queue = ((LeafQueue) reservedApplication.getQueue());
assignment = queue.assignContainers( assignment =
queue.assignContainers(
clusterResource, clusterResource,
node, node,
// TODO, now we only consider limits for parent for non-labeled // TODO, now we only consider limits for parent for non-labeled
// resources, should consider labeled resources as well. // resources, should consider labeled resources as well.
new ResourceLimits(labelManager.getResourceByLabel( new ResourceLimits(labelManager.getResourceByLabel(
RMNodeLabelsManager.NO_LABEL, clusterResource))); RMNodeLabelsManager.NO_LABEL, clusterResource)),
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
if (assignment.isFulfilledReservation()) { if (assignment.isFulfilledReservation()) {
CSAssignment tmp = CSAssignment tmp =
new CSAssignment(reservedContainer.getReservedResource(), new CSAssignment(reservedContainer.getReservedResource(),
assignment.getType()); assignment.getType());
Resources.addTo(assignment.getAssignmentInformation().getAllocated(), Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
reservedContainer.getReservedResource()); reservedContainer.getReservedResource());
tmp.getAssignmentInformation().addAllocationDetails( tmp.getAssignmentInformation().addAllocationDetails(
reservedContainer.getContainerId(), queue.getQueuePath()); reservedContainer.getContainerId(), queue.getQueuePath());
tmp.getAssignmentInformation().incrAllocations(); tmp.getAssignmentInformation().incrAllocations();
updateSchedulerHealth(lastNodeUpdateTime, node, tmp); updateSchedulerHealth(lastNodeUpdateTime, node, tmp);
schedulerHealth.updateSchedulerFulfilledReservationCounts(1); schedulerHealth.updateSchedulerFulfilledReservationCounts(1);
@ -1143,16 +1146,13 @@ public class CapacityScheduler extends
RMContainer excessReservation = assignment.getExcessReservation(); RMContainer excessReservation = assignment.getExcessReservation();
if (excessReservation != null) { if (excessReservation != null) {
Container container = excessReservation.getContainer(); Container container = excessReservation.getContainer();
queue.completedContainer( queue.completedContainer(clusterResource, assignment.getApplication(),
clusterResource, assignment.getApplication(), node, node, excessReservation, SchedulerUtils
excessReservation, .createAbnormalContainerStatus(container.getId(),
SchedulerUtils.createAbnormalContainerStatus( SchedulerUtils.UNRESERVED_CONTAINER),
container.getId(), RMContainerEventType.RELEASED, null, true);
SchedulerUtils.UNRESERVED_CONTAINER),
RMContainerEventType.RELEASED, null, true);
} }
} }
// Try to schedule more if there are no reservations to fulfill // Try to schedule more if there are no reservations to fulfill
@ -1163,22 +1163,61 @@ public class CapacityScheduler extends
LOG.debug("Trying to schedule on node: " + node.getNodeName() + LOG.debug("Trying to schedule on node: " + node.getNodeName() +
", available: " + node.getAvailableResource()); ", available: " + node.getAvailableResource());
} }
assignment = root.assignContainers( assignment = root.assignContainers(
clusterResource, clusterResource,
node, node,
// TODO, now we only consider limits for parent for non-labeled // TODO, now we only consider limits for parent for non-labeled
// resources, should consider labeled resources as well. // resources, should consider labeled resources as well.
new ResourceLimits(labelManager.getResourceByLabel( new ResourceLimits(labelManager.getResourceByLabel(
RMNodeLabelsManager.NO_LABEL, clusterResource))); RMNodeLabelsManager.NO_LABEL, clusterResource)),
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
if (Resources.greaterThan(calculator, clusterResource,
assignment.getResource(), Resources.none())) {
updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
return;
}
// Only do non-exclusive allocation when node has node-labels.
if (StringUtils.equals(node.getPartition(),
RMNodeLabelsManager.NO_LABEL)) {
return;
}
// Only do non-exclusive allocation when the node-label supports that
try {
if (rmContext.getNodeLabelManager().isExclusiveNodeLabel(
node.getPartition())) {
return;
}
} catch (IOException e) {
LOG.warn("Exception when trying to get exclusivity of node label="
+ node.getPartition(), e);
return;
}
// Try to use NON_EXCLUSIVE
assignment = root.assignContainers(
clusterResource,
node,
// TODO, now we only consider limits for parent for non-labeled
// resources, should consider labeled resources as well.
new ResourceLimits(labelManager.getResourceByLabel(
RMNodeLabelsManager.NO_LABEL, clusterResource)),
SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY);
updateSchedulerHealth(lastNodeUpdateTime, node, assignment); updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
if (Resources.greaterThan(calculator, clusterResource,
assignment.getResource(), Resources.none())) {
return;
}
} }
} else { } else {
LOG.info("Skipping scheduling since node " + node.getNodeID() + LOG.info("Skipping scheduling since node "
" is reserved by application " + + node.getNodeID()
node.getReservedContainer().getContainerId().getApplicationAttemptId() + " is reserved by application "
); + node.getReservedContainer().getContainerId()
.getApplicationAttemptId());
} }
} }
@Override @Override

View File

@ -319,6 +319,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
getMaximumApplicationMasterResourcePercent()); getMaximumApplicationMasterResourcePercent());
} }
public void setMaximumApplicationMasterResourcePerQueuePercent(String queue,
float percent) {
setFloat(getQueuePrefix(queue) + MAXIMUM_AM_RESOURCE_SUFFIX, percent);
}
public float getNonLabeledQueueCapacity(String queue) { public float getNonLabeledQueueCapacity(String queue) {
float capacity = queue.equals("root") ? 100.0f : getFloat( float capacity = queue.equals("root") ? 100.0f : getFloat(
getQueuePrefix(queue) + CAPACITY, UNDEFINED); getQueuePrefix(queue) + CAPACITY, UNDEFINED);

View File

@ -24,7 +24,6 @@ import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -58,6 +57,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.security.AccessType; import org.apache.hadoop.yarn.security.AccessType;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
@ -718,39 +718,11 @@ public class LeafQueue extends AbstractCSQueue {
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId) {
return applicationAttemptMap.get(applicationAttemptId); return applicationAttemptMap.get(applicationAttemptId);
} }
private static final CSAssignment NULL_ASSIGNMENT =
new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
private static final CSAssignment SKIP_ASSIGNMENT = new CSAssignment(true);
private static Set<String> getRequestLabelSetByExpression(
String labelExpression) {
Set<String> labels = new HashSet<String>();
if (null == labelExpression) {
return labels;
}
for (String l : labelExpression.split("&&")) {
if (l.trim().isEmpty()) {
continue;
}
labels.add(l.trim());
}
return labels;
}
private boolean checkResourceRequestMatchingNodeLabel(ResourceRequest offswitchResourceRequest,
FiCaSchedulerNode node) {
String askedNodeLabel = offswitchResourceRequest.getNodeLabelExpression();
if (null == askedNodeLabel) {
askedNodeLabel = RMNodeLabelsManager.NO_LABEL;
}
return askedNodeLabel.equals(node.getPartition());
}
@Override @Override
public synchronized CSAssignment assignContainers(Resource clusterResource, public synchronized CSAssignment assignContainers(Resource clusterResource,
FiCaSchedulerNode node, ResourceLimits currentResourceLimits) { FiCaSchedulerNode node, ResourceLimits currentResourceLimits,
SchedulingMode schedulingMode) {
updateCurrentResourceLimits(currentResourceLimits, clusterResource); updateCurrentResourceLimits(currentResourceLimits, clusterResource);
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
@ -758,12 +730,6 @@ public class LeafQueue extends AbstractCSQueue {
+ " #applications=" + activeApplications.size()); + " #applications=" + activeApplications.size());
} }
// if our queue cannot access this node, just return
if (!SchedulerUtils.checkQueueAccessToNode(accessibleLabels,
node.getLabels())) {
return NULL_ASSIGNMENT;
}
// Check for reserved resources // Check for reserved resources
RMContainer reservedContainer = node.getReservedContainer(); RMContainer reservedContainer = node.getReservedContainer();
if (reservedContainer != null) { if (reservedContainer != null) {
@ -771,10 +737,28 @@ public class LeafQueue extends AbstractCSQueue {
getApplication(reservedContainer.getApplicationAttemptId()); getApplication(reservedContainer.getApplicationAttemptId());
synchronized (application) { synchronized (application) {
return assignReservedContainer(application, node, reservedContainer, return assignReservedContainer(application, node, reservedContainer,
clusterResource); clusterResource, schedulingMode);
} }
} }
// if our queue cannot access this node, just return
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
&& !accessibleToPartition(node.getPartition())) {
return NULL_ASSIGNMENT;
}
// Check if this queue need more resource, simply skip allocation if this
// queue doesn't need more resources.
if (!hasPendingResourceRequest(node.getPartition(),
clusterResource, schedulingMode)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skip this queue=" + getQueuePath()
+ ", because it doesn't need more resource, schedulingMode="
+ schedulingMode.name() + " node-partition=" + node.getPartition());
}
return NULL_ASSIGNMENT;
}
// Try to assign containers to applications in order // Try to assign containers to applications in order
for (FiCaSchedulerApp application : activeApplications) { for (FiCaSchedulerApp application : activeApplications) {
@ -783,6 +767,17 @@ public class LeafQueue extends AbstractCSQueue {
+ application.getApplicationId()); + application.getApplicationId());
application.showRequests(); application.showRequests();
} }
// Check if application needs more resource, skip if it doesn't need more.
if (!application.hasPendingResourceRequest(resourceCalculator,
node.getPartition(), clusterResource, schedulingMode)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skip app_attempt=" + application.getApplicationAttemptId()
+ ", because it doesn't need more resource, schedulingMode="
+ schedulingMode.name() + " node-label=" + node.getPartition());
}
continue;
}
synchronized (application) { synchronized (application) {
// Check if this resource is on the blacklist // Check if this resource is on the blacklist
@ -806,10 +801,27 @@ public class LeafQueue extends AbstractCSQueue {
continue; continue;
} }
// AM container allocation doesn't support non-exclusive allocation to
// avoid painful of preempt an AM container
if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
RMAppAttempt rmAppAttempt =
csContext.getRMContext().getRMApps()
.get(application.getApplicationId()).getCurrentAppAttempt();
if (null == rmAppAttempt.getMasterContainer()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skip allocating AM container to app_attempt="
+ application.getApplicationAttemptId()
+ ", don't allow to allocate AM container in non-exclusive mode");
}
break;
}
}
// Is the node-label-expression of this offswitch resource request // Is the node-label-expression of this offswitch resource request
// matches the node's label? // matches the node's label?
// If not match, jump to next priority. // If not match, jump to next priority.
if (!checkResourceRequestMatchingNodeLabel(anyRequest, node)) { if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
anyRequest, node.getPartition(), schedulingMode)) {
continue; continue;
} }
@ -822,10 +834,6 @@ public class LeafQueue extends AbstractCSQueue {
} }
} }
Set<String> requestedNodeLabels =
getRequestLabelSetByExpression(anyRequest
.getNodeLabelExpression());
// Compute user-limit & set headroom // Compute user-limit & set headroom
// Note: We compute both user-limit & headroom with the highest // Note: We compute both user-limit & headroom with the highest
// priority request as the target. // priority request as the target.
@ -833,27 +841,61 @@ public class LeafQueue extends AbstractCSQueue {
// before all higher priority ones are serviced. // before all higher priority ones are serviced.
Resource userLimit = Resource userLimit =
computeUserLimitAndSetHeadroom(application, clusterResource, computeUserLimitAndSetHeadroom(application, clusterResource,
required, requestedNodeLabels); required, node.getPartition(), schedulingMode);
// Check queue max-capacity limit // Check queue max-capacity limit
if (!super.canAssignToThisQueue(clusterResource, node.getLabels(), if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
this.currentResourceLimits, required, application.getCurrentReservation())) { this.currentResourceLimits, required,
application.getCurrentReservation(), schedulingMode)) {
return NULL_ASSIGNMENT; return NULL_ASSIGNMENT;
} }
// Check user limit // Check user limit
if (!canAssignToUser(clusterResource, application.getUser(), userLimit, if (!canAssignToUser(clusterResource, application.getUser(), userLimit,
application, true, requestedNodeLabels)) { application, true, node.getPartition())) {
break; break;
} }
// Inform the application it is about to get a scheduling opportunity // Inform the application it is about to get a scheduling opportunity
application.addSchedulingOpportunity(priority); application.addSchedulingOpportunity(priority);
// Increase missed-non-partitioned-resource-request-opportunity.
// This is to make sure non-partitioned-resource-request will prefer
// to be allocated to non-partitioned nodes
int missedNonPartitionedRequestSchedulingOpportunity = 0;
if (anyRequest.getNodeLabelExpression().equals(
RMNodeLabelsManager.NO_LABEL)) {
missedNonPartitionedRequestSchedulingOpportunity =
application
.addMissedNonPartitionedRequestSchedulingOpportunity(priority);
}
if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
// Before doing allocation, we need to check scheduling opportunity to
// make sure : non-partitioned resource request should be scheduled to
// non-partitioned partition first.
if (missedNonPartitionedRequestSchedulingOpportunity < scheduler
.getNumClusterNodes()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skip app_attempt="
+ application.getApplicationAttemptId()
+ " priority="
+ priority
+ " because missed-non-partitioned-resource-request"
+ " opportunity under requred:"
+ " Now=" + missedNonPartitionedRequestSchedulingOpportunity
+ " required="
+ scheduler.getNumClusterNodes());
}
break;
}
}
// Try to schedule // Try to schedule
CSAssignment assignment = CSAssignment assignment =
assignContainersOnNode(clusterResource, node, application, priority, assignContainersOnNode(clusterResource, node, application, priority,
null); null, schedulingMode);
// Did the application skip this node? // Did the application skip this node?
if (assignment.getSkipped()) { if (assignment.getSkipped()) {
@ -870,9 +912,9 @@ public class LeafQueue extends AbstractCSQueue {
// Book-keeping // Book-keeping
// Note: Update headroom to account for current allocation too... // Note: Update headroom to account for current allocation too...
allocateResource(clusterResource, application, assigned, allocateResource(clusterResource, application, assigned,
node.getLabels()); node.getPartition());
// Don't reset scheduling opportunities for non-local assignments // Don't reset scheduling opportunities for offswitch assignments
// otherwise the app will be delayed for each non-local assignment. // otherwise the app will be delayed for each non-local assignment.
// This helps apps with many off-cluster requests schedule faster. // This helps apps with many off-cluster requests schedule faster.
if (assignment.getType() != NodeType.OFF_SWITCH) { if (assignment.getType() != NodeType.OFF_SWITCH) {
@ -881,6 +923,10 @@ public class LeafQueue extends AbstractCSQueue {
} }
application.resetSchedulingOpportunities(priority); application.resetSchedulingOpportunities(priority);
} }
// 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);
// Done // Done
return assignment; return assignment;
@ -904,7 +950,8 @@ public class LeafQueue extends AbstractCSQueue {
private synchronized CSAssignment assignReservedContainer( private synchronized CSAssignment assignReservedContainer(
FiCaSchedulerApp application, FiCaSchedulerNode node, FiCaSchedulerApp application, FiCaSchedulerNode node,
RMContainer rmContainer, Resource clusterResource) { RMContainer rmContainer, Resource clusterResource,
SchedulingMode schedulingMode) {
// Do we still need this reservation? // Do we still need this reservation?
Priority priority = rmContainer.getReservedPriority(); Priority priority = rmContainer.getReservedPriority();
if (application.getTotalRequiredResources(priority) == 0) { if (application.getTotalRequiredResources(priority) == 0) {
@ -915,7 +962,7 @@ public class LeafQueue extends AbstractCSQueue {
// Try to assign if we have sufficient resources // Try to assign if we have sufficient resources
CSAssignment tmp = CSAssignment tmp =
assignContainersOnNode(clusterResource, node, application, priority, assignContainersOnNode(clusterResource, node, application, priority,
rmContainer); rmContainer, schedulingMode);
// Doesn't matter... since it's already charged for at time of reservation // Doesn't matter... since it's already charged for at time of reservation
// "re-reservation" is *free* // "re-reservation" is *free*
@ -929,7 +976,8 @@ public class LeafQueue extends AbstractCSQueue {
protected Resource getHeadroom(User user, Resource queueCurrentLimit, protected Resource getHeadroom(User user, Resource queueCurrentLimit,
Resource clusterResource, FiCaSchedulerApp application, Resource required) { Resource clusterResource, FiCaSchedulerApp application, Resource required) {
return getHeadroom(user, queueCurrentLimit, clusterResource, return getHeadroom(user, queueCurrentLimit, clusterResource,
computeUserLimit(application, clusterResource, required, user, null)); computeUserLimit(application, clusterResource, required, user,
RMNodeLabelsManager.NO_LABEL, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY));
} }
private Resource getHeadroom(User user, Resource currentResourceLimit, private Resource getHeadroom(User user, Resource currentResourceLimit,
@ -973,7 +1021,8 @@ public class LeafQueue extends AbstractCSQueue {
@Lock({LeafQueue.class, FiCaSchedulerApp.class}) @Lock({LeafQueue.class, FiCaSchedulerApp.class})
Resource computeUserLimitAndSetHeadroom(FiCaSchedulerApp application, Resource computeUserLimitAndSetHeadroom(FiCaSchedulerApp application,
Resource clusterResource, Resource required, Set<String> requestedLabels) { Resource clusterResource, Resource required, String nodePartition,
SchedulingMode schedulingMode) {
String user = application.getUser(); String user = application.getUser();
User queueUser = getUser(user); User queueUser = getUser(user);
@ -981,7 +1030,7 @@ public class LeafQueue extends AbstractCSQueue {
// TODO, need consider headroom respect labels also // TODO, need consider headroom respect labels also
Resource userLimit = Resource userLimit =
computeUserLimit(application, clusterResource, required, computeUserLimit(application, clusterResource, required,
queueUser, requestedLabels); queueUser, nodePartition, schedulingMode);
setQueueResourceLimitsInfo(clusterResource); setQueueResourceLimitsInfo(clusterResource);
@ -1010,34 +1059,18 @@ public class LeafQueue extends AbstractCSQueue {
@Lock(NoLock.class) @Lock(NoLock.class)
private Resource computeUserLimit(FiCaSchedulerApp application, private Resource computeUserLimit(FiCaSchedulerApp application,
Resource clusterResource, Resource required, User user, Resource clusterResource, Resource required, User user,
Set<String> requestedLabels) { String nodePartition, SchedulingMode schedulingMode) {
// What is our current capacity? // What is our current capacity?
// * It is equal to the max(required, queue-capacity) if // * It is equal to the max(required, queue-capacity) if
// we're running below capacity. The 'max' ensures that jobs in queues // we're running below capacity. The 'max' ensures that jobs in queues
// with miniscule capacity (< 1 slot) make progress // with miniscule capacity (< 1 slot) make progress
// * If we're running over capacity, then its // * If we're running over capacity, then its
// (usedResources + required) (which extra resources we are allocating) // (usedResources + required) (which extra resources we are allocating)
Resource queueCapacity = Resource.newInstance(0, 0); Resource queueCapacity =
if (requestedLabels != null && !requestedLabels.isEmpty()) { Resources.multiplyAndNormalizeUp(resourceCalculator,
// if we have multiple labels to request, we will choose to use the first labelManager.getResourceByLabel(nodePartition, clusterResource),
// label queueCapacities.getAbsoluteCapacity(nodePartition),
String firstLabel = requestedLabels.iterator().next(); minimumAllocation);
queueCapacity =
Resources
.max(resourceCalculator, clusterResource, queueCapacity,
Resources.multiplyAndNormalizeUp(resourceCalculator,
labelManager.getResourceByLabel(firstLabel,
clusterResource),
queueCapacities.getAbsoluteCapacity(firstLabel),
minimumAllocation));
} else {
// else there's no label on request, just to use absolute capacity as
// capacity for nodes without label
queueCapacity =
Resources.multiplyAndNormalizeUp(resourceCalculator, labelManager
.getResourceByLabel(CommonNodeLabelsManager.NO_LABEL, clusterResource),
queueCapacities.getAbsoluteCapacity(), minimumAllocation);
}
// Allow progress for queues with miniscule capacity // Allow progress for queues with miniscule capacity
queueCapacity = queueCapacity =
@ -1047,33 +1080,56 @@ public class LeafQueue extends AbstractCSQueue {
required); required);
Resource currentCapacity = Resource currentCapacity =
Resources.lessThan(resourceCalculator, clusterResource, Resources.lessThan(resourceCalculator, clusterResource,
queueUsage.getUsed(), queueCapacity) ? queueUsage.getUsed(nodePartition), queueCapacity) ? queueCapacity
queueCapacity : Resources.add(queueUsage.getUsed(), required); : Resources.add(queueUsage.getUsed(nodePartition), required);
// Never allow a single user to take more than the // Never allow a single user to take more than the
// queue's configured capacity * user-limit-factor. // queue's configured capacity * user-limit-factor.
// Also, the queue's configured capacity should be higher than // Also, the queue's configured capacity should be higher than
// queue-hard-limit * ulMin // queue-hard-limit * ulMin
final int activeUsers = activeUsersManager.getNumActiveUsers(); final int activeUsers = activeUsersManager.getNumActiveUsers();
Resource limit = // User limit resource is determined by:
// max{currentCapacity / #activeUsers, currentCapacity * user-limit-percentage%)
Resource userLimitResource = Resources.max(
resourceCalculator, clusterResource,
Resources.divideAndCeil(
resourceCalculator, currentCapacity, activeUsers),
Resources.divideAndCeil(
resourceCalculator,
Resources.multiplyAndRoundDown(
currentCapacity, userLimit),
100)
);
// User limit is capped by maxUserLimit
// - maxUserLimit = queueCapacity * user-limit-factor (RESPECT_PARTITION_EXCLUSIVITY)
// - maxUserLimit = total-partition-resource (IGNORE_PARTITION_EXCLUSIVITY)
//
// In IGNORE_PARTITION_EXCLUSIVITY mode, if a queue cannot access a
// partition, its guaranteed resource on that partition is 0. And
// user-limit-factor computation is based on queue's guaranteed capacity. So
// we will not cap user-limit as well as used resource when doing
// IGNORE_PARTITION_EXCLUSIVITY allocation.
Resource maxUserLimit = Resources.none();
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
maxUserLimit =
Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor);
} else if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
maxUserLimit =
labelManager.getResourceByLabel(nodePartition, clusterResource);
}
// Cap final user limit with maxUserLimit
userLimitResource =
Resources.roundUp( Resources.roundUp(
resourceCalculator, resourceCalculator,
Resources.min( Resources.min(
resourceCalculator, clusterResource, resourceCalculator, clusterResource,
Resources.max( userLimitResource,
resourceCalculator, clusterResource, maxUserLimit
Resources.divideAndCeil(
resourceCalculator, currentCapacity, activeUsers),
Resources.divideAndCeil(
resourceCalculator,
Resources.multiplyAndRoundDown(
currentCapacity, userLimit),
100)
),
Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor)
), ),
minimumAllocation); minimumAllocation);
@ -1081,11 +1137,11 @@ public class LeafQueue extends AbstractCSQueue {
String userName = application.getUser(); String userName = application.getUser();
LOG.debug("User limit computation for " + userName + LOG.debug("User limit computation for " + userName +
" in queue " + getQueueName() + " in queue " + getQueueName() +
" userLimit=" + userLimit + " userLimitPercent=" + userLimit +
" userLimitFactor=" + userLimitFactor + " userLimitFactor=" + userLimitFactor +
" required: " + required + " required: " + required +
" consumed: " + user.getUsed() + " consumed: " + user.getUsed() +
" limit: " + limit + " user-limit-resource: " + userLimitResource +
" queueCapacity: " + queueCapacity + " queueCapacity: " + queueCapacity +
" qconsumed: " + queueUsage.getUsed() + " qconsumed: " + queueUsage.getUsed() +
" currentCapacity: " + currentCapacity + " currentCapacity: " + currentCapacity +
@ -1093,31 +1149,26 @@ public class LeafQueue extends AbstractCSQueue {
" clusterCapacity: " + clusterResource " clusterCapacity: " + clusterResource
); );
} }
user.setUserResourceLimit(limit); user.setUserResourceLimit(userLimitResource);
return limit; return userLimitResource;
} }
@Private @Private
protected synchronized boolean canAssignToUser(Resource clusterResource, protected synchronized boolean canAssignToUser(Resource clusterResource,
String userName, Resource limit, FiCaSchedulerApp application, String userName, Resource limit, FiCaSchedulerApp application,
boolean checkReservations, Set<String> requestLabels) { boolean checkReservations, String nodePartition) {
User user = getUser(userName); User user = getUser(userName);
String label = CommonNodeLabelsManager.NO_LABEL;
if (requestLabels != null && !requestLabels.isEmpty()) {
label = requestLabels.iterator().next();
}
// Note: We aren't considering the current request since there is a fixed // Note: We aren't considering the current request since there is a fixed
// overhead of the AM, but it's a > check, not a >= check, so... // overhead of the AM, but it's a > check, not a >= check, so...
if (Resources if (Resources
.greaterThan(resourceCalculator, clusterResource, .greaterThan(resourceCalculator, clusterResource,
user.getUsed(label), user.getUsed(nodePartition),
limit)) { limit)) {
// if enabled, check to see if could we potentially use this node instead // if enabled, check to see if could we potentially use this node instead
// of a reserved node if the application has reserved containers // of a reserved node if the application has reserved containers
if (this.reservationsContinueLooking && checkReservations if (this.reservationsContinueLooking && checkReservations
&& label.equals(CommonNodeLabelsManager.NO_LABEL)) { && nodePartition.equals(CommonNodeLabelsManager.NO_LABEL)) {
if (Resources.lessThanOrEqual( if (Resources.lessThanOrEqual(
resourceCalculator, resourceCalculator,
clusterResource, clusterResource,
@ -1136,7 +1187,7 @@ public class LeafQueue extends AbstractCSQueue {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("User " + userName + " in queue " + getQueueName() LOG.debug("User " + userName + " in queue " + getQueueName()
+ " will exceed limit - " + " consumed: " + " will exceed limit - " + " consumed: "
+ user.getUsed() + " limit: " + limit); + user.getUsed(nodePartition) + " limit: " + limit);
} }
return false; return false;
} }
@ -1176,7 +1227,7 @@ public class LeafQueue extends AbstractCSQueue {
private CSAssignment assignContainersOnNode(Resource clusterResource, private CSAssignment assignContainersOnNode(Resource clusterResource,
FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority,
RMContainer reservedContainer) { RMContainer reservedContainer, SchedulingMode schedulingMode) {
CSAssignment assigned; CSAssignment assigned;
@ -1190,7 +1241,7 @@ public class LeafQueue extends AbstractCSQueue {
assigned = assigned =
assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest, assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest,
node, application, priority, reservedContainer, node, application, priority, reservedContainer,
allocatedContainer); allocatedContainer, schedulingMode);
if (Resources.greaterThan(resourceCalculator, clusterResource, if (Resources.greaterThan(resourceCalculator, clusterResource,
assigned.getResource(), Resources.none())) { assigned.getResource(), Resources.none())) {
@ -1219,7 +1270,7 @@ public class LeafQueue extends AbstractCSQueue {
assigned = assigned =
assignRackLocalContainers(clusterResource, rackLocalResourceRequest, assignRackLocalContainers(clusterResource, rackLocalResourceRequest,
node, application, priority, reservedContainer, node, application, priority, reservedContainer,
allocatedContainer); allocatedContainer, schedulingMode);
if (Resources.greaterThan(resourceCalculator, clusterResource, if (Resources.greaterThan(resourceCalculator, clusterResource,
assigned.getResource(), Resources.none())) { assigned.getResource(), Resources.none())) {
@ -1248,7 +1299,7 @@ public class LeafQueue extends AbstractCSQueue {
assigned = assigned =
assignOffSwitchContainers(clusterResource, offSwitchResourceRequest, assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
node, application, priority, reservedContainer, node, application, priority, reservedContainer,
allocatedContainer); allocatedContainer, schedulingMode);
// update locality statistics // update locality statistics
if (allocatedContainer.getValue() != null) { if (allocatedContainer.getValue() != null) {
@ -1314,16 +1365,17 @@ public class LeafQueue extends AbstractCSQueue {
@Private @Private
protected boolean checkLimitsToReserve(Resource clusterResource, protected boolean checkLimitsToReserve(Resource clusterResource,
FiCaSchedulerApp application, Resource capability) { FiCaSchedulerApp application, Resource capability, String nodePartition,
SchedulingMode schedulingMode) {
// we can't reserve if we got here based on the limit // we can't reserve if we got here based on the limit
// checks assuming we could unreserve!!! // checks assuming we could unreserve!!!
Resource userLimit = computeUserLimitAndSetHeadroom(application, Resource userLimit = computeUserLimitAndSetHeadroom(application,
clusterResource, capability, null); clusterResource, capability, nodePartition, schedulingMode);
// Check queue max-capacity limit, // Check queue max-capacity limit,
// TODO: Consider reservation on labels // TODO: Consider reservation on labels
if (!canAssignToThisQueue(clusterResource, null, if (!canAssignToThisQueue(clusterResource, RMNodeLabelsManager.NO_LABEL,
this.currentResourceLimits, capability, Resources.none())) { this.currentResourceLimits, capability, Resources.none(), schedulingMode)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("was going to reserve but hit queue limit"); LOG.debug("was going to reserve but hit queue limit");
} }
@ -1332,7 +1384,7 @@ public class LeafQueue extends AbstractCSQueue {
// Check user limit // Check user limit
if (!canAssignToUser(clusterResource, application.getUser(), userLimit, if (!canAssignToUser(clusterResource, application.getUser(), userLimit,
application, false, null)) { application, false, nodePartition)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("was going to reserve but hit user limit"); LOG.debug("was going to reserve but hit user limit");
} }
@ -1345,12 +1397,13 @@ public class LeafQueue extends AbstractCSQueue {
private CSAssignment assignNodeLocalContainers(Resource clusterResource, private CSAssignment assignNodeLocalContainers(Resource clusterResource,
ResourceRequest nodeLocalResourceRequest, FiCaSchedulerNode node, ResourceRequest nodeLocalResourceRequest, FiCaSchedulerNode node,
FiCaSchedulerApp application, Priority priority, FiCaSchedulerApp application, Priority priority,
RMContainer reservedContainer, MutableObject allocatedContainer) { RMContainer reservedContainer, MutableObject allocatedContainer,
SchedulingMode schedulingMode) {
if (canAssign(application, priority, node, NodeType.NODE_LOCAL, if (canAssign(application, priority, node, NodeType.NODE_LOCAL,
reservedContainer)) { reservedContainer)) {
return assignContainer(clusterResource, node, application, priority, return assignContainer(clusterResource, node, application, priority,
nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer, nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer,
allocatedContainer); allocatedContainer, schedulingMode);
} }
return new CSAssignment(Resources.none(), NodeType.NODE_LOCAL); return new CSAssignment(Resources.none(), NodeType.NODE_LOCAL);
@ -1359,12 +1412,13 @@ public class LeafQueue extends AbstractCSQueue {
private CSAssignment assignRackLocalContainers(Resource clusterResource, private CSAssignment assignRackLocalContainers(Resource clusterResource,
ResourceRequest rackLocalResourceRequest, FiCaSchedulerNode node, ResourceRequest rackLocalResourceRequest, FiCaSchedulerNode node,
FiCaSchedulerApp application, Priority priority, FiCaSchedulerApp application, Priority priority,
RMContainer reservedContainer, MutableObject allocatedContainer) { RMContainer reservedContainer, MutableObject allocatedContainer,
SchedulingMode schedulingMode) {
if (canAssign(application, priority, node, NodeType.RACK_LOCAL, if (canAssign(application, priority, node, NodeType.RACK_LOCAL,
reservedContainer)) { reservedContainer)) {
return assignContainer(clusterResource, node, application, priority, return assignContainer(clusterResource, node, application, priority,
rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer, rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer,
allocatedContainer); allocatedContainer, schedulingMode);
} }
return new CSAssignment(Resources.none(), NodeType.RACK_LOCAL); return new CSAssignment(Resources.none(), NodeType.RACK_LOCAL);
@ -1373,16 +1427,21 @@ public class LeafQueue extends AbstractCSQueue {
private CSAssignment assignOffSwitchContainers(Resource clusterResource, private CSAssignment assignOffSwitchContainers(Resource clusterResource,
ResourceRequest offSwitchResourceRequest, FiCaSchedulerNode node, ResourceRequest offSwitchResourceRequest, FiCaSchedulerNode node,
FiCaSchedulerApp application, Priority priority, FiCaSchedulerApp application, Priority priority,
RMContainer reservedContainer, MutableObject allocatedContainer) { RMContainer reservedContainer, MutableObject allocatedContainer,
SchedulingMode schedulingMode) {
if (canAssign(application, priority, node, NodeType.OFF_SWITCH, if (canAssign(application, priority, node, NodeType.OFF_SWITCH,
reservedContainer)) { reservedContainer)) {
return assignContainer(clusterResource, node, application, priority, return assignContainer(clusterResource, node, application, priority,
offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer, offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer,
allocatedContainer); allocatedContainer, schedulingMode);
} }
return new CSAssignment(Resources.none(), NodeType.OFF_SWITCH); return new CSAssignment(Resources.none(), NodeType.OFF_SWITCH);
} }
private int getActualNodeLocalityDelay() {
return Math.min(scheduler.getNumClusterNodes(), getNodeLocalityDelay());
}
boolean canAssign(FiCaSchedulerApp application, Priority priority, boolean canAssign(FiCaSchedulerApp application, Priority priority,
FiCaSchedulerNode node, NodeType type, RMContainer reservedContainer) { FiCaSchedulerNode node, NodeType type, RMContainer reservedContainer) {
@ -1417,10 +1476,7 @@ public class LeafQueue extends AbstractCSQueue {
if (type == NodeType.RACK_LOCAL) { if (type == NodeType.RACK_LOCAL) {
// 'Delay' rack-local just a little bit... // 'Delay' rack-local just a little bit...
long missedOpportunities = application.getSchedulingOpportunities(priority); long missedOpportunities = application.getSchedulingOpportunities(priority);
return ( return getActualNodeLocalityDelay() < missedOpportunities;
Math.min(scheduler.getNumClusterNodes(), getNodeLocalityDelay()) <
missedOpportunities
);
} }
// Check if we need containers on this host // Check if we need containers on this host
@ -1460,7 +1516,7 @@ public class LeafQueue extends AbstractCSQueue {
private CSAssignment assignContainer(Resource clusterResource, FiCaSchedulerNode node, private CSAssignment assignContainer(Resource clusterResource, FiCaSchedulerNode node,
FiCaSchedulerApp application, Priority priority, FiCaSchedulerApp application, Priority priority,
ResourceRequest request, NodeType type, RMContainer rmContainer, ResourceRequest request, NodeType type, RMContainer rmContainer,
MutableObject createdContainer) { MutableObject createdContainer, SchedulingMode schedulingMode) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("assignContainers: node=" + node.getNodeName() LOG.debug("assignContainers: node=" + node.getNodeName()
+ " application=" + application.getApplicationId() + " application=" + application.getApplicationId()
@ -1469,9 +1525,8 @@ public class LeafQueue extends AbstractCSQueue {
} }
// check if the resource request can access the label // check if the resource request can access the label
if (!SchedulerUtils.checkNodeLabelExpression( if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(request,
node.getLabels(), node.getPartition(), schedulingMode)) {
request.getNodeLabelExpression())) {
// this is a reserved container, but we cannot allocate it now according // this is a reserved container, but we cannot allocate it now according
// to label not match. This can be caused by node label changed // to label not match. This can be caused by node label changed
// We should un-reserve this container. // We should un-reserve this container.
@ -1576,8 +1631,8 @@ public class LeafQueue extends AbstractCSQueue {
// If we're trying to reserve a container here, not container will be // If we're trying to reserve a container here, not container will be
// unreserved for reserving the new one. Check limits again before // unreserved for reserving the new one. Check limits again before
// reserve the new container // reserve the new container
if (!checkLimitsToReserve(clusterResource, if (!checkLimitsToReserve(clusterResource,
application, capability)) { application, capability, node.getPartition(), schedulingMode)) {
return new CSAssignment(Resources.none(), type); return new CSAssignment(Resources.none(), type);
} }
} }
@ -1666,7 +1721,7 @@ public class LeafQueue extends AbstractCSQueue {
// Book-keeping // Book-keeping
if (removed) { if (removed) {
releaseResource(clusterResource, application, releaseResource(clusterResource, application,
container.getResource(), node.getLabels()); container.getResource(), node.getPartition());
LOG.info("completedContainer" + LOG.info("completedContainer" +
" container=" + container + " container=" + container +
" queue=" + this + " queue=" + this +
@ -1684,13 +1739,13 @@ public class LeafQueue extends AbstractCSQueue {
synchronized void allocateResource(Resource clusterResource, synchronized void allocateResource(Resource clusterResource,
SchedulerApplicationAttempt application, Resource resource, SchedulerApplicationAttempt application, Resource resource,
Set<String> nodeLabels) { String nodePartition) {
super.allocateResource(clusterResource, resource, nodeLabels); super.allocateResource(clusterResource, resource, nodePartition);
// Update user metrics // Update user metrics
String userName = application.getUser(); String userName = application.getUser();
User user = getUser(userName); User user = getUser(userName);
user.assignContainer(resource, nodeLabels); user.assignContainer(resource, nodePartition);
// Note this is a bit unconventional since it gets the object and modifies // Note this is a bit unconventional since it gets the object and modifies
// it here, rather then using set routine // it here, rather then using set routine
Resources.subtractFrom(application.getHeadroom(), resource); // headroom Resources.subtractFrom(application.getHeadroom(), resource); // headroom
@ -1707,13 +1762,13 @@ public class LeafQueue extends AbstractCSQueue {
} }
synchronized void releaseResource(Resource clusterResource, synchronized void releaseResource(Resource clusterResource,
FiCaSchedulerApp application, Resource resource, Set<String> nodeLabels) { FiCaSchedulerApp application, Resource resource, String nodePartition) {
super.releaseResource(clusterResource, resource, nodeLabels); super.releaseResource(clusterResource, resource, nodePartition);
// Update user metrics // Update user metrics
String userName = application.getUser(); String userName = application.getUser();
User user = getUser(userName); User user = getUser(userName);
user.releaseContainer(resource, nodeLabels); user.releaseContainer(resource, nodePartition);
metrics.setAvailableResourcesToUser(userName, application.getHeadroom()); metrics.setAvailableResourcesToUser(userName, application.getHeadroom());
LOG.info(getQueueName() + LOG.info(getQueueName() +
@ -1723,7 +1778,8 @@ public class LeafQueue extends AbstractCSQueue {
private void updateAbsoluteCapacityResource(Resource clusterResource) { private void updateAbsoluteCapacityResource(Resource clusterResource) {
absoluteCapacityResource = absoluteCapacityResource =
Resources.multiplyAndNormalizeUp(resourceCalculator, clusterResource, Resources.multiplyAndNormalizeUp(resourceCalculator, labelManager
.getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
queueCapacities.getAbsoluteCapacity(), minimumAllocation); queueCapacities.getAbsoluteCapacity(), minimumAllocation);
} }
@ -1769,8 +1825,9 @@ public class LeafQueue extends AbstractCSQueue {
// Update application properties // Update application properties
for (FiCaSchedulerApp application : activeApplications) { for (FiCaSchedulerApp application : activeApplications) {
synchronized (application) { synchronized (application) {
computeUserLimitAndSetHeadroom(application, clusterResource, computeUserLimitAndSetHeadroom(application, clusterResource,
Resources.none(), null); Resources.none(), RMNodeLabelsManager.NO_LABEL,
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
} }
} }
} }
@ -1828,25 +1885,12 @@ public class LeafQueue extends AbstractCSQueue {
} }
} }
public void assignContainer(Resource resource, public void assignContainer(Resource resource, String nodePartition) {
Set<String> nodeLabels) { userResourceUsage.incUsed(nodePartition, resource);
if (nodeLabels == null || nodeLabels.isEmpty()) {
userResourceUsage.incUsed(resource);
} else {
for (String label : nodeLabels) {
userResourceUsage.incUsed(label, resource);
}
}
} }
public void releaseContainer(Resource resource, Set<String> nodeLabels) { public void releaseContainer(Resource resource, String nodePartition) {
if (nodeLabels == null || nodeLabels.isEmpty()) { userResourceUsage.decUsed(nodePartition, resource);
userResourceUsage.decUsed(resource);
} else {
for (String label : nodeLabels) {
userResourceUsage.decUsed(label, resource);
}
}
} }
public Resource getUserResourceLimit() { public Resource getUserResourceLimit() {
@ -1869,7 +1913,7 @@ public class LeafQueue extends AbstractCSQueue {
FiCaSchedulerNode node = FiCaSchedulerNode node =
scheduler.getNode(rmContainer.getContainer().getNodeId()); scheduler.getNode(rmContainer.getContainer().getNodeId());
allocateResource(clusterResource, attempt, rmContainer.getContainer() allocateResource(clusterResource, attempt, rmContainer.getContainer()
.getResource(), node.getLabels()); .getResource(), node.getPartition());
} }
getParent().recoverContainer(clusterResource, attempt, rmContainer); getParent().recoverContainer(clusterResource, attempt, rmContainer);
} }
@ -1909,7 +1953,7 @@ public class LeafQueue extends AbstractCSQueue {
FiCaSchedulerNode node = FiCaSchedulerNode node =
scheduler.getNode(rmContainer.getContainer().getNodeId()); scheduler.getNode(rmContainer.getContainer().getNodeId());
allocateResource(clusterResource, application, rmContainer.getContainer() allocateResource(clusterResource, application, rmContainer.getContainer()
.getResource(), node.getLabels()); .getResource(), node.getPartition());
LOG.info("movedContainer" + " container=" + rmContainer.getContainer() LOG.info("movedContainer" + " container=" + rmContainer.getContainer()
+ " resource=" + rmContainer.getContainer().getResource() + " resource=" + rmContainer.getContainer().getResource()
+ " queueMoveIn=" + this + " usedCapacity=" + getUsedCapacity() + " queueMoveIn=" + this + " usedCapacity=" + getUsedCapacity()
@ -1927,7 +1971,7 @@ public class LeafQueue extends AbstractCSQueue {
FiCaSchedulerNode node = FiCaSchedulerNode node =
scheduler.getNode(rmContainer.getContainer().getNodeId()); scheduler.getNode(rmContainer.getContainer().getNodeId());
releaseResource(clusterResource, application, rmContainer.getContainer() releaseResource(clusterResource, application, rmContainer.getContainer()
.getResource(), node.getLabels()); .getResource(), node.getPartition());
LOG.info("movedContainer" + " container=" + rmContainer.getContainer() LOG.info("movedContainer" + " container=" + rmContainer.getContainer()
+ " resource=" + rmContainer.getContainer().getResource() + " resource=" + rmContainer.getContainer().getResource()
+ " queueMoveOut=" + this + " usedCapacity=" + getUsedCapacity() + " queueMoveOut=" + this + " usedCapacity=" + getUsedCapacity()

View File

@ -56,8 +56,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
@ -377,15 +375,28 @@ public class ParentQueue extends AbstractCSQueue {
@Override @Override
public synchronized CSAssignment assignContainers(Resource clusterResource, public synchronized CSAssignment assignContainers(Resource clusterResource,
FiCaSchedulerNode node, ResourceLimits resourceLimits) { FiCaSchedulerNode node, ResourceLimits resourceLimits,
SchedulingMode schedulingMode) {
// if our queue cannot access this node, just return
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
&& !accessibleToPartition(node.getPartition())) {
return NULL_ASSIGNMENT;
}
// Check if this queue need more resource, simply skip allocation if this
// queue doesn't need more resources.
if (!super.hasPendingResourceRequest(node.getPartition(),
clusterResource, schedulingMode)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skip this queue=" + getQueuePath()
+ ", because it doesn't need more resource, schedulingMode="
+ schedulingMode.name() + " node-partition=" + node.getPartition());
}
return NULL_ASSIGNMENT;
}
CSAssignment assignment = CSAssignment assignment =
new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL); new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
Set<String> nodeLabels = node.getLabels();
// if our queue cannot access this node, just return
if (!SchedulerUtils.checkQueueAccessToNode(accessibleLabels, nodeLabels)) {
return assignment;
}
while (canAssign(clusterResource, node)) { while (canAssign(clusterResource, node)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -396,15 +407,17 @@ public class ParentQueue extends AbstractCSQueue {
// Are we over maximum-capacity for this queue? // Are we over maximum-capacity for this queue?
// This will also consider parent's limits and also continuous reservation // This will also consider parent's limits and also continuous reservation
// looking // looking
if (!super.canAssignToThisQueue(clusterResource, nodeLabels, resourceLimits, if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
minimumAllocation, Resources.createResource(getMetrics() resourceLimits, minimumAllocation, Resources.createResource(
.getReservedMB(), getMetrics().getReservedVirtualCores()))) { getMetrics().getReservedMB(), getMetrics()
.getReservedVirtualCores()), schedulingMode)) {
break; break;
} }
// Schedule // Schedule
CSAssignment assignedToChild = CSAssignment assignedToChild =
assignContainersToChildQueues(clusterResource, node, resourceLimits); assignContainersToChildQueues(clusterResource, node, resourceLimits,
schedulingMode);
assignment.setType(assignedToChild.getType()); assignment.setType(assignedToChild.getType());
// Done if no child-queue assigned anything // Done if no child-queue assigned anything
@ -413,7 +426,7 @@ public class ParentQueue extends AbstractCSQueue {
assignedToChild.getResource(), Resources.none())) { assignedToChild.getResource(), Resources.none())) {
// Track resource utilization for the parent-queue // Track resource utilization for the parent-queue
super.allocateResource(clusterResource, assignedToChild.getResource(), super.allocateResource(clusterResource, assignedToChild.getResource(),
nodeLabels); node.getPartition());
// Track resource utilization in this pass of the scheduler // Track resource utilization in this pass of the scheduler
Resources Resources
@ -510,7 +523,8 @@ public class ParentQueue extends AbstractCSQueue {
} }
private synchronized CSAssignment assignContainersToChildQueues( private synchronized CSAssignment assignContainersToChildQueues(
Resource cluster, FiCaSchedulerNode node, ResourceLimits limits) { Resource cluster, FiCaSchedulerNode node, ResourceLimits limits,
SchedulingMode schedulingMode) {
CSAssignment assignment = CSAssignment assignment =
new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL); new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
@ -523,12 +537,13 @@ public class ParentQueue extends AbstractCSQueue {
LOG.debug("Trying to assign to queue: " + childQueue.getQueuePath() LOG.debug("Trying to assign to queue: " + childQueue.getQueuePath()
+ " stats: " + childQueue); + " stats: " + childQueue);
} }
// Get ResourceLimits of child queue before assign containers // Get ResourceLimits of child queue before assign containers
ResourceLimits childLimits = ResourceLimits childLimits =
getResourceLimitsOfChild(childQueue, cluster, limits); getResourceLimitsOfChild(childQueue, cluster, limits);
assignment = childQueue.assignContainers(cluster, node, childLimits); assignment = childQueue.assignContainers(cluster, node,
childLimits, schedulingMode);
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
LOG.debug("Assigned to queue: " + childQueue.getQueuePath() + LOG.debug("Assigned to queue: " + childQueue.getQueuePath() +
" stats: " + childQueue + " --> " + " stats: " + childQueue + " --> " +
@ -584,7 +599,7 @@ public class ParentQueue extends AbstractCSQueue {
// Book keeping // Book keeping
synchronized (this) { synchronized (this) {
super.releaseResource(clusterResource, rmContainer.getContainer() super.releaseResource(clusterResource, rmContainer.getContainer()
.getResource(), node.getLabels()); .getResource(), node.getPartition());
LOG.info("completedContainer" + LOG.info("completedContainer" +
" queue=" + getQueueName() + " queue=" + getQueueName() +
@ -653,7 +668,7 @@ public class ParentQueue extends AbstractCSQueue {
FiCaSchedulerNode node = FiCaSchedulerNode node =
scheduler.getNode(rmContainer.getContainer().getNodeId()); scheduler.getNode(rmContainer.getContainer().getNodeId());
super.allocateResource(clusterResource, rmContainer.getContainer() super.allocateResource(clusterResource, rmContainer.getContainer()
.getResource(), node.getLabels()); .getResource(), node.getPartition());
} }
if (parent != null) { if (parent != null) {
parent.recoverContainer(clusterResource, attempt, rmContainer); parent.recoverContainer(clusterResource, attempt, rmContainer);
@ -681,7 +696,7 @@ public class ParentQueue extends AbstractCSQueue {
FiCaSchedulerNode node = FiCaSchedulerNode node =
scheduler.getNode(rmContainer.getContainer().getNodeId()); scheduler.getNode(rmContainer.getContainer().getNodeId());
super.allocateResource(clusterResource, rmContainer.getContainer() super.allocateResource(clusterResource, rmContainer.getContainer()
.getResource(), node.getLabels()); .getResource(), node.getPartition());
LOG.info("movedContainer" + " queueMoveIn=" + getQueueName() LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
+ " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
+ getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster=" + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster="
@ -701,7 +716,7 @@ public class ParentQueue extends AbstractCSQueue {
scheduler.getNode(rmContainer.getContainer().getNodeId()); scheduler.getNode(rmContainer.getContainer().getNodeId());
super.releaseResource(clusterResource, super.releaseResource(clusterResource,
rmContainer.getContainer().getResource(), rmContainer.getContainer().getResource(),
node.getLabels()); node.getPartition());
LOG.info("movedContainer" + " queueMoveOut=" + getQueueName() LOG.info("movedContainer" + " queueMoveOut=" + getQueueName()
+ " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
+ getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster=" + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster="

View File

@ -0,0 +1,44 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
/**
* Scheduling modes, see below for detailed explanations
*/
public enum SchedulingMode {
/**
* <p>
* When a node has partition (say partition=x), only application in the queue
* can access to partition=x AND requires for partition=x resource can get
* chance to allocate on the node.
* </p>
*
* <p>
* When a node has no partition, only application requires non-partitioned
* resource can get chance to allocate on the node.
* </p>
*/
RESPECT_PARTITION_EXCLUSIVITY,
/**
* Only used when a node has partition AND the partition isn't an exclusive
* partition AND application requires non-partitioned resource.
*/
IGNORE_PARTITION_EXCLUSIVITY
}

View File

@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.Task.State; import org.apache.hadoop.yarn.server.resourcemanager.Task.State;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@ -277,6 +278,9 @@ public class Application {
} else { } else {
request.setNumContainers(request.getNumContainers() + 1); request.setNumContainers(request.getNumContainers() + 1);
} }
if (request.getNodeLabelExpression() == null) {
request.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL);
}
// Note this down for next interaction with ResourceManager // Note this down for next interaction with ResourceManager
ask.remove(request); ask.remove(request);

View File

@ -150,8 +150,14 @@ public class MockAM {
public AllocateResponse allocate( public AllocateResponse allocate(
String host, int memory, int numContainers, String host, int memory, int numContainers,
List<ContainerId> releases, String labelExpression) throws Exception { List<ContainerId> releases, String labelExpression) throws Exception {
return allocate(host, memory, numContainers, 1, releases, labelExpression);
}
public AllocateResponse allocate(
String host, int memory, int numContainers, int priority,
List<ContainerId> releases, String labelExpression) throws Exception {
List<ResourceRequest> reqs = List<ResourceRequest> reqs =
createReq(new String[] { host }, memory, 1, numContainers, createReq(new String[] { host }, memory, priority, numContainers,
labelExpression); labelExpression);
return allocate(reqs, releases); return allocate(reqs, releases);
} }

View File

@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.resourcemanager;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.security.PrivilegedAction; import java.security.PrivilegedAction;
import java.util.Arrays;
import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -200,10 +202,18 @@ public class MockRM extends ResourceManager {
public boolean waitForState(MockNM nm, ContainerId containerId, public boolean waitForState(MockNM nm, ContainerId containerId,
RMContainerState containerState, int timeoutMillisecs) throws Exception { RMContainerState containerState, int timeoutMillisecs) throws Exception {
return waitForState(Arrays.asList(nm), containerId, containerState,
timeoutMillisecs);
}
public boolean waitForState(Collection<MockNM> nms, ContainerId containerId,
RMContainerState containerState, int timeoutMillisecs) throws Exception {
RMContainer container = getResourceScheduler().getRMContainer(containerId); RMContainer container = getResourceScheduler().getRMContainer(containerId);
int timeoutSecs = 0; int timeoutSecs = 0;
while(container == null && timeoutSecs++ < timeoutMillisecs / 100) { while(container == null && timeoutSecs++ < timeoutMillisecs / 100) {
nm.nodeHeartbeat(true); for (MockNM nm : nms) {
nm.nodeHeartbeat(true);
}
container = getResourceScheduler().getRMContainer(containerId); container = getResourceScheduler().getRMContainer(containerId);
System.out.println("Waiting for container " + containerId + " to be allocated."); System.out.println("Waiting for container " + containerId + " to be allocated.");
Thread.sleep(100); Thread.sleep(100);
@ -217,9 +227,11 @@ public class MockRM extends ResourceManager {
&& timeoutSecs++ < timeoutMillisecs / 100) { && timeoutSecs++ < timeoutMillisecs / 100) {
System.out.println("Container : " + containerId + " State is : " System.out.println("Container : " + containerId + " State is : "
+ container.getState() + " Waiting for state : " + containerState); + container.getState() + " Waiting for state : " + containerState);
nm.nodeHeartbeat(true); for (MockNM nm : nms) {
nm.nodeHeartbeat(true);
}
Thread.sleep(100); Thread.sleep(100);
if (timeoutMillisecs <= timeoutSecs * 100) { if (timeoutMillisecs <= timeoutSecs * 100) {
return false; return false;
} }
@ -650,11 +662,28 @@ public class MockRM extends ResourceManager {
am.waitForState(RMAppAttemptState.FINISHED); am.waitForState(RMAppAttemptState.FINISHED);
rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED); rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
} }
@SuppressWarnings("rawtypes")
private static void waitForSchedulerAppAttemptAdded(
ApplicationAttemptId attemptId, MockRM rm) throws InterruptedException {
int tick = 0;
// Wait for at most 5 sec
while (null == ((AbstractYarnScheduler) rm.getResourceScheduler())
.getApplicationAttempt(attemptId) && tick < 50) {
Thread.sleep(100);
if (tick % 10 == 0) {
System.out.println("waiting for SchedulerApplicationAttempt="
+ attemptId + " added.");
}
tick++;
}
}
public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm) public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
throws Exception { throws Exception {
rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED); rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED);
RMAppAttempt attempt = app.getCurrentAppAttempt(); RMAppAttempt attempt = app.getCurrentAppAttempt();
waitForSchedulerAppAttemptAdded(attempt.getAppAttemptId(), rm);
System.out.println("Launch AM " + attempt.getAppAttemptId()); System.out.println("Launch AM " + attempt.getAppAttemptId());
nm.nodeHeartbeat(true); nm.nodeHeartbeat(true);
MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId()); MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());

View File

@ -612,7 +612,7 @@ public class TestApplicationLimits {
// Schedule to compute // Schedule to compute
queue.assignContainers(clusterResource, node_0, new ResourceLimits( queue.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
Resource expectedHeadroom = Resources.createResource(10*16*GB, 1); Resource expectedHeadroom = Resources.createResource(10*16*GB, 1);
assertEquals(expectedHeadroom, app_0_0.getHeadroom()); assertEquals(expectedHeadroom, app_0_0.getHeadroom());
@ -632,7 +632,7 @@ public class TestApplicationLimits {
// Schedule to compute // Schedule to compute
queue.assignContainers(clusterResource, node_0, new ResourceLimits( queue.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); // Schedule to compute clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); // Schedule to compute
assertEquals(expectedHeadroom, app_0_0.getHeadroom()); assertEquals(expectedHeadroom, app_0_0.getHeadroom());
assertEquals(expectedHeadroom, app_0_1.getHeadroom());// no change assertEquals(expectedHeadroom, app_0_1.getHeadroom());// no change
@ -652,7 +652,7 @@ public class TestApplicationLimits {
// Schedule to compute // Schedule to compute
queue.assignContainers(clusterResource, node_0, new ResourceLimits( queue.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); // Schedule to compute clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); // Schedule to compute
expectedHeadroom = Resources.createResource(10*16*GB / 2, 1); // changes expectedHeadroom = Resources.createResource(10*16*GB / 2, 1); // changes
assertEquals(expectedHeadroom, app_0_0.getHeadroom()); assertEquals(expectedHeadroom, app_0_0.getHeadroom());
assertEquals(expectedHeadroom, app_0_1.getHeadroom()); assertEquals(expectedHeadroom, app_0_1.getHeadroom());
@ -661,7 +661,7 @@ public class TestApplicationLimits {
// Now reduce cluster size and check for the smaller headroom // Now reduce cluster size and check for the smaller headroom
clusterResource = Resources.createResource(90*16*GB); clusterResource = Resources.createResource(90*16*GB);
queue.assignContainers(clusterResource, node_0, new ResourceLimits( queue.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); // Schedule to compute clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); // Schedule to compute
expectedHeadroom = Resources.createResource(9*16*GB / 2, 1); // changes expectedHeadroom = Resources.createResource(9*16*GB / 2, 1); // changes
assertEquals(expectedHeadroom, app_0_0.getHeadroom()); assertEquals(expectedHeadroom, app_0_0.getHeadroom());
assertEquals(expectedHeadroom, app_0_1.getHeadroom()); assertEquals(expectedHeadroom, app_0_1.getHeadroom());

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher; import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@ -133,7 +134,7 @@ public class TestChildQueueOrder {
final Resource allocatedResource = Resources.createResource(allocation); final Resource allocatedResource = Resources.createResource(allocation);
if (queue instanceof ParentQueue) { if (queue instanceof ParentQueue) {
((ParentQueue)queue).allocateResource(clusterResource, ((ParentQueue)queue).allocateResource(clusterResource,
allocatedResource, null); allocatedResource, RMNodeLabelsManager.NO_LABEL);
} else { } else {
FiCaSchedulerApp app1 = getMockApplication(0, ""); FiCaSchedulerApp app1 = getMockApplication(0, "");
((LeafQueue)queue).allocateResource(clusterResource, app1, ((LeafQueue)queue).allocateResource(clusterResource, app1,
@ -145,7 +146,7 @@ public class TestChildQueueOrder {
doReturn(new CSAssignment(Resources.none(), type)). doReturn(new CSAssignment(Resources.none(), type)).
when(queue) when(queue)
.assignContainers(eq(clusterResource), eq(node), .assignContainers(eq(clusterResource), eq(node),
any(ResourceLimits.class)); any(ResourceLimits.class), any(SchedulingMode.class));
// Mock the node's resource availability // Mock the node's resource availability
Resource available = node.getAvailableResource(); Resource available = node.getAvailableResource();
@ -157,7 +158,7 @@ public class TestChildQueueOrder {
} }
}). }).
when(queue).assignContainers(eq(clusterResource), eq(node), when(queue).assignContainers(eq(clusterResource), eq(node),
any(ResourceLimits.class)); any(ResourceLimits.class), any(SchedulingMode.class));
doNothing().when(node).releaseContainer(any(Container.class)); doNothing().when(node).releaseContainer(any(Container.class));
} }
@ -241,6 +242,14 @@ public class TestChildQueueOrder {
CSQueue b = queues.get(B); CSQueue b = queues.get(B);
CSQueue c = queues.get(C); CSQueue c = queues.get(C);
CSQueue d = queues.get(D); CSQueue d = queues.get(D);
// Make a/b/c/d has >0 pending resource, so that allocation will continue.
queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
.incPending(Resources.createResource(1 * GB));
a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
c.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
d.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
final String user_0 = "user_0"; final String user_0 = "user_0";
@ -275,7 +284,7 @@ public class TestChildQueueOrder {
stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(c, clusterResource, node_0, 0*GB);
stubQueueAllocation(d, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB);
root.assignContainers(clusterResource, node_0, new ResourceLimits( root.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
for(int i=0; i < 2; i++) for(int i=0; i < 2; i++)
{ {
stubQueueAllocation(a, clusterResource, node_0, 0*GB); stubQueueAllocation(a, clusterResource, node_0, 0*GB);
@ -283,7 +292,7 @@ public class TestChildQueueOrder {
stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(c, clusterResource, node_0, 0*GB);
stubQueueAllocation(d, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB);
root.assignContainers(clusterResource, node_0, new ResourceLimits( root.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
} }
for(int i=0; i < 3; i++) for(int i=0; i < 3; i++)
{ {
@ -292,7 +301,7 @@ public class TestChildQueueOrder {
stubQueueAllocation(c, clusterResource, node_0, 1*GB); stubQueueAllocation(c, clusterResource, node_0, 1*GB);
stubQueueAllocation(d, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB);
root.assignContainers(clusterResource, node_0, new ResourceLimits( root.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
} }
for(int i=0; i < 4; i++) for(int i=0; i < 4; i++)
{ {
@ -301,7 +310,7 @@ public class TestChildQueueOrder {
stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(c, clusterResource, node_0, 0*GB);
stubQueueAllocation(d, clusterResource, node_0, 1*GB); stubQueueAllocation(d, clusterResource, node_0, 1*GB);
root.assignContainers(clusterResource, node_0, new ResourceLimits( root.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
} }
verifyQueueMetrics(a, 1*GB, clusterResource); verifyQueueMetrics(a, 1*GB, clusterResource);
verifyQueueMetrics(b, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource);
@ -335,7 +344,7 @@ public class TestChildQueueOrder {
stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(c, clusterResource, node_0, 0*GB);
stubQueueAllocation(d, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB);
root.assignContainers(clusterResource, node_0, new ResourceLimits( root.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
} }
verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(a, 3*GB, clusterResource);
verifyQueueMetrics(b, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource);
@ -363,7 +372,7 @@ public class TestChildQueueOrder {
stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(c, clusterResource, node_0, 0*GB);
stubQueueAllocation(d, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB);
root.assignContainers(clusterResource, node_0, new ResourceLimits( root.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(a, 2*GB, clusterResource);
verifyQueueMetrics(b, 3*GB, clusterResource); verifyQueueMetrics(b, 3*GB, clusterResource);
verifyQueueMetrics(c, 3*GB, clusterResource); verifyQueueMetrics(c, 3*GB, clusterResource);
@ -390,7 +399,7 @@ public class TestChildQueueOrder {
stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(c, clusterResource, node_0, 0*GB);
stubQueueAllocation(d, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB);
root.assignContainers(clusterResource, node_0, new ResourceLimits( root.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(a, 3*GB, clusterResource);
verifyQueueMetrics(b, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource);
verifyQueueMetrics(c, 3*GB, clusterResource); verifyQueueMetrics(c, 3*GB, clusterResource);
@ -405,12 +414,14 @@ public class TestChildQueueOrder {
stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(c, clusterResource, node_0, 0*GB);
stubQueueAllocation(d, clusterResource, node_0, 1*GB); stubQueueAllocation(d, clusterResource, node_0, 1*GB);
root.assignContainers(clusterResource, node_0, new ResourceLimits( root.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
InOrder allocationOrder = inOrder(d,b); InOrder allocationOrder = inOrder(d,b);
allocationOrder.verify(d).assignContainers(eq(clusterResource), allocationOrder.verify(d).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), any(ResourceLimits.class)); any(FiCaSchedulerNode.class), any(ResourceLimits.class),
allocationOrder.verify(b).assignContainers(eq(clusterResource), any(SchedulingMode.class));
any(FiCaSchedulerNode.class), any(ResourceLimits.class)); allocationOrder.verify(b).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), any(ResourceLimits.class),
any(SchedulingMode.class));
verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(a, 3*GB, clusterResource);
verifyQueueMetrics(b, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource);
verifyQueueMetrics(c, 3*GB, clusterResource); verifyQueueMetrics(c, 3*GB, clusterResource);

View File

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

View File

@ -351,7 +351,7 @@ public class TestLeafQueue {
// Only 1 container // Only 1 container
a.assignContainers(clusterResource, node_0, new ResourceLimits( a.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals( assertEquals(
(int)(node_0.getTotalResource().getMemory() * a.getCapacity()) - (1*GB), (int)(node_0.getTotalResource().getMemory() * a.getCapacity()) - (1*GB),
a.getMetrics().getAvailableMB()); a.getMetrics().getAvailableMB());
@ -487,7 +487,7 @@ public class TestLeafQueue {
// Only 1 container // Only 1 container
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -498,7 +498,7 @@ public class TestLeafQueue {
// Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
// you can get one container more than user-limit // you can get one container more than user-limit
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -507,7 +507,7 @@ public class TestLeafQueue {
// Can't allocate 3rd due to user-limit // Can't allocate 3rd due to user-limit
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -517,7 +517,7 @@ public class TestLeafQueue {
// Bump up user-limit-factor, now allocate should work // Bump up user-limit-factor, now allocate should work
a.setUserLimitFactor(10); a.setUserLimitFactor(10);
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, a.getUsedResources().getMemory());
assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -526,7 +526,7 @@ public class TestLeafQueue {
// One more should work, for app_1, due to user-limit-factor // One more should work, for app_1, due to user-limit-factor
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(4*GB, a.getUsedResources().getMemory());
assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@ -537,7 +537,7 @@ public class TestLeafQueue {
// Now - no more allocs since we are at max-cap // Now - no more allocs since we are at max-cap
a.setMaxCapacity(0.5f); a.setMaxCapacity(0.5f);
a.assignContainers(clusterResource, node_0, new ResourceLimits( a.assignContainers(clusterResource, node_0, new ResourceLimits(
clusterResource)); clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(4*GB, a.getUsedResources().getMemory());
assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@ -653,21 +653,21 @@ public class TestLeafQueue {
// 1 container to user_0 // 1 container to user_0
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
// Again one to user_0 since he hasn't exceeded user limit yet // Again one to user_0 since he hasn't exceeded user limit yet
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
// One more to user_0 since he is the only active user // One more to user_0 since he is the only active user
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(4*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(2*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_1.getCurrentConsumption().getMemory());
@ -719,10 +719,10 @@ public class TestLeafQueue {
1, qb.getActiveUsersManager().getNumActiveUsers()); 1, qb.getActiveUsersManager().getNumActiveUsers());
//get headroom //get headroom
qb.assignContainers(clusterResource, node_0, qb.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0 qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(), .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
null); "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
//maxqueue 16G, userlimit 13G, - 4G used = 9G //maxqueue 16G, userlimit 13G, - 4G used = 9G
assertEquals(9*GB,app_0.getHeadroom().getMemory()); assertEquals(9*GB,app_0.getHeadroom().getMemory());
@ -739,10 +739,10 @@ public class TestLeafQueue {
u1Priority, recordFactory))); u1Priority, recordFactory)));
qb.submitApplicationAttempt(app_2, user_1); qb.submitApplicationAttempt(app_2, user_1);
qb.assignContainers(clusterResource, node_1, qb.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0 qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(), .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
null); "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8*GB, qb.getUsedResources().getMemory()); assertEquals(8*GB, qb.getUsedResources().getMemory());
assertEquals(4*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_0.getCurrentConsumption().getMemory());
@ -782,12 +782,12 @@ public class TestLeafQueue {
qb.submitApplicationAttempt(app_1, user_0); qb.submitApplicationAttempt(app_1, user_0);
qb.submitApplicationAttempt(app_3, user_1); qb.submitApplicationAttempt(app_3, user_1);
qb.assignContainers(clusterResource, node_0, qb.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
qb.assignContainers(clusterResource, node_0, qb.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3 qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3
.getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(), .getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(),
null); "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(4*GB, qb.getUsedResources().getMemory()); assertEquals(4*GB, qb.getUsedResources().getMemory());
//maxqueue 16G, userlimit 7G, used (by each user) 2G, headroom 5G (both) //maxqueue 16G, userlimit 7G, used (by each user) 2G, headroom 5G (both)
assertEquals(5*GB, app_3.getHeadroom().getMemory()); assertEquals(5*GB, app_3.getHeadroom().getMemory());
@ -803,13 +803,13 @@ public class TestLeafQueue {
TestUtils.createResourceRequest(ResourceRequest.ANY, 6*GB, 1, true, TestUtils.createResourceRequest(ResourceRequest.ANY, 6*GB, 1, true,
u0Priority, recordFactory))); u0Priority, recordFactory)));
qb.assignContainers(clusterResource, node_1, qb.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
qb.computeUserLimitAndSetHeadroom(app_4, clusterResource, app_4 qb.computeUserLimitAndSetHeadroom(app_4, clusterResource, app_4
.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(), .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
null); "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3 qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3
.getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(), .getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(),
null); "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
//app3 is user1, active from last test case //app3 is user1, active from last test case
@ -876,7 +876,7 @@ public class TestLeafQueue {
priority, recordFactory))); priority, recordFactory)));
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -893,7 +893,7 @@ public class TestLeafQueue {
priority, recordFactory))); priority, recordFactory)));
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@ -982,7 +982,7 @@ public class TestLeafQueue {
// 1 container to user_0 // 1 container to user_0
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -993,7 +993,7 @@ public class TestLeafQueue {
// Again one to user_0 since he hasn't exceeded user limit yet // Again one to user_0 since he hasn't exceeded user limit yet
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@ -1010,7 +1010,7 @@ public class TestLeafQueue {
// No more to user_0 since he is already over user-limit // No more to user_0 since he is already over user-limit
// and no more containers to queue since it's already at max-cap // and no more containers to queue since it's already at max-cap
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@ -1024,7 +1024,7 @@ public class TestLeafQueue {
priority, recordFactory))); priority, recordFactory)));
assertEquals(1, a.getActiveUsersManager().getNumActiveUsers()); assertEquals(1, a.getActiveUsersManager().getNumActiveUsers());
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(0*GB, app_2.getHeadroom().getMemory()); // hit queue max-cap assertEquals(0*GB, app_2.getHeadroom().getMemory()); // hit queue max-cap
} }
@ -1095,7 +1095,7 @@ public class TestLeafQueue {
// Only 1 container // Only 1 container
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1103,7 +1103,7 @@ public class TestLeafQueue {
// Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
// you can get one container more than user-limit // you can get one container more than user-limit
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1111,7 +1111,7 @@ public class TestLeafQueue {
// Can't allocate 3rd due to user-limit // Can't allocate 3rd due to user-limit
a.setUserLimit(25); a.setUserLimit(25);
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1130,7 +1130,7 @@ public class TestLeafQueue {
// user_0 is at limit inspite of high user-limit-factor // user_0 is at limit inspite of high user-limit-factor
a.setUserLimitFactor(10); a.setUserLimitFactor(10);
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(5*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1140,7 +1140,7 @@ public class TestLeafQueue {
// Now allocations should goto app_0 since // Now allocations should goto app_0 since
// user_0 is at user-limit not above it // user_0 is at user-limit not above it
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(6*GB, a.getUsedResources().getMemory());
assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1151,7 +1151,7 @@ public class TestLeafQueue {
// Now - no more allocs since we are at max-cap // Now - no more allocs since we are at max-cap
a.setMaxCapacity(0.5f); a.setMaxCapacity(0.5f);
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(6*GB, a.getUsedResources().getMemory());
assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1163,7 +1163,7 @@ public class TestLeafQueue {
a.setMaxCapacity(1.0f); a.setMaxCapacity(1.0f);
a.setUserLimitFactor(1); a.setUserLimitFactor(1);
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(7*GB, a.getUsedResources().getMemory()); assertEquals(7*GB, a.getUsedResources().getMemory());
assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1172,7 +1172,7 @@ public class TestLeafQueue {
// Now we should assign to app_3 again since user_2 is under user-limit // Now we should assign to app_3 again since user_2 is under user-limit
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8*GB, a.getUsedResources().getMemory()); assertEquals(8*GB, a.getUsedResources().getMemory());
assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1272,7 +1272,7 @@ public class TestLeafQueue {
// Only 1 container // Only 1 container
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1283,7 +1283,7 @@ public class TestLeafQueue {
// Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
// you can get one container more than user-limit // you can get one container more than user-limit
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1292,7 +1292,7 @@ public class TestLeafQueue {
// Now, reservation should kick in for app_1 // Now, reservation should kick in for app_1
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(6*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1309,7 +1309,7 @@ public class TestLeafQueue {
ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
RMContainerEventType.KILL, null, true); RMContainerEventType.KILL, null, true);
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(5*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1326,7 +1326,7 @@ public class TestLeafQueue {
ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
RMContainerEventType.KILL, null, true); RMContainerEventType.KILL, null, true);
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(4*GB, a.getUsedResources().getMemory());
assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(4*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemory());
@ -1394,7 +1394,7 @@ public class TestLeafQueue {
// Start testing... // Start testing...
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1404,7 +1404,7 @@ public class TestLeafQueue {
// Now, reservation should kick in for app_1 // Now, reservation should kick in for app_1
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(6*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1418,7 +1418,7 @@ public class TestLeafQueue {
doReturn(-1).when(a).getNodeLocalityDelay(); doReturn(-1).when(a).getNodeLocalityDelay();
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(10*GB, a.getUsedResources().getMemory()); assertEquals(10*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(4*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemory());
@ -1435,7 +1435,7 @@ public class TestLeafQueue {
ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
RMContainerEventType.KILL, null, true); RMContainerEventType.KILL, null, true);
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8*GB, a.getUsedResources().getMemory()); assertEquals(8*GB, a.getUsedResources().getMemory());
assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(8*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(8*GB, app_1.getCurrentConsumption().getMemory());
@ -1504,7 +1504,7 @@ public class TestLeafQueue {
// Only 1 container // Only 1 container
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1512,14 +1512,14 @@ public class TestLeafQueue {
// Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
// you can get one container more than user-limit // you can get one container more than user-limit
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
// Now, reservation should kick in for app_1 // Now, reservation should kick in for app_1
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(6*GB, a.getUsedResources().getMemory());
assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1534,7 +1534,7 @@ public class TestLeafQueue {
ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
RMContainerEventType.KILL, null, true); RMContainerEventType.KILL, null, true);
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(5*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1544,7 +1544,7 @@ public class TestLeafQueue {
// Re-reserve // Re-reserve
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(5*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
@ -1554,7 +1554,7 @@ public class TestLeafQueue {
// Try to schedule on node_1 now, should *move* the reservation // Try to schedule on node_1 now, should *move* the reservation
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(9*GB, a.getUsedResources().getMemory()); assertEquals(9*GB, a.getUsedResources().getMemory());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(4*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemory());
@ -1572,7 +1572,7 @@ public class TestLeafQueue {
ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
RMContainerEventType.KILL, null, true); RMContainerEventType.KILL, null, true);
CSAssignment assignment = a.assignContainers(clusterResource, node_0, CSAssignment assignment = a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8*GB, a.getUsedResources().getMemory()); assertEquals(8*GB, a.getUsedResources().getMemory());
assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory());
assertEquals(4*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemory());
@ -1644,7 +1644,7 @@ public class TestLeafQueue {
// Start with off switch, shouldn't allocate due to delay scheduling // Start with off switch, shouldn't allocate due to delay scheduling
assignment = a.assignContainers(clusterResource, node_2, assignment = a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), verify(app_0, never()).allocate(any(NodeType.class), eq(node_2),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(1, app_0.getSchedulingOpportunities(priority)); assertEquals(1, app_0.getSchedulingOpportunities(priority));
@ -1653,7 +1653,7 @@ public class TestLeafQueue {
// Another off switch, shouldn't allocate due to delay scheduling // Another off switch, shouldn't allocate due to delay scheduling
assignment = a.assignContainers(clusterResource, node_2, assignment = a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), verify(app_0, never()).allocate(any(NodeType.class), eq(node_2),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(2, app_0.getSchedulingOpportunities(priority)); assertEquals(2, app_0.getSchedulingOpportunities(priority));
@ -1662,7 +1662,7 @@ public class TestLeafQueue {
// Another off switch, shouldn't allocate due to delay scheduling // Another off switch, shouldn't allocate due to delay scheduling
assignment = a.assignContainers(clusterResource, node_2, assignment = a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), verify(app_0, never()).allocate(any(NodeType.class), eq(node_2),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(3, app_0.getSchedulingOpportunities(priority)); assertEquals(3, app_0.getSchedulingOpportunities(priority));
@ -1672,7 +1672,7 @@ public class TestLeafQueue {
// Another off switch, now we should allocate // Another off switch, now we should allocate
// since missedOpportunities=3 and reqdContainers=3 // since missedOpportunities=3 and reqdContainers=3
assignment = a.assignContainers(clusterResource, node_2, assignment = a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2), verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(4, app_0.getSchedulingOpportunities(priority)); // should NOT reset assertEquals(4, app_0.getSchedulingOpportunities(priority)); // should NOT reset
@ -1681,7 +1681,7 @@ public class TestLeafQueue {
// NODE_LOCAL - node_0 // NODE_LOCAL - node_0
assignment = a.assignContainers(clusterResource, node_0, assignment = a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0), verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@ -1690,7 +1690,7 @@ public class TestLeafQueue {
// NODE_LOCAL - node_1 // NODE_LOCAL - node_1
assignment = a.assignContainers(clusterResource, node_1, assignment = a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1), verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@ -1719,14 +1719,14 @@ public class TestLeafQueue {
// Shouldn't assign RACK_LOCAL yet // Shouldn't assign RACK_LOCAL yet
assignment = a.assignContainers(clusterResource, node_3, assignment = a.assignContainers(clusterResource, node_3,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(1, app_0.getSchedulingOpportunities(priority)); assertEquals(1, app_0.getSchedulingOpportunities(priority));
assertEquals(2, app_0.getTotalRequiredResources(priority)); assertEquals(2, app_0.getTotalRequiredResources(priority));
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
// Should assign RACK_LOCAL now // Should assign RACK_LOCAL now
assignment = a.assignContainers(clusterResource, node_3, assignment = a.assignContainers(clusterResource, node_3,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0).allocate(eq(NodeType.RACK_LOCAL), eq(node_3), verify(app_0).allocate(eq(NodeType.RACK_LOCAL), eq(node_3),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@ -1808,7 +1808,7 @@ public class TestLeafQueue {
// Start with off switch, shouldn't allocate P1 due to delay scheduling // Start with off switch, shouldn't allocate P1 due to delay scheduling
// thus, no P2 either! // thus, no P2 either!
a.assignContainers(clusterResource, node_2, a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), verify(app_0, never()).allocate(any(NodeType.class), eq(node_2),
eq(priority_1), any(ResourceRequest.class), any(Container.class)); eq(priority_1), any(ResourceRequest.class), any(Container.class));
assertEquals(1, app_0.getSchedulingOpportunities(priority_1)); assertEquals(1, app_0.getSchedulingOpportunities(priority_1));
@ -1821,7 +1821,7 @@ public class TestLeafQueue {
// Another off-switch, shouldn't allocate P1 due to delay scheduling // Another off-switch, shouldn't allocate P1 due to delay scheduling
// thus, no P2 either! // thus, no P2 either!
a.assignContainers(clusterResource, node_2, a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), verify(app_0, never()).allocate(any(NodeType.class), eq(node_2),
eq(priority_1), any(ResourceRequest.class), any(Container.class)); eq(priority_1), any(ResourceRequest.class), any(Container.class));
assertEquals(2, app_0.getSchedulingOpportunities(priority_1)); assertEquals(2, app_0.getSchedulingOpportunities(priority_1));
@ -1833,7 +1833,7 @@ public class TestLeafQueue {
// Another off-switch, shouldn't allocate OFF_SWITCH P1 // Another off-switch, shouldn't allocate OFF_SWITCH P1
a.assignContainers(clusterResource, node_2, a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2), verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2),
eq(priority_1), any(ResourceRequest.class), any(Container.class)); eq(priority_1), any(ResourceRequest.class), any(Container.class));
assertEquals(3, app_0.getSchedulingOpportunities(priority_1)); assertEquals(3, app_0.getSchedulingOpportunities(priority_1));
@ -1845,7 +1845,7 @@ public class TestLeafQueue {
// Now, DATA_LOCAL for P1 // Now, DATA_LOCAL for P1
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0), verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0),
eq(priority_1), any(ResourceRequest.class), any(Container.class)); eq(priority_1), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority_1)); assertEquals(0, app_0.getSchedulingOpportunities(priority_1));
@ -1857,7 +1857,7 @@ public class TestLeafQueue {
// Now, OFF_SWITCH for P2 // Now, OFF_SWITCH for P2
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_1), verify(app_0, never()).allocate(any(NodeType.class), eq(node_1),
eq(priority_1), any(ResourceRequest.class), any(Container.class)); eq(priority_1), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority_1)); assertEquals(0, app_0.getSchedulingOpportunities(priority_1));
@ -1934,7 +1934,7 @@ public class TestLeafQueue {
// NODE_LOCAL - node_0_1 // NODE_LOCAL - node_0_1
a.assignContainers(clusterResource, node_0_0, a.assignContainers(clusterResource, node_0_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0_0), verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@ -1943,7 +1943,7 @@ public class TestLeafQueue {
// No allocation on node_1_0 even though it's node/rack local since // No allocation on node_1_0 even though it's node/rack local since
// required(ANY) == 0 // required(ANY) == 0
a.assignContainers(clusterResource, node_1_0, a.assignContainers(clusterResource, node_1_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // Still zero assertEquals(0, app_0.getSchedulingOpportunities(priority)); // Still zero
@ -1960,7 +1960,7 @@ public class TestLeafQueue {
// No allocation on node_0_1 even though it's node/rack local since // No allocation on node_0_1 even though it's node/rack local since
// required(rack_1) == 0 // required(rack_1) == 0
a.assignContainers(clusterResource, node_0_1, a.assignContainers(clusterResource, node_0_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(1, app_0.getSchedulingOpportunities(priority)); assertEquals(1, app_0.getSchedulingOpportunities(priority));
@ -1968,7 +1968,7 @@ public class TestLeafQueue {
// NODE_LOCAL - node_1 // NODE_LOCAL - node_1
a.assignContainers(clusterResource, node_1_0, a.assignContainers(clusterResource, node_1_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0), verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
@ -2221,7 +2221,7 @@ public class TestLeafQueue {
// node_0_1 // node_0_1
// Shouldn't allocate since RR(rack_0) = null && RR(ANY) = relax: false // Shouldn't allocate since RR(rack_0) = null && RR(ANY) = relax: false
a.assignContainers(clusterResource, node_0_1, a.assignContainers(clusterResource, node_0_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1), verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0 assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
@ -2244,7 +2244,7 @@ public class TestLeafQueue {
// node_1_1 // node_1_1
// Shouldn't allocate since RR(rack_1) = relax: false // Shouldn't allocate since RR(rack_1) = relax: false
a.assignContainers(clusterResource, node_1_1, a.assignContainers(clusterResource, node_1_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1), verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0 assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
@ -2275,7 +2275,7 @@ public class TestLeafQueue {
// node_1_1 // node_1_1
// Shouldn't allocate since node_1_1 is blacklisted // Shouldn't allocate since node_1_1 is blacklisted
a.assignContainers(clusterResource, node_1_1, a.assignContainers(clusterResource, node_1_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1), verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0 assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
@ -2304,7 +2304,7 @@ public class TestLeafQueue {
// node_1_1 // node_1_1
// Shouldn't allocate since rack_1 is blacklisted // Shouldn't allocate since rack_1 is blacklisted
a.assignContainers(clusterResource, node_1_1, a.assignContainers(clusterResource, node_1_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1), verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0 assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0
@ -2331,7 +2331,7 @@ public class TestLeafQueue {
// Now, should allocate since RR(rack_1) = relax: true // Now, should allocate since RR(rack_1) = relax: true
a.assignContainers(clusterResource, node_1_1, a.assignContainers(clusterResource, node_1_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0,never()).allocate(eq(NodeType.RACK_LOCAL), eq(node_1_1), verify(app_0,never()).allocate(eq(NodeType.RACK_LOCAL), eq(node_1_1),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); assertEquals(0, app_0.getSchedulingOpportunities(priority));
@ -2362,7 +2362,7 @@ public class TestLeafQueue {
// host_1_1: 7G // host_1_1: 7G
a.assignContainers(clusterResource, node_1_0, a.assignContainers(clusterResource, node_1_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0), verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class)); any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); assertEquals(0, app_0.getSchedulingOpportunities(priority));
@ -2445,7 +2445,7 @@ public class TestLeafQueue {
try { try {
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
} catch (NullPointerException e) { } catch (NullPointerException e) {
Assert.fail("NPE when allocating container on node but " Assert.fail("NPE when allocating container on node but "
+ "forget to set off-switch request should be handled"); + "forget to set off-switch request should be handled");

View File

@ -23,7 +23,6 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import static org.mockito.Matchers.any; import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.eq; import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doReturn;
@ -45,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.YarnAuthorizationProvider; import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@ -146,7 +146,7 @@ public class TestParentQueue {
final Resource allocatedResource = Resources.createResource(allocation); final Resource allocatedResource = Resources.createResource(allocation);
if (queue instanceof ParentQueue) { if (queue instanceof ParentQueue) {
((ParentQueue)queue).allocateResource(clusterResource, ((ParentQueue)queue).allocateResource(clusterResource,
allocatedResource, null); allocatedResource, RMNodeLabelsManager.NO_LABEL);
} else { } else {
FiCaSchedulerApp app1 = getMockApplication(0, ""); FiCaSchedulerApp app1 = getMockApplication(0, "");
((LeafQueue)queue).allocateResource(clusterResource, app1, ((LeafQueue)queue).allocateResource(clusterResource, app1,
@ -157,7 +157,7 @@ public class TestParentQueue {
if (allocation > 0) { if (allocation > 0) {
doReturn(new CSAssignment(Resources.none(), type)).when(queue) doReturn(new CSAssignment(Resources.none(), type)).when(queue)
.assignContainers(eq(clusterResource), eq(node), .assignContainers(eq(clusterResource), eq(node),
any(ResourceLimits.class)); any(ResourceLimits.class), any(SchedulingMode.class));
// Mock the node's resource availability // Mock the node's resource availability
Resource available = node.getAvailableResource(); Resource available = node.getAvailableResource();
@ -168,7 +168,7 @@ public class TestParentQueue {
return new CSAssignment(allocatedResource, type); return new CSAssignment(allocatedResource, type);
} }
}).when(queue).assignContainers(eq(clusterResource), eq(node), }).when(queue).assignContainers(eq(clusterResource), eq(node),
any(ResourceLimits.class)); any(ResourceLimits.class), any(SchedulingMode.class));
} }
private float computeQueueAbsoluteUsedCapacity(CSQueue queue, private float computeQueueAbsoluteUsedCapacity(CSQueue queue,
@ -228,11 +228,16 @@ public class TestParentQueue {
LeafQueue a = (LeafQueue)queues.get(A); LeafQueue a = (LeafQueue)queues.get(A);
LeafQueue b = (LeafQueue)queues.get(B); LeafQueue b = (LeafQueue)queues.get(B);
a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
.incPending(Resources.createResource(1 * GB));
// Simulate B returning a container on node_0 // Simulate B returning a container on node_0
stubQueueAllocation(a, clusterResource, node_0, 0*GB); stubQueueAllocation(a, clusterResource, node_0, 0*GB);
stubQueueAllocation(b, clusterResource, node_0, 1*GB); stubQueueAllocation(b, clusterResource, node_0, 1*GB);
root.assignContainers(clusterResource, node_0, root.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(a, 0*GB, clusterResource);
verifyQueueMetrics(b, 1*GB, clusterResource); verifyQueueMetrics(b, 1*GB, clusterResource);
@ -240,12 +245,12 @@ public class TestParentQueue {
stubQueueAllocation(a, clusterResource, node_1, 2*GB); stubQueueAllocation(a, clusterResource, node_1, 2*GB);
stubQueueAllocation(b, clusterResource, node_1, 1*GB); stubQueueAllocation(b, clusterResource, node_1, 1*GB);
root.assignContainers(clusterResource, node_1, root.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
InOrder allocationOrder = inOrder(a, b); InOrder allocationOrder = inOrder(a, b);
allocationOrder.verify(a).assignContainers(eq(clusterResource), allocationOrder.verify(a).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(b).assignContainers(eq(clusterResource), allocationOrder.verify(b).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(a, 2*GB, clusterResource);
verifyQueueMetrics(b, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource);
@ -254,12 +259,12 @@ public class TestParentQueue {
stubQueueAllocation(a, clusterResource, node_0, 1*GB); stubQueueAllocation(a, clusterResource, node_0, 1*GB);
stubQueueAllocation(b, clusterResource, node_0, 2*GB); stubQueueAllocation(b, clusterResource, node_0, 2*GB);
root.assignContainers(clusterResource, node_0, root.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
allocationOrder = inOrder(b, a); allocationOrder = inOrder(b, a);
allocationOrder.verify(b).assignContainers(eq(clusterResource), allocationOrder.verify(b).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(a).assignContainers(eq(clusterResource), allocationOrder.verify(a).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(a, 3*GB, clusterResource);
verifyQueueMetrics(b, 4*GB, clusterResource); verifyQueueMetrics(b, 4*GB, clusterResource);
@ -268,12 +273,12 @@ public class TestParentQueue {
stubQueueAllocation(a, clusterResource, node_0, 0*GB); stubQueueAllocation(a, clusterResource, node_0, 0*GB);
stubQueueAllocation(b, clusterResource, node_0, 4*GB); stubQueueAllocation(b, clusterResource, node_0, 4*GB);
root.assignContainers(clusterResource, node_0, root.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
allocationOrder = inOrder(b, a); allocationOrder = inOrder(b, a);
allocationOrder.verify(b).assignContainers(eq(clusterResource), allocationOrder.verify(b).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(a).assignContainers(eq(clusterResource), allocationOrder.verify(a).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(a, 3*GB, clusterResource);
verifyQueueMetrics(b, 8*GB, clusterResource); verifyQueueMetrics(b, 8*GB, clusterResource);
@ -282,12 +287,12 @@ public class TestParentQueue {
stubQueueAllocation(a, clusterResource, node_1, 1*GB); stubQueueAllocation(a, clusterResource, node_1, 1*GB);
stubQueueAllocation(b, clusterResource, node_1, 1*GB); stubQueueAllocation(b, clusterResource, node_1, 1*GB);
root.assignContainers(clusterResource, node_1, root.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
allocationOrder = inOrder(a, b); allocationOrder = inOrder(a, b);
allocationOrder.verify(b).assignContainers(eq(clusterResource), allocationOrder.verify(b).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(a).assignContainers(eq(clusterResource), allocationOrder.verify(a).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(a, 4*GB, clusterResource); verifyQueueMetrics(a, 4*GB, clusterResource);
verifyQueueMetrics(b, 9*GB, clusterResource); verifyQueueMetrics(b, 9*GB, clusterResource);
} }
@ -448,16 +453,27 @@ public class TestParentQueue {
// Start testing // Start testing
CSQueue a = queues.get(A); CSQueue a = queues.get(A);
a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
CSQueue b = queues.get(B); CSQueue b = queues.get(B);
b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
CSQueue c = queues.get(C); CSQueue c = queues.get(C);
c.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
CSQueue d = queues.get(D); CSQueue d = queues.get(D);
d.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
CSQueue a1 = queues.get(A1); CSQueue a1 = queues.get(A1);
a1.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
CSQueue a2 = queues.get(A2); CSQueue a2 = queues.get(A2);
a2.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
CSQueue b1 = queues.get(B1); CSQueue b1 = queues.get(B1);
b1.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
CSQueue b2 = queues.get(B2); CSQueue b2 = queues.get(B2);
b2.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
CSQueue b3 = queues.get(B3); CSQueue b3 = queues.get(B3);
b3.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
.incPending(Resources.createResource(1 * GB));
// Simulate C returning a container on node_0 // Simulate C returning a container on node_0
stubQueueAllocation(a, clusterResource, node_0, 0*GB); stubQueueAllocation(a, clusterResource, node_0, 0*GB);
@ -465,7 +481,7 @@ public class TestParentQueue {
stubQueueAllocation(c, clusterResource, node_0, 1*GB); stubQueueAllocation(c, clusterResource, node_0, 1*GB);
stubQueueAllocation(d, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB);
root.assignContainers(clusterResource, node_0, root.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(a, 0*GB, clusterResource);
verifyQueueMetrics(b, 0*GB, clusterResource); verifyQueueMetrics(b, 0*GB, clusterResource);
verifyQueueMetrics(c, 1*GB, clusterResource); verifyQueueMetrics(c, 1*GB, clusterResource);
@ -478,7 +494,7 @@ public class TestParentQueue {
stubQueueAllocation(b2, clusterResource, node_1, 4*GB); stubQueueAllocation(b2, clusterResource, node_1, 4*GB);
stubQueueAllocation(c, clusterResource, node_1, 0*GB); stubQueueAllocation(c, clusterResource, node_1, 0*GB);
root.assignContainers(clusterResource, node_1, root.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(a, 0*GB, clusterResource);
verifyQueueMetrics(b, 4*GB, clusterResource); verifyQueueMetrics(b, 4*GB, clusterResource);
verifyQueueMetrics(c, 1*GB, clusterResource); verifyQueueMetrics(c, 1*GB, clusterResource);
@ -490,14 +506,14 @@ public class TestParentQueue {
stubQueueAllocation(b3, clusterResource, node_0, 2*GB); stubQueueAllocation(b3, clusterResource, node_0, 2*GB);
stubQueueAllocation(c, clusterResource, node_0, 2*GB); stubQueueAllocation(c, clusterResource, node_0, 2*GB);
root.assignContainers(clusterResource, node_0, root.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
InOrder allocationOrder = inOrder(a, c, b); InOrder allocationOrder = inOrder(a, c, b);
allocationOrder.verify(a).assignContainers(eq(clusterResource), allocationOrder.verify(a).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(c).assignContainers(eq(clusterResource), allocationOrder.verify(c).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(b).assignContainers(eq(clusterResource), allocationOrder.verify(b).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(a, 1*GB, clusterResource); verifyQueueMetrics(a, 1*GB, clusterResource);
verifyQueueMetrics(b, 6*GB, clusterResource); verifyQueueMetrics(b, 6*GB, clusterResource);
verifyQueueMetrics(c, 3*GB, clusterResource); verifyQueueMetrics(c, 3*GB, clusterResource);
@ -517,16 +533,16 @@ public class TestParentQueue {
stubQueueAllocation(b1, clusterResource, node_2, 1*GB); stubQueueAllocation(b1, clusterResource, node_2, 1*GB);
stubQueueAllocation(c, clusterResource, node_2, 1*GB); stubQueueAllocation(c, clusterResource, node_2, 1*GB);
root.assignContainers(clusterResource, node_2, root.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
allocationOrder = inOrder(a, a2, a1, b, c); allocationOrder = inOrder(a, a2, a1, b, c);
allocationOrder.verify(a).assignContainers(eq(clusterResource), allocationOrder.verify(a).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(a2).assignContainers(eq(clusterResource), allocationOrder.verify(a2).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(b).assignContainers(eq(clusterResource), allocationOrder.verify(b).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(c).assignContainers(eq(clusterResource), allocationOrder.verify(c).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(a, 3*GB, clusterResource);
verifyQueueMetrics(b, 8*GB, clusterResource); verifyQueueMetrics(b, 8*GB, clusterResource);
verifyQueueMetrics(c, 4*GB, clusterResource); verifyQueueMetrics(c, 4*GB, clusterResource);
@ -622,12 +638,16 @@ public class TestParentQueue {
// Start testing // Start testing
LeafQueue a = (LeafQueue)queues.get(A); LeafQueue a = (LeafQueue)queues.get(A);
LeafQueue b = (LeafQueue)queues.get(B); LeafQueue b = (LeafQueue)queues.get(B);
a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
.incPending(Resources.createResource(1 * GB));
// Simulate B returning a container on node_0 // Simulate B returning a container on node_0
stubQueueAllocation(a, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH); stubQueueAllocation(a, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH); stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
root.assignContainers(clusterResource, node_0, root.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(a, 0*GB, clusterResource);
verifyQueueMetrics(b, 1*GB, clusterResource); verifyQueueMetrics(b, 1*GB, clusterResource);
@ -636,12 +656,12 @@ public class TestParentQueue {
stubQueueAllocation(a, clusterResource, node_1, 2*GB, NodeType.RACK_LOCAL); stubQueueAllocation(a, clusterResource, node_1, 2*GB, NodeType.RACK_LOCAL);
stubQueueAllocation(b, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH); stubQueueAllocation(b, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
root.assignContainers(clusterResource, node_1, root.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
InOrder allocationOrder = inOrder(a, b); InOrder allocationOrder = inOrder(a, b);
allocationOrder.verify(a).assignContainers(eq(clusterResource), allocationOrder.verify(a).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(b).assignContainers(eq(clusterResource), allocationOrder.verify(b).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(a, 2*GB, clusterResource);
verifyQueueMetrics(b, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource);
@ -651,12 +671,12 @@ public class TestParentQueue {
stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL); stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL);
stubQueueAllocation(b, clusterResource, node_0, 2*GB, NodeType.OFF_SWITCH); stubQueueAllocation(b, clusterResource, node_0, 2*GB, NodeType.OFF_SWITCH);
root.assignContainers(clusterResource, node_0, root.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
allocationOrder = inOrder(b, a); allocationOrder = inOrder(b, a);
allocationOrder.verify(b).assignContainers(eq(clusterResource), allocationOrder.verify(b).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(a).assignContainers(eq(clusterResource), allocationOrder.verify(a).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(a, 2*GB, clusterResource);
verifyQueueMetrics(b, 4*GB, clusterResource); verifyQueueMetrics(b, 4*GB, clusterResource);
@ -691,12 +711,19 @@ public class TestParentQueue {
// Start testing // Start testing
LeafQueue b3 = (LeafQueue)queues.get(B3); LeafQueue b3 = (LeafQueue)queues.get(B3);
LeafQueue b2 = (LeafQueue)queues.get(B2); LeafQueue b2 = (LeafQueue)queues.get(B2);
b2.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
b3.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
.incPending(Resources.createResource(1 * GB));
CSQueue b = queues.get(B);
b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
// Simulate B3 returning a container on node_0 // Simulate B3 returning a container on node_0
stubQueueAllocation(b2, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH); stubQueueAllocation(b2, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH); stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
root.assignContainers(clusterResource, node_0, root.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
verifyQueueMetrics(b2, 0*GB, clusterResource); verifyQueueMetrics(b2, 0*GB, clusterResource);
verifyQueueMetrics(b3, 1*GB, clusterResource); verifyQueueMetrics(b3, 1*GB, clusterResource);
@ -705,12 +732,12 @@ public class TestParentQueue {
stubQueueAllocation(b2, clusterResource, node_1, 1*GB, NodeType.RACK_LOCAL); stubQueueAllocation(b2, clusterResource, node_1, 1*GB, NodeType.RACK_LOCAL);
stubQueueAllocation(b3, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH); stubQueueAllocation(b3, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
root.assignContainers(clusterResource, node_1, root.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
InOrder allocationOrder = inOrder(b2, b3); InOrder allocationOrder = inOrder(b2, b3);
allocationOrder.verify(b2).assignContainers(eq(clusterResource), allocationOrder.verify(b2).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(b3).assignContainers(eq(clusterResource), allocationOrder.verify(b3).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(b2, 1*GB, clusterResource); verifyQueueMetrics(b2, 1*GB, clusterResource);
verifyQueueMetrics(b3, 2*GB, clusterResource); verifyQueueMetrics(b3, 2*GB, clusterResource);
@ -720,12 +747,12 @@ public class TestParentQueue {
stubQueueAllocation(b2, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL); stubQueueAllocation(b2, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL);
stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH); stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
root.assignContainers(clusterResource, node_0, root.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
allocationOrder = inOrder(b3, b2); allocationOrder = inOrder(b3, b2);
allocationOrder.verify(b3).assignContainers(eq(clusterResource), allocationOrder.verify(b3).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
allocationOrder.verify(b2).assignContainers(eq(clusterResource), allocationOrder.verify(b2).assignContainers(eq(clusterResource),
any(FiCaSchedulerNode.class), anyResourceLimits()); any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
verifyQueueMetrics(b2, 1*GB, clusterResource); verifyQueueMetrics(b2, 1*GB, clusterResource);
verifyQueueMetrics(b3, 3*GB, clusterResource); verifyQueueMetrics(b3, 3*GB, clusterResource);

View File

@ -48,10 +48,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher; import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@ -266,7 +266,7 @@ public class TestReservations {
// Start testing... // Start testing...
// Only AM // Only AM
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, a.getUsedResources().getMemory());
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -278,7 +278,7 @@ public class TestReservations {
// Only 1 map - simulating reduce // Only 1 map - simulating reduce
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, a.getUsedResources().getMemory());
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -290,7 +290,7 @@ public class TestReservations {
// Only 1 map to other node - simulating reduce // Only 1 map to other node - simulating reduce
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -305,7 +305,7 @@ public class TestReservations {
// try to assign reducer (5G on node 0 and should reserve) // try to assign reducer (5G on node 0 and should reserve)
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(5 * GB, a.getMetrics().getReservedMB()); assertEquals(5 * GB, a.getMetrics().getReservedMB());
@ -321,7 +321,7 @@ public class TestReservations {
// assign reducer to node 2 // assign reducer to node 2
a.assignContainers(clusterResource, node_2, a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(18 * GB, a.getUsedResources().getMemory()); assertEquals(18 * GB, a.getUsedResources().getMemory());
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(5 * GB, a.getMetrics().getReservedMB()); assertEquals(5 * GB, a.getMetrics().getReservedMB());
@ -338,7 +338,7 @@ public class TestReservations {
// node_1 heartbeat and unreserves from node_0 in order to allocate // node_1 heartbeat and unreserves from node_0 in order to allocate
// on node_1 // on node_1
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(18 * GB, a.getUsedResources().getMemory()); assertEquals(18 * GB, a.getUsedResources().getMemory());
assertEquals(18 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(18 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -422,7 +422,7 @@ public class TestReservations {
// Start testing... // Start testing...
// Only AM // Only AM
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, a.getUsedResources().getMemory());
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -434,7 +434,7 @@ public class TestReservations {
// Only 1 map - simulating reduce // Only 1 map - simulating reduce
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, a.getUsedResources().getMemory());
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -446,7 +446,7 @@ public class TestReservations {
// Only 1 map to other node - simulating reduce // Only 1 map to other node - simulating reduce
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -461,7 +461,7 @@ public class TestReservations {
// try to assign reducer (5G on node 0 and should reserve) // try to assign reducer (5G on node 0 and should reserve)
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(5 * GB, a.getMetrics().getReservedMB()); assertEquals(5 * GB, a.getMetrics().getReservedMB());
@ -477,7 +477,7 @@ public class TestReservations {
// assign reducer to node 2 // assign reducer to node 2
a.assignContainers(clusterResource, node_2, a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(18 * GB, a.getUsedResources().getMemory()); assertEquals(18 * GB, a.getUsedResources().getMemory());
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(5 * GB, a.getMetrics().getReservedMB()); assertEquals(5 * GB, a.getMetrics().getReservedMB());
@ -494,7 +494,7 @@ public class TestReservations {
// node_1 heartbeat and won't unreserve from node_0, potentially stuck // node_1 heartbeat and won't unreserve from node_0, potentially stuck
// if AM doesn't handle // if AM doesn't handle
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(18 * GB, a.getUsedResources().getMemory()); assertEquals(18 * GB, a.getUsedResources().getMemory());
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(5 * GB, a.getMetrics().getReservedMB()); assertEquals(5 * GB, a.getMetrics().getReservedMB());
@ -570,7 +570,7 @@ public class TestReservations {
// Start testing... // Start testing...
// Only AM // Only AM
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, a.getUsedResources().getMemory());
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -581,7 +581,7 @@ public class TestReservations {
// Only 1 map - simulating reduce // Only 1 map - simulating reduce
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, a.getUsedResources().getMemory());
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -592,7 +592,7 @@ public class TestReservations {
// Only 1 map to other node - simulating reduce // Only 1 map to other node - simulating reduce
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -606,7 +606,7 @@ public class TestReservations {
// try to assign reducer (5G on node 0 and should reserve) // try to assign reducer (5G on node 0 and should reserve)
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(5 * GB, a.getMetrics().getReservedMB()); assertEquals(5 * GB, a.getMetrics().getReservedMB());
@ -621,7 +621,7 @@ public class TestReservations {
// could allocate but told need to unreserve first // could allocate but told need to unreserve first
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, a.getUsedResources().getMemory());
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -823,7 +823,7 @@ public class TestReservations {
// Start testing... // Start testing...
// Only AM // Only AM
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, a.getUsedResources().getMemory());
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -834,7 +834,7 @@ public class TestReservations {
// Only 1 map - simulating reduce // Only 1 map - simulating reduce
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, a.getUsedResources().getMemory());
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -845,7 +845,7 @@ public class TestReservations {
// Only 1 map to other node - simulating reduce // Only 1 map to other node - simulating reduce
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -860,15 +860,16 @@ public class TestReservations {
Resource capability = Resources.createResource(32 * GB, 0); Resource capability = Resources.createResource(32 * GB, 0);
boolean res = boolean res =
a.canAssignToThisQueue(clusterResource, a.canAssignToThisQueue(clusterResource,
CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
clusterResource), capability, Resources.none()); clusterResource), capability, Resources.none(),
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertFalse(res); assertFalse(res);
// now add in reservations and make sure it continues if config set // now add in reservations and make sure it continues if config set
// allocate to queue so that the potential new capacity is greater then // allocate to queue so that the potential new capacity is greater then
// absoluteMaxCapacity // absoluteMaxCapacity
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(5 * GB, a.getMetrics().getReservedMB()); assertEquals(5 * GB, a.getMetrics().getReservedMB());
@ -881,16 +882,17 @@ public class TestReservations {
capability = Resources.createResource(5 * GB, 0); capability = Resources.createResource(5 * GB, 0);
res = res =
a.canAssignToThisQueue(clusterResource, a.canAssignToThisQueue(clusterResource,
CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
clusterResource), capability, Resources clusterResource), capability, Resources.createResource(5 * GB),
.createResource(5 * GB)); SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertTrue(res); assertTrue(res);
// tell to not check reservations // tell to not check reservations
res = res =
a.canAssignToThisQueue(clusterResource, a.canAssignToThisQueue(clusterResource,
CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
clusterResource), capability, Resources.none()); clusterResource), capability, Resources.none(),
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertFalse(res); assertFalse(res);
refreshQueuesTurnOffReservationsContLook(a, csConf); refreshQueuesTurnOffReservationsContLook(a, csConf);
@ -899,15 +901,16 @@ public class TestReservations {
// in since feature is off // in since feature is off
res = res =
a.canAssignToThisQueue(clusterResource, a.canAssignToThisQueue(clusterResource,
CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
clusterResource), capability, Resources.none()); clusterResource), capability, Resources.none(),
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertFalse(res); assertFalse(res);
res = res =
a.canAssignToThisQueue(clusterResource, a.canAssignToThisQueue(clusterResource,
CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( RMNodeLabelsManager.NO_LABEL, new ResourceLimits(
clusterResource), capability, Resources clusterResource), capability, Resources.createResource(5 * GB),
.createResource(5 * GB)); SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertFalse(res); assertFalse(res);
} }
@ -1008,7 +1011,7 @@ public class TestReservations {
// Start testing... // Start testing...
// Only AM // Only AM
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, a.getUsedResources().getMemory());
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -1019,7 +1022,7 @@ public class TestReservations {
// Only 1 map - simulating reduce // Only 1 map - simulating reduce
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, a.getUsedResources().getMemory());
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -1030,7 +1033,7 @@ public class TestReservations {
// Only 1 map to other node - simulating reduce // Only 1 map to other node - simulating reduce
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -1044,7 +1047,7 @@ public class TestReservations {
// allocate to queue so that the potential new capacity is greater then // allocate to queue so that the potential new capacity is greater then
// absoluteMaxCapacity // absoluteMaxCapacity
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(5 * GB, app_0.getCurrentReservation().getMemory()); assertEquals(5 * GB, app_0.getCurrentReservation().getMemory());
@ -1059,18 +1062,18 @@ public class TestReservations {
// set limit so subtrace reservations it can continue // set limit so subtrace reservations it can continue
Resource limit = Resources.createResource(12 * GB, 0); Resource limit = Resources.createResource(12 * GB, 0);
boolean res = a.canAssignToUser(clusterResource, user_0, limit, app_0, boolean res = a.canAssignToUser(clusterResource, user_0, limit, app_0,
true, null); true, "");
assertTrue(res); assertTrue(res);
// tell it not to check for reservations and should fail as already over // tell it not to check for reservations and should fail as already over
// limit // limit
res = a.canAssignToUser(clusterResource, user_0, limit, app_0, false, null); res = a.canAssignToUser(clusterResource, user_0, limit, app_0, false, "");
assertFalse(res); assertFalse(res);
refreshQueuesTurnOffReservationsContLook(a, csConf); refreshQueuesTurnOffReservationsContLook(a, csConf);
// should now return false since feature off // should now return false since feature off
res = a.canAssignToUser(clusterResource, user_0, limit, app_0, true, null); res = a.canAssignToUser(clusterResource, user_0, limit, app_0, true, "");
assertFalse(res); assertFalse(res);
} }
@ -1143,7 +1146,7 @@ public class TestReservations {
// Start testing... // Start testing...
// Only AM // Only AM
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, a.getUsedResources().getMemory());
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -1155,7 +1158,7 @@ public class TestReservations {
// Only 1 map - simulating reduce // Only 1 map - simulating reduce
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, a.getUsedResources().getMemory());
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -1167,7 +1170,7 @@ public class TestReservations {
// Only 1 map to other node - simulating reduce // Only 1 map to other node - simulating reduce
a.assignContainers(clusterResource, node_1, a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -1183,7 +1186,7 @@ public class TestReservations {
// some resource. Even with continous reservation looking, we don't allow // some resource. Even with continous reservation looking, we don't allow
// unreserve resource to reserve container. // unreserve resource to reserve container.
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(Resources.createResource(10 * GB))); new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -1199,7 +1202,7 @@ public class TestReservations {
// used (8G) + required (5G). It will not reserved since it has to unreserve // used (8G) + required (5G). It will not reserved since it has to unreserve
// some resource. Unfortunately, there's nothing to unreserve. // some resource. Unfortunately, there's nothing to unreserve.
a.assignContainers(clusterResource, node_2, a.assignContainers(clusterResource, node_2,
new ResourceLimits(Resources.createResource(10 * GB))); new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, a.getUsedResources().getMemory());
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -1213,7 +1216,7 @@ public class TestReservations {
// let it assign 5G to node_2 // let it assign 5G to node_2
a.assignContainers(clusterResource, node_2, a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, a.getUsedResources().getMemory());
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(0 * GB, a.getMetrics().getReservedMB()); assertEquals(0 * GB, a.getMetrics().getReservedMB());
@ -1226,7 +1229,7 @@ public class TestReservations {
// reserve 8G node_0 // reserve 8G node_0
a.assignContainers(clusterResource, node_0, a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(21 * GB, a.getUsedResources().getMemory()); assertEquals(21 * GB, a.getUsedResources().getMemory());
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(8 * GB, a.getMetrics().getReservedMB()); assertEquals(8 * GB, a.getMetrics().getReservedMB());
@ -1241,7 +1244,7 @@ public class TestReservations {
// continued to try due to having reservation above, // continued to try due to having reservation above,
// but hits queue limits so can't reserve anymore. // but hits queue limits so can't reserve anymore.
a.assignContainers(clusterResource, node_2, a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource)); new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertEquals(21 * GB, a.getUsedResources().getMemory()); assertEquals(21 * GB, a.getUsedResources().getMemory());
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
assertEquals(8 * GB, a.getMetrics().getReservedMB()); assertEquals(8 * GB, a.getMetrics().getReservedMB());

View File

@ -160,6 +160,7 @@ public class TestUtils {
request.setCapability(capability); request.setCapability(capability);
request.setRelaxLocality(relaxLocality); request.setRelaxLocality(relaxLocality);
request.setPriority(priority); request.setPriority(priority);
request.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL);
return request; return request;
} }
@ -273,6 +274,7 @@ public class TestUtils {
conf.setCapacity(B1, 100); conf.setCapacity(B1, 100);
conf.setMaximumCapacity(B1, 100); conf.setMaximumCapacity(B1, 100);
conf.setCapacityByLabel(B1, "y", 100); conf.setCapacityByLabel(B1, "y", 100);
conf.setMaximumApplicationMasterResourcePerQueuePercent(B1, 1f);
final String C1 = C + ".c1"; final String C1 = C + ".c1";
conf.setQueues(C, new String[] {"c1"}); conf.setQueues(C, new String[] {"c1"});