YARN-3361. CapacityScheduler side changes to support non-exclusive node labels. Contributed by Wangda Tan
This commit is contained in:
parent
b46ee1e7a3
commit
0fefda645b
|
@ -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
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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()
|
||||||
|
|
|
@ -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="
|
||||||
|
|
|
@ -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
|
||||||
|
}
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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());
|
||||||
|
|
|
@ -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());
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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");
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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);
|
||||||
|
|
||||||
|
|
|
@ -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());
|
||||||
|
|
|
@ -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"});
|
||||||
|
|
Loading…
Reference in New Issue