YARN-7574. Add support for Node Labels on Auto Created Leaf Queue Template. Contributed by Suma Shivaprasad.

(cherry picked from commit 821b0de4c5)
This commit is contained in:
Sunil G 2018-04-09 21:17:22 +05:30
parent 091db4d0eb
commit 8311fcc75b
17 changed files with 831 additions and 441 deletions

View File

@ -236,13 +236,14 @@ public class RMServerUtils {
*/ */
public static void normalizeAndValidateRequests(List<ResourceRequest> ask, public static void normalizeAndValidateRequests(List<ResourceRequest> ask,
Resource maximumResource, String queueName, YarnScheduler scheduler, Resource maximumResource, String queueName, YarnScheduler scheduler,
RMContext rmContext) RMContext rmContext) throws InvalidResourceRequestException {
throws InvalidResourceRequestException {
// Get queue from scheduler // Get queue from scheduler
QueueInfo queueInfo = null; QueueInfo queueInfo = null;
try { try {
queueInfo = scheduler.getQueueInfo(queueName, false, false); queueInfo = scheduler.getQueueInfo(queueName, false, false);
} catch (IOException e) { } catch (IOException e) {
//Queue may not exist since it could be auto-created in case of
// dynamic queues
} }
for (ResourceRequest resReq : ask) { for (ResourceRequest resReq : ask) {

View File

@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
@ -75,6 +76,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistManager; import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistManager;
import org.apache.hadoop.yarn.server.resourcemanager.blacklist.DisabledBlacklistManager; import org.apache.hadoop.yarn.server.resourcemanager.blacklist.DisabledBlacklistManager;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels
.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable; import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
@ -1109,6 +1113,49 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
amBlacklist.getBlacklistAdditions() + ") and removals(" + amBlacklist.getBlacklistAdditions() + ") and removals(" +
amBlacklist.getBlacklistRemovals() + ")"); amBlacklist.getBlacklistRemovals() + ")");
} }
QueueInfo queueInfo = null;
for (ResourceRequest amReq : appAttempt.amReqs) {
if (amReq.getNodeLabelExpression() == null && ResourceRequest.ANY
.equals(amReq.getResourceName())) {
String queue = appAttempt.rmApp.getQueue();
//Load queue only once since queue will be same across attempts
if (queueInfo == null) {
try {
queueInfo = appAttempt.scheduler.getQueueInfo(queue, false,
false);
} catch (IOException e) {
LOG.error("Could not find queue for application : ", e);
// Set application status to REJECTED since we cant find the
// queue
appAttempt.rmContext.getDispatcher().getEventHandler().handle(
new RMAppAttemptEvent(appAttempt.getAppAttemptId(),
RMAppAttemptEventType.FAIL,
"Could not find queue for application : " +
appAttempt.rmApp.getQueue()));
appAttempt.rmContext.getDispatcher().getEventHandler().handle(
new RMAppEvent(appAttempt.rmApp.getApplicationId(), RMAppEventType
.APP_REJECTED,
"Could not find queue for application : " +
appAttempt.rmApp.getQueue()));
return RMAppAttemptState.FAILED;
}
}
String labelExp = RMNodeLabelsManager.NO_LABEL;
if (queueInfo != null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Setting default node label expression : " + queueInfo
.getDefaultNodeLabelExpression());
}
labelExp = queueInfo.getDefaultNodeLabelExpression();
}
amReq.setNodeLabelExpression(labelExp);
}
}
// AM resource has been checked when submission // AM resource has been checked when submission
Allocation amContainerAllocation = Allocation amContainerAllocation =
appAttempt.scheduler.allocate( appAttempt.scheduler.allocate(

View File

@ -132,4 +132,16 @@ public class Allocation {
public void setResourceLimit(Resource resource) { public void setResourceLimit(Resource resource) {
this.resourceLimit = resource; this.resourceLimit = resource;
} }
@Override
public String toString() {
return "Allocation{" + "containers=" + containers + ", strictContainers="
+ strictContainers + ", fungibleContainers=" + fungibleContainers
+ ", fungibleResources=" + fungibleResources + ", nmTokens=" + nmTokens
+ ", increasedContainers=" + increasedContainers
+ ", decreasedContainers=" + decreasedContainers
+ ", promotedContainers=" + promotedContainers + ", demotedContainers="
+ demotedContainers + ", previousAttemptContainers="
+ previousAttemptContainers + ", resourceLimit=" + resourceLimit + '}';
}
} }

View File

@ -186,19 +186,33 @@ public class SchedulerUtils {
ResourceRequest resReq, QueueInfo queueInfo) { ResourceRequest resReq, QueueInfo queueInfo) {
String labelExp = resReq.getNodeLabelExpression(); String labelExp = resReq.getNodeLabelExpression();
if (LOG.isDebugEnabled()) {
LOG.debug("Requested Node Label Expression : " + labelExp);
LOG.debug("Queue Info : " + queueInfo);
}
// if queue has default label expression, and RR doesn't have, use the // if queue has default label expression, and RR doesn't have, use the
// default label expression of queue // default label expression of queue
if (labelExp == null && queueInfo != null && ResourceRequest.ANY if (labelExp == null && queueInfo != null && ResourceRequest.ANY
.equals(resReq.getResourceName())) { .equals(resReq.getResourceName())) {
if ( LOG.isDebugEnabled()) {
LOG.debug("Setting default node label expression : " + queueInfo
.getDefaultNodeLabelExpression());
}
labelExp = queueInfo.getDefaultNodeLabelExpression(); labelExp = queueInfo.getDefaultNodeLabelExpression();
} }
// If labelExp still equals to null, set it to be NO_LABEL // If labelExp still equals to null, it could either be a dynamic queue
if (labelExp == null) { // or the label is not configured
// set it to be NO_LABEL in case of a pre-configured queue. Dynamic
// queues are handled in RMAppAttemptImp.ScheduledTransition
if (labelExp == null && queueInfo != null) {
labelExp = RMNodeLabelsManager.NO_LABEL; labelExp = RMNodeLabelsManager.NO_LABEL;
} }
resReq.setNodeLabelExpression(labelExp);
if ( labelExp != null) {
resReq.setNodeLabelExpression(labelExp);
}
} }
public static void normalizeAndValidateRequest(ResourceRequest resReq, public static void normalizeAndValidateRequest(ResourceRequest resReq,
@ -209,6 +223,7 @@ public class SchedulerUtils {
isRecovery, rmContext, null); isRecovery, rmContext, null);
} }
public static void normalizeAndValidateRequest(ResourceRequest resReq, public static void normalizeAndValidateRequest(ResourceRequest resReq,
Resource maximumResource, String queueName, YarnScheduler scheduler, Resource maximumResource, String queueName, YarnScheduler scheduler,
boolean isRecovery, RMContext rmContext, QueueInfo queueInfo) boolean isRecovery, RMContext rmContext, QueueInfo queueInfo)
@ -233,11 +248,12 @@ public class SchedulerUtils {
try { try {
queueInfo = scheduler.getQueueInfo(queueName, false, false); queueInfo = scheduler.getQueueInfo(queueName, false, false);
} catch (IOException e) { } catch (IOException e) {
// it is possible queue cannot get when queue mapping is set, just ignore //Queue may not exist since it could be auto-created in case of
// the queueInfo here, and move forward // dynamic queues
} }
} }
SchedulerUtils.normalizeNodeLabelExpressionInRequest(resReq, queueInfo); SchedulerUtils.normalizeNodeLabelExpressionInRequest(resReq, queueInfo);
if (!isRecovery) { if (!isRecovery) {
validateResourceRequest(resReq, maximumResource, queueInfo, rmContext); validateResourceRequest(resReq, maximumResource, queueInfo, rmContext);
} }
@ -245,8 +261,7 @@ public class SchedulerUtils {
public static void normalizeAndvalidateRequest(ResourceRequest resReq, public static void normalizeAndvalidateRequest(ResourceRequest resReq,
Resource maximumResource, String queueName, YarnScheduler scheduler, Resource maximumResource, String queueName, YarnScheduler scheduler,
RMContext rmContext) RMContext rmContext) throws InvalidResourceRequestException {
throws InvalidResourceRequestException {
normalizeAndvalidateRequest(resReq, maximumResource, queueName, scheduler, normalizeAndvalidateRequest(resReq, maximumResource, queueName, scheduler,
rmContext, null); rmContext, null);
} }
@ -296,7 +311,7 @@ public class SchedulerUtils {
+ "resource request has resource name = " + "resource request has resource name = "
+ resReq.getResourceName()); + resReq.getResourceName());
} }
// we don't allow specify label expression with more than one node labels now // we don't allow specify label expression with more than one node labels now
if (labelExp != null && labelExp.contains("&&")) { if (labelExp != null && labelExp.contains("&&")) {
throw new InvalidLabelResourceRequestException( throw new InvalidLabelResourceRequestException(
@ -305,7 +320,7 @@ public class SchedulerUtils {
+ "in a node label expression, node label expression = " + "in a node label expression, node label expression = "
+ labelExp); + labelExp);
} }
if (labelExp != null && !labelExp.trim().isEmpty() && queueInfo != null) { if (labelExp != null && !labelExp.trim().isEmpty() && queueInfo != null) {
if (!checkQueueLabelExpression(queueInfo.getAccessibleNodeLabels(), if (!checkQueueLabelExpression(queueInfo.getAccessibleNodeLabels(),
labelExp, rmContext)) { labelExp, rmContext)) {

View File

@ -148,11 +148,10 @@ public class AutoCreatedLeafQueue extends AbstractAutoCreatedLeafQueue {
try { try {
for( String nodeLabel : parent.getQueueCapacities().getExistingNodeLabels for( String nodeLabel : parent.getQueueCapacities().getExistingNodeLabels
()) { ()) {
//TODO - update to use getMaximumCapacity(nodeLabel) in YARN-7574
setEntitlement(nodeLabel, new QueueEntitlement(0.0f, setEntitlement(nodeLabel, new QueueEntitlement(0.0f,
parent.getLeafQueueTemplate() parent.getLeafQueueTemplate()
.getQueueCapacities() .getQueueCapacities()
.getMaximumCapacity())); .getMaximumCapacity(nodeLabel)));
} }
} catch (SchedulerDynamicEditException e) { } catch (SchedulerDynamicEditException e) {
throw new IOException(e); throw new IOException(e);

View File

@ -18,6 +18,9 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
import java.io.IOException;
import java.util.List; import java.util.List;
public interface AutoCreatedQueueManagementPolicy { public interface AutoCreatedQueueManagementPolicy {
@ -26,14 +29,15 @@ public interface AutoCreatedQueueManagementPolicy {
* Initialize policy * Initialize policy
* @param schedulerContext Capacity Scheduler context * @param schedulerContext Capacity Scheduler context
*/ */
void init(CapacitySchedulerContext schedulerContext, ParentQueue parentQueue); void init(CapacitySchedulerContext schedulerContext, ParentQueue
parentQueue) throws IOException;
/** /**
* Reinitialize policy state ( if required ) * Reinitialize policy state ( if required )
* @param schedulerContext Capacity Scheduler context * @param schedulerContext Capacity Scheduler context
*/ */
void reinitialize(CapacitySchedulerContext schedulerContext, void reinitialize(CapacitySchedulerContext schedulerContext,
ParentQueue parentQueue); ParentQueue parentQueue) throws IOException;
/** /**
* Get initial template for the specified leaf queue * Get initial template for the specified leaf queue
@ -48,6 +52,10 @@ public interface AutoCreatedQueueManagementPolicy {
/** /**
* Compute/Adjust child queue capacities * Compute/Adjust child queue capacities
* for auto created leaf queues * for auto created leaf queues
* This computes queue entitlements but does not update LeafQueueState or
* queue capacities. Scheduler calls commitQueueManagemetChanges after
* validation after applying queue changes and commits to LeafQueueState
* are done in commitQueueManagementChanges.
* *
* @return returns a list of suggested QueueEntitlementChange(s) which may * @return returns a list of suggested QueueEntitlementChange(s) which may
* or may not be be enforced by the scheduler * or may not be be enforced by the scheduler

View File

@ -1167,6 +1167,8 @@ public class CapacityScheduler extends
updateDemandForQueue.getOrderingPolicy().demandUpdated(application); updateDemandForQueue.getOrderingPolicy().demandUpdated(application);
} }
LOG.info("Allocation for application " + applicationAttemptId + " : " +
allocation + " with cluster resource : " + getClusterResource());
return allocation; return allocation;
} }

View File

@ -1829,6 +1829,15 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
setCapacity(leafQueueConfPrefix, val); setCapacity(leafQueueConfPrefix, val);
} }
@VisibleForTesting
@Private
public void setAutoCreatedLeafQueueTemplateCapacityByLabel(String queuePath,
String label, float val) {
String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
queuePath);
setCapacityByLabel(leafQueueConfPrefix, label, val);
}
@Private @Private
@VisibleForTesting @VisibleForTesting
public void setAutoCreatedLeafQueueConfigMaxCapacity(String queuePath, public void setAutoCreatedLeafQueueConfigMaxCapacity(String queuePath,
@ -1838,6 +1847,15 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
setMaximumCapacity(leafQueueConfPrefix, val); setMaximumCapacity(leafQueueConfPrefix, val);
} }
@Private
@VisibleForTesting
public void setAutoCreatedLeafQueueTemplateMaxCapacity(String queuePath,
String label, float val) {
String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
queuePath);
setMaximumCapacityByLabel(leafQueueConfPrefix, label, val);
}
@VisibleForTesting @VisibleForTesting
@Private @Private
public void setAutoCreatedLeafQueueConfigUserLimit(String queuePath, public void setAutoCreatedLeafQueueConfigUserLimit(String queuePath,
@ -1856,6 +1874,16 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
setUserLimitFactor(leafQueueConfPrefix, val); setUserLimitFactor(leafQueueConfPrefix, val);
} }
@Private
@VisibleForTesting
public void setAutoCreatedLeafQueueConfigDefaultNodeLabelExpression(String
queuePath,
String expression) {
String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
queuePath);
setDefaultNodeLabelExpression(leafQueueConfPrefix, expression);
}
public static String getUnits(String resourceValue) { public static String getUnits(String resourceValue) {
String units; String units;
for (int i = 0; i < resourceValue.length(); i++) { for (int i = 0; i < resourceValue.length(); i++) {

View File

@ -778,6 +778,17 @@ public class LeafQueue extends AbstractCSQueue {
metrics.setAMResouceLimit(nodePartition, amResouceLimit); metrics.setAMResouceLimit(nodePartition, amResouceLimit);
queueUsage.setAMLimit(nodePartition, amResouceLimit); queueUsage.setAMLimit(nodePartition, amResouceLimit);
if(LOG.isDebugEnabled()) {
LOG.debug("Queue: " + getQueueName() + ", node label : " +
nodePartition
+ ", queue "
+ "partition "
+ "resource : " + queuePartitionResource + ','
+ " queue current limit : " + queueCurrentLimit + ","
+ " queue partition usable resource : "
+ queuePartitionUsableResource + ","
+ " amResourceLimit : " + amResouceLimit);
}
return amResouceLimit; return amResouceLimit;
} finally { } finally {
writeLock.unlock(); writeLock.unlock();

View File

@ -132,7 +132,7 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
} }
} }
private void initializeQueueManagementPolicy() { private void initializeQueueManagementPolicy() throws IOException {
queueManagementPolicy = queueManagementPolicy =
csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass( csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass(
getQueuePath()); getQueuePath());
@ -140,7 +140,7 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
queueManagementPolicy.init(csContext, this); queueManagementPolicy.init(csContext, this);
} }
private void reinitializeQueueManagementPolicy() { private void reinitializeQueueManagementPolicy() throws IOException {
AutoCreatedQueueManagementPolicy managementPolicy = AutoCreatedQueueManagementPolicy managementPolicy =
csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass( csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass(
getQueuePath()); getQueuePath());
@ -339,6 +339,7 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
((AutoCreatedLeafQueue) childQueue).validateConfigurations(template); ((AutoCreatedLeafQueue) childQueue).validateConfigurations(template);
break; break;
} }
} }
} }

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.MonotonicClock; import org.apache.hadoop.yarn.util.MonotonicClock;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
@ -63,8 +64,6 @@ import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock;
import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
.NO_LABEL;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CSQueueUtils.EPSILON; .capacity.CSQueueUtils.EPSILON;
@ -85,8 +84,6 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
private static final Log LOG = LogFactory.getLog( private static final Log LOG = LogFactory.getLog(
GuaranteedOrZeroCapacityOverTimePolicy.class); GuaranteedOrZeroCapacityOverTimePolicy.class);
private AutoCreatedLeafQueueConfig ZERO_CAPACITY_ENTITLEMENT;
private ReentrantReadWriteLock.WriteLock writeLock; private ReentrantReadWriteLock.WriteLock writeLock;
private ReentrantReadWriteLock.ReadLock readLock; private ReentrantReadWriteLock.ReadLock readLock;
@ -97,12 +94,70 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
private QueueCapacities leafQueueTemplateCapacities; private QueueCapacities leafQueueTemplateCapacities;
private Map<String, LeafQueueState> leafQueueStateMap = new HashMap<>(); private Set<String> leafQueueTemplateNodeLabels;
private LeafQueueState leafQueueState = new LeafQueueState();
private Clock clock = new MonotonicClock(); private Clock clock = new MonotonicClock();
private class LeafQueueState { private class LeafQueueState {
//map of partition-> queueName->{leaf queue's state}
private Map<String, Map<String, LeafQueueStatePerPartition>>
leafQueueStateMap = new HashMap<>();
public boolean containsLeafQueue(String leafQueueName, String partition) {
if (leafQueueStateMap.containsKey(partition)) {
return leafQueueStateMap.get(partition).containsKey(leafQueueName);
}
return false;
}
private boolean containsPartition(String partition) {
if (leafQueueStateMap.containsKey(partition)) {
return true;
}
return false;
}
private boolean addLeafQueueStateIfNotExists(String leafQueueName,
String partition, LeafQueueStatePerPartition leafQueueState) {
if (!containsPartition(partition)) {
leafQueueStateMap.put(partition, new HashMap<>());
}
if (!containsLeafQueue(leafQueueName, partition)) {
leafQueueStateMap.get(partition).put(leafQueueName, leafQueueState);
return true;
}
return false;
}
public boolean createLeafQueueStateIfNotExists(LeafQueue leafQueue,
String partition) {
return addLeafQueueStateIfNotExists(leafQueue.getQueueName(), partition,
new LeafQueueStatePerPartition());
}
public LeafQueueStatePerPartition getLeafQueueStatePerPartition(
String leafQueueName, String partition) {
if (leafQueueStateMap.get(partition) != null) {
return leafQueueStateMap.get(partition).get(leafQueueName);
}
return null;
}
public Map<String, Map<String, LeafQueueStatePerPartition>>
getLeafQueueStateMap() {
return leafQueueStateMap;
}
private void clear() {
leafQueueStateMap.clear();
}
}
private class LeafQueueStatePerPartition {
private AtomicBoolean isActive = new AtomicBoolean(false); private AtomicBoolean isActive = new AtomicBoolean(false);
private long mostRecentActivationTime; private long mostRecentActivationTime;
@ -139,41 +194,16 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
} }
} }
private boolean containsLeafQueue(String leafQueueName) {
return leafQueueStateMap.containsKey(leafQueueName);
}
private boolean addLeafQueueStateIfNotExists(String leafQueueName,
LeafQueueState leafQueueState) {
if (!containsLeafQueue(leafQueueName)) {
leafQueueStateMap.put(leafQueueName, leafQueueState);
return true;
}
return false;
}
private boolean addLeafQueueStateIfNotExists(LeafQueue leafQueue) {
return addLeafQueueStateIfNotExists(leafQueue.getQueueName(),
new LeafQueueState());
}
private void clearLeafQueueState() {
leafQueueStateMap.clear();
}
private class ParentQueueState { private class ParentQueueState {
private Map<String, Float> totalAbsoluteActivatedChildQueueCapacityByLabel = private Map<String, Float> totalAbsoluteActivatedChildQueueCapacityByLabel =
new HashMap<String, Float>(); new HashMap<String, Float>();
private float getAbsoluteActivatedChildQueueCapacity() {
return getAbsoluteActivatedChildQueueCapacity(NO_LABEL);
}
private float getAbsoluteActivatedChildQueueCapacity(String nodeLabel) { private float getAbsoluteActivatedChildQueueCapacity(String nodeLabel) {
try { try {
readLock.lock(); readLock.lock();
Float totalActivatedCapacity = getByLabel(nodeLabel); Float totalActivatedCapacity = getAbsActivatedChildQueueCapacityByLabel(
nodeLabel);
if (totalActivatedCapacity != null) { if (totalActivatedCapacity != null) {
return totalActivatedCapacity; return totalActivatedCapacity;
} else{ } else{
@ -188,11 +218,14 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
float childQueueCapacity) { float childQueueCapacity) {
try { try {
writeLock.lock(); writeLock.lock();
Float activatedChildCapacity = getByLabel(nodeLabel); Float activatedChildCapacity = getAbsActivatedChildQueueCapacityByLabel(
nodeLabel);
if (activatedChildCapacity != null) { if (activatedChildCapacity != null) {
setByLabel(nodeLabel, activatedChildCapacity + childQueueCapacity); setAbsActivatedChildQueueCapacityByLabel(nodeLabel,
activatedChildCapacity + childQueueCapacity);
} else{ } else{
setByLabel(nodeLabel, childQueueCapacity); setAbsActivatedChildQueueCapacityByLabel(nodeLabel,
childQueueCapacity);
} }
} finally { } finally {
writeLock.unlock(); writeLock.unlock();
@ -203,22 +236,25 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
float childQueueCapacity) { float childQueueCapacity) {
try { try {
writeLock.lock(); writeLock.lock();
Float activatedChildCapacity = getByLabel(nodeLabel); Float activatedChildCapacity = getAbsActivatedChildQueueCapacityByLabel(
nodeLabel);
if (activatedChildCapacity != null) { if (activatedChildCapacity != null) {
setByLabel(nodeLabel, activatedChildCapacity - childQueueCapacity); setAbsActivatedChildQueueCapacityByLabel(nodeLabel,
activatedChildCapacity - childQueueCapacity);
} else{ } else{
setByLabel(nodeLabel, childQueueCapacity); setAbsActivatedChildQueueCapacityByLabel(nodeLabel,
childQueueCapacity);
} }
} finally { } finally {
writeLock.unlock(); writeLock.unlock();
} }
} }
Float getByLabel(String label) { Float getAbsActivatedChildQueueCapacityByLabel(String label) {
return totalAbsoluteActivatedChildQueueCapacityByLabel.get(label); return totalAbsoluteActivatedChildQueueCapacityByLabel.get(label);
} }
Float setByLabel(String label, float val) { Float setAbsActivatedChildQueueCapacityByLabel(String label, float val) {
return totalAbsoluteActivatedChildQueueCapacityByLabel.put(label, val); return totalAbsoluteActivatedChildQueueCapacityByLabel.put(label, val);
} }
@ -256,13 +292,12 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
@Override @Override
public void init(final CapacitySchedulerContext schedulerContext, public void init(final CapacitySchedulerContext schedulerContext,
final ParentQueue parentQueue) { final ParentQueue parentQueue) throws IOException {
this.scheduler = schedulerContext; this.scheduler = schedulerContext;
ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
readLock = lock.readLock(); readLock = lock.readLock();
writeLock = lock.writeLock(); writeLock = lock.writeLock();
if (!(parentQueue instanceof ManagedParentQueue)) { if (!(parentQueue instanceof ManagedParentQueue)) {
throw new IllegalArgumentException( throw new IllegalArgumentException(
"Expected instance of type " + ManagedParentQueue.class); "Expected instance of type " + ManagedParentQueue.class);
@ -278,15 +313,43 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
+ leafQueueTemplate.getQueueCapacities() + "]"); + leafQueueTemplate.getQueueCapacities() + "]");
} }
private void initializeLeafQueueTemplate(ManagedParentQueue parentQueue) { private void initializeLeafQueueTemplate(ManagedParentQueue parentQueue)
throws IOException {
leafQueueTemplate = parentQueue.getLeafQueueTemplate(); leafQueueTemplate = parentQueue.getLeafQueueTemplate();
leafQueueTemplateCapacities = leafQueueTemplate.getQueueCapacities(); leafQueueTemplateCapacities = leafQueueTemplate.getQueueCapacities();
ZERO_CAPACITY_ENTITLEMENT = buildTemplate(0.0f, Set<String> parentQueueLabels = parentQueue.getNodeLabelsForQueue();
leafQueueTemplateCapacities.getMaximumCapacity()); for (String nodeLabel : leafQueueTemplateCapacities
.getExistingNodeLabels()) {
if (!parentQueueLabels.contains(nodeLabel)) {
LOG.error("Invalid node label " + nodeLabel
+ " on configured leaf template on parent" + " queue " + parentQueue
.getQueueName());
throw new IOException("Invalid node label " + nodeLabel
+ " on configured leaf template on parent" + " queue " + parentQueue
.getQueueName());
}
}
leafQueueTemplateNodeLabels =
leafQueueTemplateCapacities.getExistingNodeLabels();
} }
/**
* Compute/Adjust child queue capacities
* for auto created leaf queues
* This computes queue entitlements but does not update LeafQueueState or
* queue capacities. Scheduler calls commitQueueManagemetChanges after
* validation after applying queue changes and commits to LeafQueueState
* are done in commitQueueManagementChanges.
*
* @return List of Queue Management change suggestions which could potentially
* be committed/rejected by the scheduler due to validation failures
* @throws SchedulerDynamicEditException
*/
@Override @Override
public List<QueueManagementChange> computeQueueManagementChanges() public List<QueueManagementChange> computeQueueManagementChanges()
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException {
@ -298,70 +361,92 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
try { try {
readLock.lock(); readLock.lock();
List<QueueManagementChange> queueManagementChanges = new ArrayList<>(); List<QueueManagementChange> queueManagementChanges = new ArrayList<>();
List<FiCaSchedulerApp> pendingApps = getSortedPendingApplications();
// check if any leaf queues need to be deactivated based on pending //Map of LeafQueue->QueueCapacities - keep adding the computed
// applications and // entitlements to this map and finally
float parentAbsoluteCapacity = // build the leaf queue configuration Template for all identified leaf
managedParentQueue.getQueueCapacities().getAbsoluteCapacity(); // queues
Map<String, QueueCapacities> leafQueueEntitlements = new HashMap<>();
for (String nodeLabel : leafQueueTemplateNodeLabels) {
// check if any leaf queues need to be deactivated based on pending
// applications
float parentAbsoluteCapacity =
managedParentQueue.getQueueCapacities().getAbsoluteCapacity(
nodeLabel);
float leafQueueTemplateAbsoluteCapacity =
leafQueueTemplateCapacities.getAbsoluteCapacity(nodeLabel);
Map<String, QueueCapacities> deactivatedLeafQueues =
deactivateLeafQueuesIfInActive(managedParentQueue, nodeLabel,
leafQueueEntitlements);
float leafQueueTemplateAbsoluteCapacity = float deactivatedCapacity = getTotalDeactivatedCapacity(
leafQueueTemplateCapacities.getAbsoluteCapacity(); deactivatedLeafQueues, nodeLabel);
Map<String, QueueCapacities> deactivatedLeafQueues =
deactivateLeafQueuesIfInActive(managedParentQueue, queueManagementChanges);
float deactivatedCapacity = getTotalDeactivatedCapacity( float sumOfChildQueueActivatedCapacity = parentQueueState.
deactivatedLeafQueues); getAbsoluteActivatedChildQueueCapacity(nodeLabel);
float sumOfChildQueueActivatedCapacity = parentQueueState. //Check if we need to activate anything at all?
getAbsoluteActivatedChildQueueCapacity(); float availableCapacity =
parentAbsoluteCapacity - sumOfChildQueueActivatedCapacity
+ deactivatedCapacity + EPSILON;
//Check if we need to activate anything at all? if (LOG.isDebugEnabled()) {
float availableCapacity = getAvailableCapacity(parentAbsoluteCapacity, LOG.debug("Parent queue : " + managedParentQueue.getQueueName()
deactivatedCapacity, sumOfChildQueueActivatedCapacity); + ", nodeLabel = " + nodeLabel + ", absCapacity = "
+ parentAbsoluteCapacity + ", leafQueueAbsoluteCapacity = "
+ leafQueueTemplateAbsoluteCapacity + ", deactivatedCapacity = "
+ deactivatedCapacity + " , absChildActivatedCapacity = "
+ sumOfChildQueueActivatedCapacity + ", availableCapacity = "
+ availableCapacity);
}
if (LOG.isDebugEnabled()) { if (availableCapacity >= leafQueueTemplateAbsoluteCapacity) {
LOG.debug( //sort applications across leaf queues by submit time
"Parent queue : " + managedParentQueue.getQueueName() + " absCapacity = " if (pendingApps.size() > 0) {
+ parentAbsoluteCapacity + ", leafQueueAbsoluteCapacity = " int maxLeafQueuesTobeActivated = getMaxLeavesToBeActivated(
+ leafQueueTemplateAbsoluteCapacity + ", deactivatedCapacity = " availableCapacity, leafQueueTemplateAbsoluteCapacity,
+ deactivatedCapacity + " , absChildActivatedCapacity = " pendingApps.size());
+ sumOfChildQueueActivatedCapacity + ", availableCapacity = "
+ availableCapacity);
}
if (availableCapacity >= leafQueueTemplateAbsoluteCapacity) { if (LOG.isDebugEnabled()) {
//sort applications across leaf queues by submit time LOG.debug("Found " + maxLeafQueuesTobeActivated + " leaf queues"
List<FiCaSchedulerApp> pendingApps = getSortedPendingApplications(); + " to be activated with " + pendingApps.size() + " apps ");
}
if (pendingApps.size() > 0) { LinkedHashSet<String> leafQueuesToBeActivated = getSortedLeafQueues(
int maxLeafQueuesTobeActivated = getMaxLeavesToBeActivated( nodeLabel, pendingApps, maxLeafQueuesTobeActivated,
availableCapacity, leafQueueTemplateAbsoluteCapacity, deactivatedLeafQueues.keySet());
pendingApps.size());
if (LOG.isDebugEnabled()) { //Compute entitlement changes for the identified leaf queues
LOG.debug("Found " + maxLeafQueuesTobeActivated // which is appended to the List of computedEntitlements
+ " leaf queues to be activated with " + pendingApps.size() updateLeafQueueCapacitiesByLabel(nodeLabel, leafQueuesToBeActivated,
+ " apps "); leafQueueEntitlements);
}
LinkedHashSet<String> leafQueuesToBeActivated = getSortedLeafQueues( if (LOG.isDebugEnabled()) {
pendingApps, maxLeafQueuesTobeActivated, if (leafQueuesToBeActivated.size() > 0) {
deactivatedLeafQueues.keySet()); LOG.debug("Activated leaf queues : [" + leafQueuesToBeActivated
+ "]");
//Compute entitlement changes for the identified leaf queues }
// which is appended to the List of queueManagementChanges
computeQueueManagementChanges(leafQueuesToBeActivated,
queueManagementChanges, availableCapacity,
leafQueueTemplateAbsoluteCapacity);
if (LOG.isDebugEnabled()) {
if (leafQueuesToBeActivated.size() > 0) {
LOG.debug(
"Activated leaf queues : [" + leafQueuesToBeActivated + "]");
} }
} }
} }
} }
//Populate new entitlements
for (final Iterator<Map.Entry<String, QueueCapacities>> iterator =
leafQueueEntitlements.entrySet().iterator(); iterator.hasNext(); ) {
Map.Entry<String, QueueCapacities> queueCapacities = iterator.next();
String leafQueueName = queueCapacities.getKey();
AutoCreatedLeafQueue leafQueue =
(AutoCreatedLeafQueue) scheduler.getCapacitySchedulerQueueManager()
.getQueue(leafQueueName);
AutoCreatedLeafQueueConfig newTemplate = buildTemplate(
queueCapacities.getValue());
queueManagementChanges.add(
new QueueManagementChange.UpdateQueue(leafQueue, newTemplate));
}
return queueManagementChanges; return queueManagementChanges;
} finally { } finally {
readLock.unlock(); readLock.unlock();
@ -369,14 +454,14 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
} }
private float getTotalDeactivatedCapacity( private float getTotalDeactivatedCapacity(
Map<String, QueueCapacities> deactivatedLeafQueues) { Map<String, QueueCapacities> deactivatedLeafQueues, String nodeLabel) {
float deactivatedCapacity = 0; float deactivatedCapacity = 0;
for (Iterator<Map.Entry<String, QueueCapacities>> iterator = for (Iterator<Map.Entry<String, QueueCapacities>> iterator =
deactivatedLeafQueues.entrySet().iterator(); iterator.hasNext(); ) { deactivatedLeafQueues.entrySet().iterator(); iterator.hasNext(); ) {
Map.Entry<String, QueueCapacities> deactivatedQueueCapacity = Map.Entry<String, QueueCapacities> deactivatedQueueCapacity =
iterator.next(); iterator.next();
deactivatedCapacity += deactivatedCapacity +=
deactivatedQueueCapacity.getValue().getAbsoluteCapacity(); deactivatedQueueCapacity.getValue().getAbsoluteCapacity(nodeLabel);
} }
return deactivatedCapacity; return deactivatedCapacity;
} }
@ -385,20 +470,42 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
void updateLeafQueueState() { void updateLeafQueueState() {
try { try {
writeLock.lock(); writeLock.lock();
Set<String> newPartitions = new HashSet<>();
Set<String> newQueues = new HashSet<>(); Set<String> newQueues = new HashSet<>();
for (CSQueue newQueue : managedParentQueue.getChildQueues()) { for (CSQueue newQueue : managedParentQueue.getChildQueues()) {
if (newQueue instanceof LeafQueue) { if (newQueue instanceof LeafQueue) {
addLeafQueueStateIfNotExists((LeafQueue) newQueue); for (String nodeLabel : leafQueueTemplateNodeLabels) {
leafQueueState.createLeafQueueStateIfNotExists((LeafQueue) newQueue,
nodeLabel);
newPartitions.add(nodeLabel);
}
newQueues.add(newQueue.getQueueName()); newQueues.add(newQueue.getQueueName());
} }
} }
for (Iterator<Map.Entry<String, LeafQueueState>> itr = for (Iterator<Map.Entry<String, Map<String, LeafQueueStatePerPartition>>>
leafQueueStateMap.entrySet().iterator(); itr.hasNext(); ) { itr = leafQueueState.getLeafQueueStateMap().entrySet().iterator();
Map.Entry<String, LeafQueueState> e = itr.next(); itr.hasNext(); ) {
String queueName = e.getKey(); Map.Entry<String, Map<String, LeafQueueStatePerPartition>> e =
if (!newQueues.contains(queueName)) { itr.next();
String partition = e.getKey();
if (!newPartitions.contains(partition)) {
itr.remove(); itr.remove();
LOG.info(
"Removed partition " + partition + " from leaf queue " + "state");
} else{
Map<String, LeafQueueStatePerPartition> queues = e.getValue();
for (
Iterator<Map.Entry<String, LeafQueueStatePerPartition>> queueItr =
queues.entrySet().iterator(); queueItr.hasNext(); ) {
String queue = queueItr.next().getKey();
if (!newQueues.contains(queue)) {
queueItr.remove();
LOG.info("Removed queue " + queue + " from leaf queue "
+ "state from partition " + partition);
}
}
} }
} }
} finally { } finally {
@ -406,22 +513,20 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
} }
} }
private LinkedHashSet<String> getSortedLeafQueues( private LinkedHashSet<String> getSortedLeafQueues(String nodeLabel,
final List<FiCaSchedulerApp> pendingApps, int leafQueuesNeeded, final List<FiCaSchedulerApp> pendingApps, int leafQueuesNeeded,
Set<String> deactivatedQueues) throws SchedulerDynamicEditException { Set<String> deactivatedQueues) throws SchedulerDynamicEditException {
LinkedHashSet<String> leafQueues = new LinkedHashSet<>(leafQueuesNeeded); LinkedHashSet<String> leafQueues = new LinkedHashSet<>(leafQueuesNeeded);
int ctr = 0; int ctr = 0;
for (FiCaSchedulerApp app : pendingApps) { for (FiCaSchedulerApp app : pendingApps) {
AutoCreatedLeafQueue leafQueue = AutoCreatedLeafQueue leafQueue =
(AutoCreatedLeafQueue) app.getCSLeafQueue(); (AutoCreatedLeafQueue) app.getCSLeafQueue();
String leafQueueName = leafQueue.getQueueName(); String leafQueueName = leafQueue.getQueueName();
//Check if leafQueue is not active already and has any pending apps //Check if leafQueue is not active already and has any pending apps
if (ctr < leafQueuesNeeded) { if (ctr < leafQueuesNeeded) {
if (!isActive(leafQueue, nodeLabel)) {
if (!isActive(leafQueue)) {
if (!deactivatedQueues.contains(leafQueueName)) { if (!deactivatedQueues.contains(leafQueueName)) {
if (addLeafQueueIfNotExists(leafQueues, leafQueueName)) { if (addLeafQueueIfNotExists(leafQueues, leafQueueName)) {
ctr++; ctr++;
@ -445,11 +550,12 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
} }
@VisibleForTesting @VisibleForTesting
public boolean isActive(final AutoCreatedLeafQueue leafQueue) public boolean isActive(final AutoCreatedLeafQueue leafQueue,
throws SchedulerDynamicEditException { String nodeLabel) throws SchedulerDynamicEditException {
try { try {
readLock.lock(); readLock.lock();
LeafQueueState leafQueueStatus = getLeafQueueState(leafQueue); LeafQueueStatePerPartition leafQueueStatus = getLeafQueueState(leafQueue,
nodeLabel);
return leafQueueStatus.isActive(); return leafQueueStatus.isActive();
} finally { } finally {
readLock.unlock(); readLock.unlock();
@ -457,64 +563,52 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
} }
private Map<String, QueueCapacities> deactivateLeafQueuesIfInActive( private Map<String, QueueCapacities> deactivateLeafQueuesIfInActive(
ParentQueue parentQueue, ParentQueue parentQueue, String nodeLabel,
List<QueueManagementChange> queueManagementChanges) Map<String, QueueCapacities> leafQueueEntitlements)
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException {
Map<String, QueueCapacities> deactivatedQueues = new HashMap<>(); Map<String, QueueCapacities> deactivatedQueues = new HashMap<>();
for (CSQueue childQueue : parentQueue.getChildQueues()) { for (CSQueue childQueue : parentQueue.getChildQueues()) {
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue; AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue;
if (leafQueue != null) {
if (isActive(leafQueue, nodeLabel) && !hasPendingApps(leafQueue)) {
if (!leafQueueEntitlements.containsKey(leafQueue.getQueueName())) {
leafQueueEntitlements.put(leafQueue.getQueueName(),
new QueueCapacities(false));
}
if (isActive(leafQueue) && !hasPendingApps(leafQueue)) { QueueCapacities capacities = leafQueueEntitlements.get(
queueManagementChanges.add( leafQueue.getQueueName());
new QueueManagementChange.UpdateQueue(leafQueue, updateToZeroCapacity(capacities, nodeLabel);
ZERO_CAPACITY_ENTITLEMENT)); deactivatedQueues.put(leafQueue.getQueueName(),
deactivatedQueues.put(leafQueue.getQueueName(), leafQueueTemplateCapacities);
leafQueueTemplateCapacities); } else{
} else{ if (LOG.isDebugEnabled()) {
if (LOG.isDebugEnabled()) { LOG.debug(" Leaf queue has pending applications or is " + "inactive"
LOG.debug(" Leaf queue has pending applications : " + leafQueue + " : " + leafQueue.getNumApplications()
.getNumApplications() + ".Skipping deactivation for " + ".Skipping deactivation for " + leafQueue);
+ leafQueue); }
} }
} else{
LOG.warn("Could not find queue in scheduler while trying" + " to "
+ "deactivate for " + parentQueue);
} }
} }
if (LOG.isDebugEnabled()) {
if (deactivatedQueues.size() > 0) {
LOG.debug("Deactivated leaf queues : " + deactivatedQueues);
}
}
return deactivatedQueues; return deactivatedQueues;
} }
private void computeQueueManagementChanges( private void updateLeafQueueCapacitiesByLabel(String nodeLabel,
Set<String> leafQueuesToBeActivated, Set<String> leafQueuesToBeActivated,
List<QueueManagementChange> queueManagementChanges, Map<String, QueueCapacities> leafQueueEntitlements) {
final float availableCapacity,
final float leafQueueTemplateAbsoluteCapacity) {
float curAvailableCapacity = availableCapacity;
for (String curLeafQueue : leafQueuesToBeActivated) { for (String curLeafQueue : leafQueuesToBeActivated) {
// Activate queues if capacity is available if (!leafQueueEntitlements.containsKey(curLeafQueue)) {
if (curAvailableCapacity >= leafQueueTemplateAbsoluteCapacity) { leafQueueEntitlements.put(curLeafQueue, new QueueCapacities(false));
AutoCreatedLeafQueue leafQueue = // Activate queues if capacity is available
(AutoCreatedLeafQueue) scheduler.getCapacitySchedulerQueueManager()
.getQueue(curLeafQueue);
if (leafQueue != null) {
AutoCreatedLeafQueueConfig newTemplate = buildTemplate(
leafQueueTemplateCapacities.getCapacity(),
leafQueueTemplateCapacities.getMaximumCapacity());
queueManagementChanges.add(
new QueueManagementChange.UpdateQueue(leafQueue, newTemplate));
curAvailableCapacity -= leafQueueTemplateAbsoluteCapacity;
} else{
LOG.warn(
"Could not find queue in scheduler while trying to deactivate "
+ curLeafQueue);
}
} }
QueueCapacities capacities = leafQueueEntitlements.get(curLeafQueue);
updateCapacityFromTemplate(capacities, nodeLabel);
} }
} }
@ -528,17 +622,8 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
availableCapacity / childQueueAbsoluteCapacity); availableCapacity / childQueueAbsoluteCapacity);
return Math.min(numLeafQueuesNeeded, numPendingApps); return Math.min(numLeafQueuesNeeded, numPendingApps);
} else{
throw new SchedulerDynamicEditException("Child queue absolute capacity "
+ "is initialized to 0. Check parent queue's " + managedParentQueue
.getQueueName() + " leaf queue template configuration");
} }
} return 0;
private float getAvailableCapacity(float parentAbsCapacity,
float deactivatedAbsCapacity, float totalChildQueueActivatedCapacity) {
return parentAbsCapacity - totalChildQueueActivatedCapacity
+ deactivatedAbsCapacity + EPSILON;
} }
/** /**
@ -567,25 +652,27 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) queue; AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) queue;
if (updatedQueueTemplate.getQueueCapacities().getCapacity() > 0) { for (String nodeLabel : updatedQueueTemplate.getQueueCapacities()
if (isActive(leafQueue)) { .getExistingNodeLabels()) {
if (LOG.isDebugEnabled()) { if (updatedQueueTemplate.getQueueCapacities().
LOG.debug( getCapacity(nodeLabel) > 0) {
"Queue is already active. Skipping activation : " + queue if (isActive(leafQueue, nodeLabel)) {
.getQueuePath()); if (LOG.isDebugEnabled()) {
LOG.debug("Queue is already active." + " Skipping activation : "
+ queue.getQueuePath());
}
} else{
activate(leafQueue, nodeLabel);
} }
} else{ } else{
activate(leafQueue); if (!isActive(leafQueue, nodeLabel)) {
} if (LOG.isDebugEnabled()) {
} else{ LOG.debug("Queue is already de-activated. Skipping "
if (!isActive(leafQueue)) { + "de-activation : " + leafQueue.getQueuePath());
if (LOG.isDebugEnabled()) { }
LOG.debug( } else{
"Queue is already de-activated. " + "Skipping de-activation " deactivate(leafQueue, nodeLabel);
+ ": " + leafQueue.getQueuePath());
} }
} else{
deactivate(leafQueue);
} }
} }
} }
@ -594,30 +681,26 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
} }
} }
private void activate(final AutoCreatedLeafQueue leafQueue) private void activate(final AbstractAutoCreatedLeafQueue leafQueue,
throws SchedulerDynamicEditException { String nodeLabel) throws SchedulerDynamicEditException {
try { try {
writeLock.lock(); writeLock.lock();
getLeafQueueState(leafQueue).activate(); getLeafQueueState(leafQueue, nodeLabel).activate();
parentQueueState.incAbsoluteActivatedChildCapacity(nodeLabel,
parentQueueState.incAbsoluteActivatedChildCapacity(NO_LABEL, leafQueueTemplateCapacities.getAbsoluteCapacity(nodeLabel));
leafQueueTemplateCapacities.getAbsoluteCapacity());
} finally { } finally {
writeLock.unlock(); writeLock.unlock();
} }
} }
private void deactivate(final AutoCreatedLeafQueue leafQueue) private void deactivate(final AbstractAutoCreatedLeafQueue leafQueue,
throws SchedulerDynamicEditException { String nodeLabel) throws SchedulerDynamicEditException {
try { try {
writeLock.lock(); writeLock.lock();
getLeafQueueState(leafQueue).deactivate(); getLeafQueueState(leafQueue, nodeLabel).deactivate();
for (String nodeLabel : managedParentQueue.getQueueCapacities() parentQueueState.decAbsoluteActivatedChildCapacity(nodeLabel,
.getExistingNodeLabels()) { leafQueueTemplateCapacities.getAbsoluteCapacity(nodeLabel));
parentQueueState.decAbsoluteActivatedChildCapacity(nodeLabel,
leafQueueTemplateCapacities.getAbsoluteCapacity());
}
} finally { } finally {
writeLock.unlock(); writeLock.unlock();
} }
@ -629,7 +712,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
@Override @Override
public void reinitialize(CapacitySchedulerContext schedulerContext, public void reinitialize(CapacitySchedulerContext schedulerContext,
final ParentQueue parentQueue) { final ParentQueue parentQueue) throws IOException {
if (!(parentQueue instanceof ManagedParentQueue)) { if (!(parentQueue instanceof ManagedParentQueue)) {
throw new IllegalStateException( throw new IllegalStateException(
"Expected instance of type " + ManagedParentQueue.class + " found " "Expected instance of type " + ManagedParentQueue.class + " found "
@ -649,12 +732,11 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
//clear state //clear state
parentQueueState.clear(); parentQueueState.clear();
clearLeafQueueState(); leafQueueState.clear();
LOG.info( LOG.info(
"Reinitialized queue management policy for parent queue " "Reinitialized queue management policy for parent queue " + parentQueue
+ parentQueue.getQueueName() +" with leaf queue template " .getQueueName() + " with leaf queue template " + "capacities : ["
+ "capacities : ["
+ leafQueueTemplate.getQueueCapacities() + "]"); + leafQueueTemplate.getQueueCapacities() + "]");
} }
@ -663,51 +745,74 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
AbstractAutoCreatedLeafQueue leafQueue) AbstractAutoCreatedLeafQueue leafQueue)
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException {
if ( !(leafQueue instanceof AutoCreatedLeafQueue)) { AutoCreatedLeafQueueConfig template;
throw new SchedulerDynamicEditException("Not an instance of "
+ "AutoCreatedLeafQueue : " + leafQueue.getClass()); if (!(leafQueue instanceof AutoCreatedLeafQueue)) {
throw new SchedulerDynamicEditException(
"Not an instance of " + "AutoCreatedLeafQueue : " + leafQueue
.getClass());
} }
AutoCreatedLeafQueue autoCreatedLeafQueue =
(AutoCreatedLeafQueue) leafQueue;
AutoCreatedLeafQueueConfig template = ZERO_CAPACITY_ENTITLEMENT;
try { try {
writeLock.lock(); writeLock.lock();
if (!addLeafQueueStateIfNotExists(leafQueue)) {
LOG.error("Leaf queue already exists in state : " + getLeafQueueState( QueueCapacities capacities = new QueueCapacities(false);
leafQueue)); for (String nodeLabel : leafQueueTemplateNodeLabels) {
throw new SchedulerDynamicEditException( if (!leafQueueState.createLeafQueueStateIfNotExists(leafQueue,
"Leaf queue already exists in state : " + getLeafQueueState( nodeLabel)) {
leafQueue)); String message =
"Leaf queue already exists in state : " + getLeafQueueState(
leafQueue, nodeLabel);
LOG.error(message);
}
float availableCapacity = managedParentQueue.getQueueCapacities().
getAbsoluteCapacity(nodeLabel) - parentQueueState.
getAbsoluteActivatedChildQueueCapacity(nodeLabel) + EPSILON;
if (availableCapacity >= leafQueueTemplateCapacities
.getAbsoluteCapacity(nodeLabel)) {
updateCapacityFromTemplate(capacities, nodeLabel);
activate(leafQueue, nodeLabel);
} else{
updateToZeroCapacity(capacities, nodeLabel);
}
} }
float availableCapacity = getAvailableCapacity( template = buildTemplate(capacities);
managedParentQueue.getQueueCapacities().getAbsoluteCapacity(), 0,
parentQueueState.getAbsoluteActivatedChildQueueCapacity());
if (availableCapacity >= leafQueueTemplateCapacities
.getAbsoluteCapacity()) {
activate(autoCreatedLeafQueue);
template = buildTemplate(leafQueueTemplateCapacities.getCapacity(),
leafQueueTemplateCapacities.getMaximumCapacity());
}
} finally { } finally {
writeLock.unlock(); writeLock.unlock();
} }
return template; return template;
} }
private void updateToZeroCapacity(QueueCapacities capacities,
String nodeLabel) {
capacities.setCapacity(nodeLabel, 0.0f);
capacities.setMaximumCapacity(nodeLabel,
leafQueueTemplateCapacities.getMaximumCapacity(nodeLabel));
}
private void updateCapacityFromTemplate(QueueCapacities capacities,
String nodeLabel) {
capacities.setCapacity(nodeLabel,
leafQueueTemplateCapacities.getCapacity(nodeLabel));
capacities.setMaximumCapacity(nodeLabel,
leafQueueTemplateCapacities.getMaximumCapacity(nodeLabel));
}
@VisibleForTesting @VisibleForTesting
LeafQueueState getLeafQueueState(LeafQueue queue) LeafQueueStatePerPartition getLeafQueueState(LeafQueue queue,
throws SchedulerDynamicEditException { String partition) throws SchedulerDynamicEditException {
try { try {
readLock.lock(); readLock.lock();
String queueName = queue.getQueueName(); String queueName = queue.getQueueName();
if (!containsLeafQueue(queueName)) { if (!leafQueueState.containsLeafQueue(queueName, partition)) {
throw new SchedulerDynamicEditException( throw new SchedulerDynamicEditException(
"Could not find leaf queue in " + "state " + queueName); "Could not find leaf queue in " + "state " + queueName);
} else{ } else{
return leafQueueStateMap.get(queueName); return leafQueueState.
getLeafQueueStatePerPartition(queueName, partition);
} }
} finally { } finally {
readLock.unlock(); readLock.unlock();
@ -715,8 +820,8 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
} }
@VisibleForTesting @VisibleForTesting
public float getAbsoluteActivatedChildQueueCapacity() { public float getAbsoluteActivatedChildQueueCapacity(String nodeLabel) {
return parentQueueState.getAbsoluteActivatedChildQueueCapacity(); return parentQueueState.getAbsoluteActivatedChildQueueCapacity(nodeLabel);
} }
private List<FiCaSchedulerApp> getSortedPendingApplications() { private List<FiCaSchedulerApp> getSortedPendingApplications() {
@ -726,20 +831,10 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
return apps; return apps;
} }
private AutoCreatedLeafQueueConfig buildTemplate(float capacity, private AutoCreatedLeafQueueConfig buildTemplate(QueueCapacities capacities) {
float maxCapacity) {
AutoCreatedLeafQueueConfig.Builder templateBuilder = AutoCreatedLeafQueueConfig.Builder templateBuilder =
new AutoCreatedLeafQueueConfig.Builder(); new AutoCreatedLeafQueueConfig.Builder();
QueueCapacities capacities = new QueueCapacities(false);
templateBuilder.capacities(capacities); templateBuilder.capacities(capacities);
for (String nodeLabel : managedParentQueue.getQueueCapacities()
.getExistingNodeLabels()) {
capacities.setCapacity(nodeLabel, capacity);
capacities.setMaximumCapacity(nodeLabel, maxCapacity);
}
return new AutoCreatedLeafQueueConfig(templateBuilder); return new AutoCreatedLeafQueueConfig(templateBuilder);
} }
} }

View File

@ -62,4 +62,12 @@ public class PendingAskUpdateResult {
public String getNewNodePartition() { public String getNewNodePartition() {
return newNodePartition; return newNodePartition;
} }
@Override
public String toString() {
return "PendingAskUpdateResult{" + "lastPendingAsk=" + lastPendingAsk
+ ", lastNodePartition='" + lastNodePartition + '\''
+ ", newPendingAsk=" + newPendingAsk + ", newNodePartition='"
+ newNodePartition + '\'' + '}';
}
} }

View File

@ -24,6 +24,7 @@ import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@ -33,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
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.Resource; 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.server.api.protocolrecords.NMContainerStatus; import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
@ -65,6 +67,7 @@ public class MockNM {
new HashMap<ContainerId, ContainerStatus>(); new HashMap<ContainerId, ContainerStatus>();
private Map<ApplicationId, AppCollectorData> registeringCollectors private Map<ApplicationId, AppCollectorData> registeringCollectors
= new ConcurrentHashMap<>(); = new ConcurrentHashMap<>();
private Set<NodeLabel> nodeLabels;
public MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) { public MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) {
// scale vcores based on the requested memory // scale vcores based on the requested memory
@ -101,6 +104,13 @@ public class MockNM {
nodeId = BuilderUtils.newNodeId(splits[0], Integer.parseInt(splits[1])); nodeId = BuilderUtils.newNodeId(splits[0], Integer.parseInt(splits[1]));
} }
public MockNM(String nodeIdStr, Resource capability,
ResourceTrackerService resourceTracker, String version, Set<NodeLabel>
nodeLabels) {
this(nodeIdStr, capability, resourceTracker, version);
this.nodeLabels = nodeLabels;
}
public NodeId getNodeId() { public NodeId getNodeId() {
return nodeId; return nodeId;
} }
@ -164,12 +174,17 @@ public class MockNM {
List<ApplicationId> runningApplications) throws Exception { List<ApplicationId> runningApplications) throws Exception {
RegisterNodeManagerRequest req = Records.newRecord( RegisterNodeManagerRequest req = Records.newRecord(
RegisterNodeManagerRequest.class); RegisterNodeManagerRequest.class);
req.setNodeId(nodeId); req.setNodeId(nodeId);
req.setHttpPort(httpPort); req.setHttpPort(httpPort);
req.setResource(capability); req.setResource(capability);
req.setContainerStatuses(containerReports); req.setContainerStatuses(containerReports);
req.setNMVersion(version); req.setNMVersion(version);
req.setRunningApplications(runningApplications); req.setRunningApplications(runningApplications);
if ( nodeLabels != null && nodeLabels.size() > 0) {
req.setNodeLabels(nodeLabels);
}
RegisterNodeManagerResponse registrationResponse = RegisterNodeManagerResponse registrationResponse =
resourceTracker.registerNodeManager(req); resourceTracker.registerNodeManager(req);
this.currentContainerTokenMasterKey = this.currentContainerTokenMasterKey =

View File

@ -36,6 +36,7 @@ import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
@ -57,6 +58,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ExecutionType; import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest; import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.api.records.ResourceInformation;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
@ -247,10 +249,11 @@ public class TestAppManager{
private TestRMAppManager appMonitor; private TestRMAppManager appMonitor;
private ApplicationSubmissionContext asContext; private ApplicationSubmissionContext asContext;
private ApplicationId appId; private ApplicationId appId;
private QueueInfo mockDefaultQueueInfo;
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@Before @Before
public void setUp() { public void setUp() throws IOException {
long now = System.currentTimeMillis(); long now = System.currentTimeMillis();
rmContext = mockRMContext(1, now - 10); rmContext = mockRMContext(1, now - 10);
@ -258,6 +261,7 @@ public class TestAppManager{
.setRMTimelineCollectorManager(mock(RMTimelineCollectorManager.class)); .setRMTimelineCollectorManager(mock(RMTimelineCollectorManager.class));
ResourceScheduler scheduler = mockResourceScheduler(); ResourceScheduler scheduler = mockResourceScheduler();
((RMContextImpl)rmContext).setScheduler(scheduler); ((RMContextImpl)rmContext).setScheduler(scheduler);
Configuration conf = new Configuration(); Configuration conf = new Configuration();
conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
((RMContextImpl) rmContext).setYarnConfiguration(conf); ((RMContextImpl) rmContext).setYarnConfiguration(conf);
@ -275,6 +279,11 @@ public class TestAppManager{
asContext.setAMContainerSpec(mockContainerLaunchContext(recordFactory)); asContext.setAMContainerSpec(mockContainerLaunchContext(recordFactory));
asContext.setResource(mockResource()); asContext.setResource(mockResource());
asContext.setPriority(Priority.newInstance(0)); asContext.setPriority(Priority.newInstance(0));
asContext.setQueue("default");
mockDefaultQueueInfo = mock(QueueInfo.class);
when(scheduler.getQueueInfo("default", false, false))
.thenReturn(mockDefaultQueueInfo);
setupDispatcher(rmContext, conf); setupDispatcher(rmContext, conf);
} }
@ -709,6 +718,7 @@ public class TestAppManager{
for (ResourceRequest req : reqs) { for (ResourceRequest req : reqs) {
req.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL); req.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL);
} }
// setAMContainerResourceRequests has priority over // setAMContainerResourceRequests has priority over
// setAMContainerResourceRequest and setResource // setAMContainerResourceRequest and setResource
Assert.assertEquals(reqs, app.getAMResourceRequests()); Assert.assertEquals(reqs, app.getAMResourceRequests());
@ -722,6 +732,7 @@ public class TestAppManager{
ResourceRequest req = ResourceRequest req =
ResourceRequest.newInstance(Priority.newInstance(0), ResourceRequest.newInstance(Priority.newInstance(0),
ResourceRequest.ANY, Resources.createResource(1025), 1, true); ResourceRequest.ANY, Resources.createResource(1025), 1, true);
req.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL);
asContext.setAMContainerResourceRequest(cloneResourceRequest(req)); asContext.setAMContainerResourceRequest(cloneResourceRequest(req));
// getAMContainerResourceRequests uses a singleton list of // getAMContainerResourceRequests uses a singleton list of
// getAMContainerResourceRequest // getAMContainerResourceRequest
@ -729,7 +740,6 @@ public class TestAppManager{
Assert.assertEquals(req, asContext.getAMContainerResourceRequests().get(0)); Assert.assertEquals(req, asContext.getAMContainerResourceRequests().get(0));
Assert.assertEquals(1, asContext.getAMContainerResourceRequests().size()); Assert.assertEquals(1, asContext.getAMContainerResourceRequests().size());
RMApp app = testRMAppSubmit(); RMApp app = testRMAppSubmit();
req.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL);
// setAMContainerResourceRequest has priority over setResource // setAMContainerResourceRequest has priority over setResource
Assert.assertEquals(Collections.singletonList(req), Assert.assertEquals(Collections.singletonList(req),
app.getAMResourceRequests()); app.getAMResourceRequests());
@ -740,10 +750,12 @@ public class TestAppManager{
asContext.setResource(Resources.createResource(1024)); asContext.setResource(Resources.createResource(1024));
asContext.setAMContainerResourceRequests(null); asContext.setAMContainerResourceRequests(null);
RMApp app = testRMAppSubmit(); RMApp app = testRMAppSubmit();
// setResource // setResource
Assert.assertEquals(Collections.singletonList( Assert.assertEquals(Collections.singletonList(
ResourceRequest.newInstance(RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.newInstance(RMAppAttemptImpl.AM_CONTAINER_PRIORITY,
ResourceRequest.ANY, Resources.createResource(1024), 1, true, "")), ResourceRequest.ANY, Resources.createResource(1024), 1, true,
"")),
app.getAMResourceRequests()); app.getAMResourceRequests());
} }
@ -766,6 +778,8 @@ public class TestAppManager{
throws Exception { throws Exception {
asContext.setResource(null); asContext.setResource(null);
List<ResourceRequest> reqs = new ArrayList<>(); List<ResourceRequest> reqs = new ArrayList<>();
when(mockDefaultQueueInfo.getAccessibleNodeLabels()).thenReturn
(new HashSet<String>() {{ add("label1"); add(""); }});
ResourceRequest anyReq = ResourceRequest.newInstance( ResourceRequest anyReq = ResourceRequest.newInstance(
Priority.newInstance(1), Priority.newInstance(1),
ResourceRequest.ANY, Resources.createResource(1024), 1, false, "label1", ResourceRequest.ANY, Resources.createResource(1024), 1, false, "label1",

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.security.TestGroupsCaching;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
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.Resource; 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.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.AsyncDispatcher;
@ -65,6 +66,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
.SimpleGroupsMapping; .SimpleGroupsMapping;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.YarnVersionInfo;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
@ -89,6 +92,8 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CapacitySchedulerConfiguration.DOT; .capacity.CapacitySchedulerConfiguration.DOT;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CapacitySchedulerConfiguration.FAIR_APP_ORDERING_POLICY; .capacity.CapacitySchedulerConfiguration.FAIR_APP_ORDERING_POLICY;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CapacitySchedulerConfiguration.ROOT;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
@ -99,7 +104,7 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
private static final Log LOG = LogFactory.getLog( private static final Log LOG = LogFactory.getLog(
TestCapacitySchedulerAutoCreatedQueueBase.class); TestCapacitySchedulerAutoCreatedQueueBase.class);
public static final int GB = 1024; public static final int GB = 1024;
public final static ContainerUpdates NULL_UPDATE_REQUESTS = public static final ContainerUpdates NULL_UPDATE_REQUESTS =
new ContainerUpdates(); new ContainerUpdates();
public static final String A = CapacitySchedulerConfiguration.ROOT + ".a"; public static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
@ -112,9 +117,6 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
public static final String B1 = B + ".b1"; public static final String B1 = B + ".b1";
public static final String B2 = B + ".b2"; public static final String B2 = B + ".b2";
public static final String B3 = B + ".b3"; public static final String B3 = B + ".b3";
public static final String C1 = C + ".c1";
public static final String C2 = C + ".c2";
public static final String C3 = C + ".c3";
public static final float A_CAPACITY = 20f; public static final float A_CAPACITY = 20f;
public static final float B_CAPACITY = 40f; public static final float B_CAPACITY = 40f;
public static final float C_CAPACITY = 20f; public static final float C_CAPACITY = 20f;
@ -124,8 +126,6 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
public static final float B1_CAPACITY = 60f; public static final float B1_CAPACITY = 60f;
public static final float B2_CAPACITY = 20f; public static final float B2_CAPACITY = 20f;
public static final float B3_CAPACITY = 20f; public static final float B3_CAPACITY = 20f;
public static final float C1_CAPACITY = 20f;
public static final float C2_CAPACITY = 20f;
public static final int NODE_MEMORY = 16; public static final int NODE_MEMORY = 16;
@ -147,12 +147,14 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
public static final String NODEL_LABEL_GPU = "GPU"; public static final String NODEL_LABEL_GPU = "GPU";
public static final String NODEL_LABEL_SSD = "SSD"; public static final String NODEL_LABEL_SSD = "SSD";
public static final float NODE_LABEL_GPU_TEMPLATE_CAPACITY = 30.0f;
public static final float NODEL_LABEL_SSD_TEMPLATE_CAPACITY = 40.0f;
protected MockRM mockRM = null; protected MockRM mockRM = null;
protected MockNM nm1 = null; protected MockNM nm1 = null;
protected MockNM nm2 = null; protected MockNM nm2 = null;
protected MockNM nm3 = null; protected MockNM nm3 = null;
protected CapacityScheduler cs; protected CapacityScheduler cs;
private final TestCapacityScheduler tcs = new TestCapacityScheduler();
protected SpyDispatcher dispatcher; protected SpyDispatcher dispatcher;
private static EventHandler<Event> rmAppEventEventHandler; private static EventHandler<Event> rmAppEventEventHandler;
@ -215,15 +217,29 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
} }
protected void setupNodes(MockRM newMockRM) throws Exception { protected void setupNodes(MockRM newMockRM) throws Exception {
NodeLabel ssdLabel = Records.newRecord(NodeLabel.class);
ssdLabel.setName(NODEL_LABEL_SSD);
ssdLabel.setExclusivity(true);
nm1 = // label = SSD nm1 = // label = SSD
new MockNM("h1:1234", NODE_MEMORY * GB, NODE1_VCORES, newMockRM new MockNM("h1:1234",
.getResourceTrackerService()); Resource.newInstance(NODE_MEMORY * GB, NODE1_VCORES),
newMockRM.getResourceTrackerService(),
YarnVersionInfo.getVersion(),
new HashSet<NodeLabel>() {{ add(ssdLabel); }});
nm1.registerNode(); nm1.registerNode();
nm2 = // label = GPU NodeLabel gpuLabel = Records.newRecord(NodeLabel.class);
new MockNM("h2:1234", NODE_MEMORY * GB, NODE2_VCORES, newMockRM ssdLabel.setName(NODEL_LABEL_GPU);
.getResourceTrackerService ssdLabel.setExclusivity(true);
());
//Label = GPU
nm2 = new MockNM("h2:1234",
Resource.newInstance(NODE_MEMORY * GB, NODE2_VCORES),
newMockRM.getResourceTrackerService(),
YarnVersionInfo.getVersion(),
new HashSet<NodeLabel>() {{ add(gpuLabel); }});
nm2.registerNode(); nm2.registerNode();
nm3 = // label = "" nm3 = // label = ""
@ -295,19 +311,23 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
/** /**
* @param conf, to be modified * @param conf, to be modified
* @return, CS configuration which has C as an auto creation enabled parent * @return, CS configuration which has C
* queue * as an auto creation enabled parent queue
* <p> * <p>
* root / \ \ \ a b c d / \ / | \ a1 a2 b1 * root
* b2 b3 * / \ \ \
* a b c d
* / \ / | \
* a1 a2 b1 b2 b3
*/ */
public static CapacitySchedulerConfiguration setupQueueConfiguration( public static CapacitySchedulerConfiguration setupQueueConfiguration(
CapacitySchedulerConfiguration conf) { CapacitySchedulerConfiguration conf) {
//setup new queues with one of them auto enabled //setup new queues with one of them auto enabled
// Define top-level queues // Define top-level queues
// Set childQueue for root // Set childQueue for root
conf.setQueues(CapacitySchedulerConfiguration.ROOT, conf.setQueues(ROOT,
new String[] { "a", "b", "c", "d" }); new String[] { "a", "b", "c", "d" });
conf.setCapacity(A, A_CAPACITY); conf.setCapacity(A, A_CAPACITY);
@ -339,6 +359,19 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
conf.setAutoCreatedLeafQueueConfigUserLimit(C, 100); conf.setAutoCreatedLeafQueueConfigUserLimit(C, 100);
conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f); conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f);
conf.setAutoCreatedLeafQueueTemplateCapacityByLabel(C, NODEL_LABEL_GPU,
NODE_LABEL_GPU_TEMPLATE_CAPACITY);
conf.setAutoCreatedLeafQueueTemplateMaxCapacity(C, NODEL_LABEL_GPU, 100.0f);
conf.setAutoCreatedLeafQueueTemplateCapacityByLabel(C, NODEL_LABEL_SSD,
NODEL_LABEL_SSD_TEMPLATE_CAPACITY);
conf.setAutoCreatedLeafQueueTemplateMaxCapacity(C, NODEL_LABEL_SSD,
100.0f);
conf.setDefaultNodeLabelExpression(C, NODEL_LABEL_GPU);
conf.setAutoCreatedLeafQueueConfigDefaultNodeLabelExpression
(C, NODEL_LABEL_SSD);
LOG.info("Setup " + C + " as an auto leaf creation enabled parent queue"); LOG.info("Setup " + C + " as an auto leaf creation enabled parent queue");
conf.setUserLimitFactor(D, 1.0f); conf.setUserLimitFactor(D, 1.0f);
@ -363,8 +396,13 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
accessibleNodeLabelsOnC.add(NO_LABEL); accessibleNodeLabelsOnC.add(NO_LABEL);
conf.setAccessibleNodeLabels(C, accessibleNodeLabelsOnC); conf.setAccessibleNodeLabels(C, accessibleNodeLabelsOnC);
conf.setCapacityByLabel(C, NODEL_LABEL_GPU, 50); conf.setAccessibleNodeLabels(ROOT, accessibleNodeLabelsOnC);
conf.setCapacityByLabel(C, NODEL_LABEL_SSD, 50); conf.setCapacityByLabel(ROOT, NODEL_LABEL_GPU, 100f);
conf.setCapacityByLabel(ROOT, NODEL_LABEL_SSD, 100f);
conf.setAccessibleNodeLabels(C, accessibleNodeLabelsOnC);
conf.setCapacityByLabel(C, NODEL_LABEL_GPU, 100f);
conf.setCapacityByLabel(C, NODEL_LABEL_SSD, 100f);
LOG.info("Setup " + D + " as an auto leaf creation enabled parent queue"); LOG.info("Setup " + D + " as an auto leaf creation enabled parent queue");
@ -541,19 +579,21 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
autoCreatedLeafQueue.getMaxApplicationsPerUser()); autoCreatedLeafQueue.getMaxApplicationsPerUser());
} }
protected void validateInitialQueueEntitlement(CSQueue parentQueue, protected void validateInitialQueueEntitlement(CSQueue parentQueue, String
String leafQueueName, float expectedTotalChildQueueAbsCapacity, leafQueueName, Map<String, Float>
expectedTotalChildQueueAbsCapacityByLabel,
Set<String> nodeLabels) Set<String> nodeLabels)
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException, InterruptedException {
validateInitialQueueEntitlement(cs, parentQueue, leafQueueName, validateInitialQueueEntitlement(cs, parentQueue, leafQueueName,
expectedTotalChildQueueAbsCapacity, nodeLabels); expectedTotalChildQueueAbsCapacityByLabel, nodeLabels);
} }
protected void validateInitialQueueEntitlement( protected void validateInitialQueueEntitlement(
CapacityScheduler capacityScheduler, CSQueue parentQueue, CapacityScheduler capacityScheduler, CSQueue parentQueue,
String leafQueueName, float expectedTotalChildQueueAbsCapacity, String leafQueueName,
Map<String, Float> expectedTotalChildQueueAbsCapacityByLabel,
Set<String> nodeLabels) Set<String> nodeLabels)
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException, InterruptedException {
ManagedParentQueue autoCreateEnabledParentQueue = ManagedParentQueue autoCreateEnabledParentQueue =
(ManagedParentQueue) parentQueue; (ManagedParentQueue) parentQueue;
@ -561,11 +601,7 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
(GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue (GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue
.getAutoCreatedQueueManagementPolicy(); .getAutoCreatedQueueManagementPolicy();
assertEquals(expectedTotalChildQueueAbsCapacity, AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) capacityScheduler.getQueue(leafQueueName);
policy.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
AutoCreatedLeafQueue leafQueue =
(AutoCreatedLeafQueue) capacityScheduler.getQueue(leafQueueName);
Map<String, QueueEntitlement> expectedEntitlements = new HashMap<>(); Map<String, QueueEntitlement> expectedEntitlements = new HashMap<>();
QueueCapacities cap = autoCreateEnabledParentQueue.getLeafQueueTemplate() QueueCapacities cap = autoCreateEnabledParentQueue.getLeafQueueTemplate()
@ -573,6 +609,10 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
for (String label : nodeLabels) { for (String label : nodeLabels) {
validateCapacitiesByLabel(autoCreateEnabledParentQueue, leafQueue, label); validateCapacitiesByLabel(autoCreateEnabledParentQueue, leafQueue, label);
assertEquals(true, policy.isActive(leafQueue, label));
assertEquals(expectedTotalChildQueueAbsCapacityByLabel.get(label),
policy.getAbsoluteActivatedChildQueueCapacity(label), EPSILON);
QueueEntitlement expectedEntitlement = new QueueEntitlement( QueueEntitlement expectedEntitlement = new QueueEntitlement(
cap.getCapacity(label), cap.getMaximumCapacity(label)); cap.getCapacity(label), cap.getMaximumCapacity(label));
@ -581,21 +621,19 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
validateEffectiveMinResource(leafQueue, label, expectedEntitlements); validateEffectiveMinResource(leafQueue, label, expectedEntitlements);
} }
assertEquals(true, policy.isActive(leafQueue));
} }
protected void validateCapacitiesByLabel( protected void validateCapacitiesByLabel(ManagedParentQueue
ManagedParentQueue autoCreateEnabledParentQueue, autoCreateEnabledParentQueue, AutoCreatedLeafQueue leafQueue, String
AutoCreatedLeafQueue leafQueue, String label) { label) throws InterruptedException {
assertEquals( assertEquals(autoCreateEnabledParentQueue.getLeafQueueTemplate()
autoCreateEnabledParentQueue.getLeafQueueTemplate().getQueueCapacities() .getQueueCapacities().getCapacity(label),
.getCapacity(), leafQueue.getQueueCapacities().getCapacity(label), leafQueue.getQueueCapacities()
EPSILON); .getCapacity(label), EPSILON);
assertEquals( assertEquals(autoCreateEnabledParentQueue.getLeafQueueTemplate()
autoCreateEnabledParentQueue.getLeafQueueTemplate().getQueueCapacities() .getQueueCapacities().getMaximumCapacity(label),
.getMaximumCapacity(), leafQueue.getQueueCapacities()
leafQueue.getQueueCapacities().getMaximumCapacity(label), EPSILON); .getMaximumCapacity(label), EPSILON);
} }
protected void validateEffectiveMinResource(CSQueue leafQueue, protected void validateEffectiveMinResource(CSQueue leafQueue,
@ -621,8 +659,10 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
} }
protected void validateActivatedQueueEntitlement(CSQueue parentQueue, protected void validateActivatedQueueEntitlement(CSQueue parentQueue,
String leafQueueName, float expectedTotalChildQueueAbsCapacity, String leafQueueName, Map<String, Float>
List<QueueManagementChange> queueManagementChanges) expectedTotalChildQueueAbsCapacity,
List<QueueManagementChange> queueManagementChanges, Set<String>
expectedNodeLabels)
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException {
ManagedParentQueue autoCreateEnabledParentQueue = ManagedParentQueue autoCreateEnabledParentQueue =
(ManagedParentQueue) parentQueue; (ManagedParentQueue) parentQueue;
@ -633,67 +673,84 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
QueueCapacities cap = autoCreateEnabledParentQueue.getLeafQueueTemplate() QueueCapacities cap = autoCreateEnabledParentQueue.getLeafQueueTemplate()
.getQueueCapacities(); .getQueueCapacities();
QueueEntitlement expectedEntitlement = new QueueEntitlement(
cap.getCapacity(), cap.getMaximumCapacity()); AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue)
cs.getQueue(leafQueueName);
Map<String, QueueEntitlement> expectedEntitlements = new HashMap<>();
for (String label : expectedNodeLabels) {
//validate leaf queue state
assertEquals(true, policy.isActive(leafQueue, label));
QueueEntitlement expectedEntitlement = new QueueEntitlement(
cap.getCapacity(label), cap.getMaximumCapacity(label));
//validate parent queue state
assertEquals(expectedTotalChildQueueAbsCapacity.get(label),
policy.getAbsoluteActivatedChildQueueCapacity(label), EPSILON);
expectedEntitlements.put(label, expectedEntitlement);
}
//validate capacity //validate capacity
validateQueueEntitlements(leafQueueName, expectedEntitlement, validateQueueEntitlements(leafQueueName, expectedEntitlements,
queueManagementChanges); queueManagementChanges, expectedNodeLabels);
//validate parent queue state
assertEquals(expectedTotalChildQueueAbsCapacity,
policy.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
leafQueueName);
//validate leaf queue state
assertEquals(true, policy.isActive(leafQueue));
} }
protected void validateDeactivatedQueueEntitlement(CSQueue parentQueue, protected void validateDeactivatedQueueEntitlement(CSQueue parentQueue,
String leafQueueName, float expectedTotalChildQueueAbsCapacity, String leafQueueName, Map<String, Float>
List<QueueManagementChange> queueManagementChanges) expectedTotalChildQueueAbsCapacity,
List<QueueManagementChange>
queueManagementChanges)
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException {
QueueEntitlement expectedEntitlement = new QueueEntitlement(0.0f, 1.0f); QueueEntitlement expectedEntitlement =
new QueueEntitlement(0.0f, 1.0f);
ManagedParentQueue autoCreateEnabledParentQueue = ManagedParentQueue autoCreateEnabledParentQueue =
(ManagedParentQueue) parentQueue; (ManagedParentQueue) parentQueue;
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue( AutoCreatedLeafQueue leafQueue =
leafQueueName); (AutoCreatedLeafQueue) cs.getQueue(leafQueueName);
GuaranteedOrZeroCapacityOverTimePolicy policy = GuaranteedOrZeroCapacityOverTimePolicy policy =
(GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue (GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue
.getAutoCreatedQueueManagementPolicy(); .getAutoCreatedQueueManagementPolicy();
//validate parent queue state Map<String, QueueEntitlement> expectedEntitlements = new HashMap<>();
assertEquals(expectedTotalChildQueueAbsCapacity,
policy.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
//validate leaf queue state for (String label : accessibleNodeLabelsOnC) {
assertEquals(false, policy.isActive(leafQueue)); //validate parent queue state
LOG.info("Validating label " + label);
assertEquals(expectedTotalChildQueueAbsCapacity.get(label), policy
.getAbsoluteActivatedChildQueueCapacity(label), EPSILON);
//validate leaf queue state
assertEquals(false, policy.isActive(leafQueue, label));
expectedEntitlements.put(label, expectedEntitlement);
}
//validate capacity //validate capacity
validateQueueEntitlements(leafQueueName, expectedEntitlement, validateQueueEntitlements(leafQueueName, expectedEntitlements,
queueManagementChanges); queueManagementChanges, accessibleNodeLabelsOnC);
} }
private void validateQueueEntitlements(String leafQueueName, void validateQueueEntitlements(String leafQueueName,
QueueEntitlement expectedEntitlement, Map<String, QueueEntitlement> expectedEntitlements,
List<QueueManagementChange> queueEntitlementChanges) { List<QueueManagementChange>
queueEntitlementChanges, Set<String> expectedNodeLabels) {
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue( AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
leafQueueName); leafQueueName);
validateQueueEntitlementChangesForLeafQueue(leafQueue, expectedEntitlement, validateQueueEntitlementChanges(leafQueue, expectedEntitlements,
queueEntitlementChanges); queueEntitlementChanges, expectedNodeLabels);
} }
private void validateQueueEntitlementChangesForLeafQueue(CSQueue leafQueue, private void validateQueueEntitlementChanges(AutoCreatedLeafQueue leafQueue,
QueueEntitlement expectedQueueEntitlement, Map<String, QueueEntitlement> expectedQueueEntitlements,
final List<QueueManagementChange> queueEntitlementChanges) { final List<QueueManagementChange> queueEntitlementChanges, Set<String>
expectedNodeLabels) {
boolean found = false; boolean found = false;
Map<String, QueueEntitlement> expectedQueueEntitlements = new HashMap<>();
for (QueueManagementChange entitlementChange : queueEntitlementChanges) { for (QueueManagementChange entitlementChange : queueEntitlementChanges) {
if (leafQueue.getQueueName().equals( if (leafQueue.getQueueName().equals(
entitlementChange.getQueue().getQueueName())) { entitlementChange.getQueue().getQueueName())) {
@ -701,13 +758,12 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
AutoCreatedLeafQueueConfig updatedQueueTemplate = AutoCreatedLeafQueueConfig updatedQueueTemplate =
entitlementChange.getUpdatedQueueTemplate(); entitlementChange.getUpdatedQueueTemplate();
for (String label : accessibleNodeLabelsOnC) { for (String label : expectedNodeLabels) {
QueueEntitlement newEntitlement = new QueueEntitlement( QueueEntitlement newEntitlement = new QueueEntitlement(
updatedQueueTemplate.getQueueCapacities().getCapacity(label), updatedQueueTemplate.getQueueCapacities().getCapacity(label),
updatedQueueTemplate.getQueueCapacities() updatedQueueTemplate.getQueueCapacities().getMaximumCapacity
.getMaximumCapacity(label)); (label));
assertEquals(expectedQueueEntitlement, newEntitlement); assertEquals(expectedQueueEntitlements.get(label), newEntitlement);
expectedQueueEntitlements.put(label, expectedQueueEntitlement);
validateEffectiveMinResource(leafQueue, label, validateEffectiveMinResource(leafQueue, label,
expectedQueueEntitlements); expectedQueueEntitlements);
} }
@ -716,9 +772,20 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
} }
} }
if (!found) { if (!found) {
fail("Could not find the specified leaf queue in entitlement changes : " fail(
+ leafQueue.getQueueName()); "Could not find the specified leaf queue in entitlement changes : "
+ leafQueue.getQueueName());
} }
} }
protected Map<String, Float> populateExpectedAbsCapacityByLabelForParentQueue
(int numLeafQueues) {
Map<String, Float> expectedChildQueueAbsCapacity = new HashMap<>();
expectedChildQueueAbsCapacity.put(NODEL_LABEL_GPU,
NODE_LABEL_GPU_TEMPLATE_CAPACITY/100 * numLeafQueues);
expectedChildQueueAbsCapacity.put(NODEL_LABEL_SSD,
NODEL_LABEL_SSD_TEMPLATE_CAPACITY/100 * numLeafQueues);
expectedChildQueueAbsCapacity.put(NO_LABEL, 0.1f * numLeafQueues);
return expectedChildQueueAbsCapacity;
}
} }

View File

@ -72,15 +72,18 @@ import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
.NO_LABEL; .NO_LABEL;
import static org.apache.hadoop.yarn.server.resourcemanager.placement import static org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule.CURRENT_USER_MAPPING;
.UserGroupMappingPlacementRule.CURRENT_USER_MAPPING; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CSQueueUtils.EPSILON;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
@ -90,7 +93,7 @@ import static org.mockito.Mockito.when;
/** /**
* Tests for creation and reinitialization of auto created leaf queues * Tests for creation and reinitialization of auto created leaf queues
* under a ManagedParentQueue. * and capacity management under a ManagedParentQueue.
*/ */
public class TestCapacitySchedulerAutoQueueCreation public class TestCapacitySchedulerAutoQueueCreation
extends TestCapacitySchedulerAutoCreatedQueueBase { extends TestCapacitySchedulerAutoCreatedQueueBase {
@ -105,7 +108,7 @@ public class TestCapacitySchedulerAutoQueueCreation
4); 4);
@Test(timeout = 10000) @Test(timeout = 20000)
public void testAutoCreateLeafQueueCreation() throws Exception { public void testAutoCreateLeafQueueCreation() throws Exception {
try { try {
@ -122,7 +125,12 @@ public class TestCapacitySchedulerAutoQueueCreation
ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue( ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue(
PARENT_QUEUE); PARENT_QUEUE);
assertEquals(parentQueue, autoCreatedLeafQueue.getParent()); assertEquals(parentQueue, autoCreatedLeafQueue.getParent());
validateInitialQueueEntitlement(parentQueue, USER0, 0.1f, accessibleNodeLabelsOnC);
Map<String, Float> expectedChildQueueAbsCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(1);
validateInitialQueueEntitlement(parentQueue, USER0,
expectedChildQueueAbsCapacity, accessibleNodeLabelsOnC);
validateUserAndAppLimits(autoCreatedLeafQueue, 1000, 1000); validateUserAndAppLimits(autoCreatedLeafQueue, 1000, 1000);
assertTrue(autoCreatedLeafQueue assertTrue(autoCreatedLeafQueue
@ -136,7 +144,14 @@ public class TestCapacitySchedulerAutoQueueCreation
(AutoCreatedLeafQueue) cs.getQueue(TEST_GROUPUSER); (AutoCreatedLeafQueue) cs.getQueue(TEST_GROUPUSER);
parentQueue = (ManagedParentQueue) cs.getQueue("d"); parentQueue = (ManagedParentQueue) cs.getQueue("d");
assertEquals(parentQueue, autoCreatedLeafQueue.getParent()); assertEquals(parentQueue, autoCreatedLeafQueue.getParent());
validateInitialQueueEntitlement(parentQueue, TEST_GROUPUSER, 0.02f,
expectedChildQueueAbsCapacity =
new HashMap<String, Float>() {{
put(NO_LABEL, 0.02f);
}};
validateInitialQueueEntitlement(parentQueue, TEST_GROUPUSER,
expectedChildQueueAbsCapacity,
new HashSet<String>() {{ add(NO_LABEL); }}); new HashSet<String>() {{ add(NO_LABEL); }});
} finally { } finally {
@ -173,10 +188,17 @@ public class TestCapacitySchedulerAutoQueueCreation
USER0); USER0);
ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue( ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue(
PARENT_QUEUE); PARENT_QUEUE);
assertEquals(parentQueue, user0Queue.getParent()); assertEquals(parentQueue, user0Queue.getParent());
assertEquals(parentQueue, user1Queue.getParent()); assertEquals(parentQueue, user1Queue.getParent());
validateInitialQueueEntitlement(parentQueue, USER0, 0.2f, accessibleNodeLabelsOnC);
validateInitialQueueEntitlement(parentQueue, USER1, 0.2f, accessibleNodeLabelsOnC); Map<String, Float>
expectedAbsChildQueueCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(2);
validateInitialQueueEntitlement(parentQueue, USER0,
expectedAbsChildQueueCapacity, accessibleNodeLabelsOnC);
validateInitialQueueEntitlement(parentQueue, USER1,
expectedAbsChildQueueCapacity, accessibleNodeLabelsOnC);
ApplicationAttemptId appAttemptId = appsInC.get(0); ApplicationAttemptId appAttemptId = appsInC.get(0);
@ -184,7 +206,8 @@ public class TestCapacitySchedulerAutoQueueCreation
RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory( RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(
null); null);
ResourceRequest r1 = TestUtils.createResourceRequest(ResourceRequest.ANY, ResourceRequest r1 = TestUtils.createResourceRequest(ResourceRequest.ANY,
1 * GB, 1, true, priority, recordFactory); 1 * GB, 1, true, priority,
recordFactory);
cs.allocate(appAttemptId, Collections.<ResourceRequest>singletonList(r1), cs.allocate(appAttemptId, Collections.<ResourceRequest>singletonList(r1),
null, Collections.<ContainerId>emptyList(), Collections.singletonList(host), null, Collections.<ContainerId>emptyList(), Collections.singletonList(host),
@ -216,8 +239,12 @@ public class TestCapacitySchedulerAutoQueueCreation
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue( AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
USER1); USER1);
expectedAbsChildQueueCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(1);
validateInitialQueueEntitlement(parentQueue, leafQueue.getQueueName(), validateInitialQueueEntitlement(parentQueue, leafQueue.getQueueName(),
0.1f, accessibleNodeLabelsOnC); expectedAbsChildQueueCapacity, accessibleNodeLabelsOnC);
} finally { } finally {
cleanupQueue(USER0); cleanupQueue(USER0);
@ -498,52 +525,80 @@ public class TestCapacitySchedulerAutoQueueCreation
CSQueue parentQueue = cs.getQueue(PARENT_QUEUE); CSQueue parentQueue = cs.getQueue(PARENT_QUEUE);
//submit app1 as USER1 //submit app1 as USER1
submitApp(mockRM, parentQueue, USER1, USER1, 1, 1); ApplicationId user1AppId = submitApp(mockRM, parentQueue, USER1, USER1,
validateInitialQueueEntitlement(parentQueue, USER1, 0.1f, accessibleNodeLabelsOnC); 1, 1);
Map<String, Float> expectedAbsChildQueueCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(1);
validateInitialQueueEntitlement(parentQueue, USER1,
expectedAbsChildQueueCapacity, accessibleNodeLabelsOnC);
//submit another app2 as USER2 //submit another app2 as USER2
ApplicationId user2AppId = submitApp(mockRM, parentQueue, USER2, USER2, 2, ApplicationId user2AppId = submitApp(mockRM, parentQueue, USER2, USER2, 2,
1); 1);
validateInitialQueueEntitlement(parentQueue, USER2, 0.2f, accessibleNodeLabelsOnC);
expectedAbsChildQueueCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(2);
validateInitialQueueEntitlement(parentQueue, USER2,
expectedAbsChildQueueCapacity, accessibleNodeLabelsOnC);
//submit another app3 as USER1 //submit another app3 as USER1
submitApp(mockRM, parentQueue, USER1, USER1, 3, 2); submitApp(mockRM, parentQueue, USER1, USER1, 3, 2);
//validate total activated abs capacity remains the same //validate total activated abs capacity remains the same
GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy = GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy =
(GuaranteedOrZeroCapacityOverTimePolicy) ((ManagedParentQueue) (GuaranteedOrZeroCapacityOverTimePolicy) ((ManagedParentQueue) parentQueue)
parentQueue)
.getAutoCreatedQueueManagementPolicy(); .getAutoCreatedQueueManagementPolicy();
assertEquals(autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON);
//submit user_3 app. This cant be scheduled since there is no capacity for (String nodeLabel : accessibleNodeLabelsOnC) {
assertEquals(expectedAbsChildQueueCapacity.get(nodeLabel),
autoCreatedQueueManagementPolicy.getAbsoluteActivatedChildQueueCapacity(nodeLabel), EPSILON);
}
//submit user_3 app. This cant be allocated since there is no capacity
// in NO_LABEL, SSD but can be in GPU label
submitApp(mockRM, parentQueue, USER3, USER3, 4, 1); submitApp(mockRM, parentQueue, USER3, USER3, 4, 1);
final CSQueue user3LeafQueue = cs.getQueue(USER3); final CSQueue user3LeafQueue = cs.getQueue(USER3);
validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.0f, 0.0f, validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.0f, 0.0f,
1.0f, 1.0f); 1.0f, 1.0f);
validateCapacitiesByLabel((ManagedParentQueue) parentQueue,
(AutoCreatedLeafQueue)
user3LeafQueue, NODEL_LABEL_GPU);
assertEquals(autoCreatedQueueManagementPolicy assertEquals(0.2f, autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON); .getAbsoluteActivatedChildQueueCapacity(NO_LABEL), EPSILON);
assertEquals(0.9f, autoCreatedQueueManagementPolicy.getAbsoluteActivatedChildQueueCapacity(NODEL_LABEL_GPU),
EPSILON);
//deactivate USER2 queue //Verify that AMs can be allocated
//Node 1 has SSD and default node label expression on C is SSD.
//This validates that the default node label expression with SSD is set
// on the AM attempt
// and app attempt reaches ALLOCATED state for a dynamic queue 'USER1'
mockRM.launchAM(mockRM.getRMContext().getRMApps().get(user1AppId),
mockRM, nm1);
// //deactivate USER2 queue
cs.killAllAppsInQueue(USER2); cs.killAllAppsInQueue(USER2);
mockRM.waitForState(user2AppId, RMAppState.KILLED); mockRM.waitForState(user2AppId, RMAppState.KILLED);
//Verify if USER_2 can be deactivated since it has no pending appsA //Verify if USER_2 can be deactivated since it has no pending apps
List<QueueManagementChange> queueManagementChanges = List<QueueManagementChange> queueManagementChanges =
autoCreatedQueueManagementPolicy.computeQueueManagementChanges(); autoCreatedQueueManagementPolicy.computeQueueManagementChanges();
ManagedParentQueue managedParentQueue = (ManagedParentQueue) parentQueue; ManagedParentQueue managedParentQueue = (ManagedParentQueue) parentQueue;
managedParentQueue.validateAndApplyQueueManagementChanges( managedParentQueue.
queueManagementChanges); validateAndApplyQueueManagementChanges(queueManagementChanges);
validateDeactivatedQueueEntitlement(parentQueue, USER2, 0.2f, validateDeactivatedQueueEntitlement(parentQueue, USER2,
queueManagementChanges); expectedAbsChildQueueCapacity, queueManagementChanges);
//USER_3 should now get activated //USER_3 should now get activated for SSD, NO_LABEL
validateActivatedQueueEntitlement(parentQueue, USER3, 0.2f, Set<String> expectedNodeLabelsUpdated = new HashSet<>();
queueManagementChanges); expectedNodeLabelsUpdated.add(NO_LABEL);
expectedNodeLabelsUpdated.add(NODEL_LABEL_SSD);
validateActivatedQueueEntitlement(parentQueue, USER3,
expectedAbsChildQueueCapacity , queueManagementChanges, expectedNodeLabelsUpdated);
} finally { } finally {
cleanupQueue(USER1); cleanupQueue(USER1);
@ -565,13 +620,18 @@ public class TestCapacitySchedulerAutoQueueCreation
//submit app1 as USER1 //submit app1 as USER1
submitApp(newMockRM, parentQueue, USER1, USER1, 1, 1); submitApp(newMockRM, parentQueue, USER1, USER1, 1, 1);
validateInitialQueueEntitlement(newCS, parentQueue, USER1, 0.1f, accessibleNodeLabelsOnC); Map<String, Float> expectedAbsChildQueueCapacity =
CSQueue user1LeafQueue = newCS.getQueue(USER1); populateExpectedAbsCapacityByLabelForParentQueue(1);
validateInitialQueueEntitlement(newCS, parentQueue, USER1,
expectedAbsChildQueueCapacity, accessibleNodeLabelsOnC);
//submit another app2 as USER2 //submit another app2 as USER2
submitApp(newMockRM, parentQueue, USER2, USER2, 2, 1); ApplicationId user2AppId = submitApp(newMockRM, parentQueue, USER2, USER2, 2,
validateInitialQueueEntitlement(newCS, parentQueue, USER2, 0.2f, accessibleNodeLabelsOnC); 1);
CSQueue user2LeafQueue = newCS.getQueue(USER2); expectedAbsChildQueueCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(2);
validateInitialQueueEntitlement(newCS, parentQueue, USER2,
expectedAbsChildQueueCapacity, accessibleNodeLabelsOnC);
//validate total activated abs capacity remains the same //validate total activated abs capacity remains the same
GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy = GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy =
@ -579,7 +639,7 @@ public class TestCapacitySchedulerAutoQueueCreation
parentQueue) parentQueue)
.getAutoCreatedQueueManagementPolicy(); .getAutoCreatedQueueManagementPolicy();
assertEquals(autoCreatedQueueManagementPolicy assertEquals(autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON); .getAbsoluteActivatedChildQueueCapacity(NO_LABEL), 0.2f, EPSILON);
//submit user_3 app. This cant be scheduled since there is no capacity //submit user_3 app. This cant be scheduled since there is no capacity
submitApp(newMockRM, parentQueue, USER3, USER3, 3, 1); submitApp(newMockRM, parentQueue, USER3, USER3, 3, 1);
@ -588,7 +648,7 @@ public class TestCapacitySchedulerAutoQueueCreation
1.0f, 1.0f); 1.0f, 1.0f);
assertEquals(autoCreatedQueueManagementPolicy assertEquals(autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON); .getAbsoluteActivatedChildQueueCapacity(NO_LABEL), 0.2f, EPSILON);
// add new NM. // add new NM.
newMockRM.registerNode("127.0.0.3:1234", 125 * GB, 20); newMockRM.registerNode("127.0.0.3:1234", 125 * GB, 20);
@ -596,31 +656,33 @@ public class TestCapacitySchedulerAutoQueueCreation
// There will be change in effective resource when nodes are added // There will be change in effective resource when nodes are added
// since we deal with percentages // since we deal with percentages
Resource MAX_RES = Resources.addTo(TEMPLATE_MAX_RES, Resource MAX_RES = Resources.addTo(TEMPLATE_MAX_RES, Resources.createResource(125 *
Resources.createResource(125 * GB, 20)); GB, 20));
Resource MIN_RES = Resources.createResource(14438, 6); Resource MIN_RES = Resources.createResource(14438, 6);
Assert.assertEquals("Effective Min resource for USER3 is not correct", Assert.assertEquals("Effective Min resource for USER3 is not correct",
Resources.none(), Resources.none(), user3LeafQueue.getQueueResourceQuotas()
user3LeafQueue.getQueueResourceQuotas().getEffectiveMinResource()); .getEffectiveMinResource());
Assert.assertEquals("Effective Max resource for USER3 is not correct", Assert.assertEquals("Effective Max resource for USER3 is not correct",
MAX_RES, MAX_RES, user3LeafQueue
user3LeafQueue.getQueueResourceQuotas().getEffectiveMaxResource()); .getQueueResourceQuotas()
.getEffectiveMaxResource());
CSQueue user1LeafQueue = newCS.getQueue(USER1);
CSQueue user2LeafQueue = newCS.getQueue(USER2);
Assert.assertEquals("Effective Min resource for USER2 is not correct", Assert.assertEquals("Effective Min resource for USER2 is not correct",
MIN_RES, MIN_RES, user1LeafQueue.getQueueResourceQuotas()
user1LeafQueue.getQueueResourceQuotas().getEffectiveMinResource()); .getEffectiveMinResource());
Assert.assertEquals("Effective Max resource for USER2 is not correct", Assert.assertEquals("Effective Max resource for USER2 is not correct",
MAX_RES, MAX_RES, user1LeafQueue.getQueueResourceQuotas().getEffectiveMaxResource());
user1LeafQueue.getQueueResourceQuotas().getEffectiveMaxResource());
Assert.assertEquals("Effective Min resource for USER1 is not correct", Assert.assertEquals("Effective Min resource for USER1 is not correct",
MIN_RES, MIN_RES, user2LeafQueue.getQueueResourceQuotas()
user2LeafQueue.getQueueResourceQuotas().getEffectiveMinResource()); .getEffectiveMinResource());
Assert.assertEquals("Effective Max resource for USER1 is not correct", Assert.assertEquals("Effective Max resource for USER1 is not correct",
MAX_RES, MAX_RES, user2LeafQueue.getQueueResourceQuotas()
user2LeafQueue.getQueueResourceQuotas().getEffectiveMaxResource()); .getEffectiveMaxResource());
// unregister one NM. // unregister one NM.
newMockRM.unRegisterNode(nm3); newMockRM.unRegisterNode(nm3);
@ -629,11 +691,11 @@ public class TestCapacitySchedulerAutoQueueCreation
// After loosing one NM, resources will reduce // After loosing one NM, resources will reduce
Assert.assertEquals("Effective Min resource for USER2 is not correct", Assert.assertEquals("Effective Min resource for USER2 is not correct",
MIN_RES_UPDATED, MIN_RES_UPDATED, user1LeafQueue.getQueueResourceQuotas().getEffectiveMinResource
user1LeafQueue.getQueueResourceQuotas().getEffectiveMinResource()); ());
Assert.assertEquals("Effective Max resource for USER2 is not correct", Assert.assertEquals("Effective Max resource for USER2 is not correct",
MAX_RES_UPDATED, MAX_RES_UPDATED, user2LeafQueue.getQueueResourceQuotas()
user2LeafQueue.getQueueResourceQuotas().getEffectiveMaxResource()); .getEffectiveMaxResource());
} finally { } finally {
cleanupQueue(USER1); cleanupQueue(USER1);
@ -646,25 +708,6 @@ public class TestCapacitySchedulerAutoQueueCreation
} }
} }
@Test
public void testAutoCreatedQueueInheritsNodeLabels() throws Exception {
try {
String host = "127.0.0.1";
RMNode node = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1,
host);
cs.handle(new NodeAddedSchedulerEvent(node));
CSQueue parentQueue = cs.getQueue(PARENT_QUEUE);
submitApp(USER1, USER1, NODEL_LABEL_GPU);
//submit app1 as USER1
validateInitialQueueEntitlement(parentQueue, USER1, 0.1f, accessibleNodeLabelsOnC);
} finally {
cleanupQueue(USER1);
}
}
@Test @Test
public void testReinitializeQueuesWithAutoCreatedLeafQueues() public void testReinitializeQueuesWithAutoCreatedLeafQueues()
throws Exception { throws Exception {
@ -679,12 +722,20 @@ public class TestCapacitySchedulerAutoQueueCreation
//submit app1 as USER1 //submit app1 as USER1
submitApp(newMockRM, parentQueue, USER1, USER1, 1, 1); submitApp(newMockRM, parentQueue, USER1, USER1, 1, 1);
validateInitialQueueEntitlement(newCS, parentQueue, USER1, 0.1f, accessibleNodeLabelsOnC);
Map<String, Float> expectedChildQueueAbsCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(1);
validateInitialQueueEntitlement(newCS, parentQueue, USER1,
expectedChildQueueAbsCapacity, accessibleNodeLabelsOnC);
//submit another app2 as USER2 //submit another app2 as USER2
ApplicationId user2AppId = submitApp(newMockRM, parentQueue, USER2, USER2, ApplicationId user2AppId = submitApp(newMockRM, parentQueue, USER2,
2, 1); USER2, 2,
validateInitialQueueEntitlement(newCS, parentQueue, USER2, 0.2f, accessibleNodeLabelsOnC); 1);
expectedChildQueueAbsCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(2);
validateInitialQueueEntitlement(newCS, parentQueue, USER2,
expectedChildQueueAbsCapacity, accessibleNodeLabelsOnC);
//update parent queue capacity //update parent queue capacity
conf.setCapacity(C, 30f); conf.setCapacity(C, 30f);
@ -709,19 +760,27 @@ public class TestCapacitySchedulerAutoQueueCreation
//submit app1 as USER3 //submit app1 as USER3
submitApp(newMockRM, parentQueue, USER3, USER3, 3, 1); submitApp(newMockRM, parentQueue, USER3, USER3, 3, 1);
validateInitialQueueEntitlement(newCS, parentQueue, USER3, 0.27f, accessibleNodeLabelsOnC); AutoCreatedLeafQueue user3Queue =
AutoCreatedLeafQueue user3Queue = (AutoCreatedLeafQueue) newCS.getQueue( (AutoCreatedLeafQueue) newCS.getQueue(USER1);
USER1); validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f,0.2f);
validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f, 0.2f);
validateUserAndAppLimits(user3Queue, 900, 900); validateUserAndAppLimits(user3Queue, 900, 900);
//submit app1 as USER1 - is already activated. there should be no diff //submit app1 as USER1 - is already activated. there should be no diff
// in capacities // in capacities
submitApp(newMockRM, parentQueue, USER3, USER3, 4, 2); submitApp(newMockRM, parentQueue, USER3, USER3, 4, 2);
validateInitialQueueEntitlement(newCS, parentQueue, USER3, 0.27f, accessibleNodeLabelsOnC);
validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f, 0.2f); validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f,0.2f);
validateUserAndAppLimits(user3Queue, 900, 900); validateUserAndAppLimits(user3Queue, 900, 900);
GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy =
(GuaranteedOrZeroCapacityOverTimePolicy) ((ManagedParentQueue)
parentQueue)
.getAutoCreatedQueueManagementPolicy();
assertEquals(0.27f, autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity
(NO_LABEL), EPSILON);
} finally { } finally {
cleanupQueue(USER1); cleanupQueue(USER1);
cleanupQueue(USER2); cleanupQueue(USER2);

View File

@ -24,7 +24,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.util.Map;
import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
.NO_LABEL;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CSQueueUtils.EPSILON; .capacity.CSQueueUtils.EPSILON;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -54,21 +57,27 @@ public class TestQueueManagementDynamicEditPolicy extends
parentQueue) parentQueue)
.getAutoCreatedQueueManagementPolicy(); .getAutoCreatedQueueManagementPolicy();
assertEquals(0f, autoCreatedQueueManagementPolicy assertEquals(0f, autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), EPSILON); .getAbsoluteActivatedChildQueueCapacity(NO_LABEL), EPSILON);
//submit app1 as USER1 //submit app1 as USER1
ApplicationId user1AppId = submitApp(mockRM, parentQueue, USER1, USER1, 1, ApplicationId user1AppId = submitApp(mockRM, parentQueue, USER1, USER1, 1,
1); 1);
validateInitialQueueEntitlement(parentQueue, USER1, 0.1f, accessibleNodeLabelsOnC); Map<String, Float> expectedAbsChildQueueCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(1);
validateInitialQueueEntitlement(parentQueue, USER1,
expectedAbsChildQueueCapacity, accessibleNodeLabelsOnC);
//submit another app2 as USER2 //submit another app2 as USER2
ApplicationId user2AppId = submitApp(mockRM, parentQueue, USER2, USER2, 2, ApplicationId user2AppId = submitApp(mockRM, parentQueue, USER2, USER2, 2,
1); 1);
validateInitialQueueEntitlement(parentQueue, USER2, 0.2f, accessibleNodeLabelsOnC); expectedAbsChildQueueCapacity =
populateExpectedAbsCapacityByLabelForParentQueue(2);
validateInitialQueueEntitlement(parentQueue, USER2,
expectedAbsChildQueueCapacity, accessibleNodeLabelsOnC);
//validate total activated abs capacity //validate total activated abs capacity
assertEquals(0.2f, autoCreatedQueueManagementPolicy assertEquals(0.2f, autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), EPSILON); .getAbsoluteActivatedChildQueueCapacity(NO_LABEL), EPSILON);
//submit user_3 app. This cant be scheduled since there is no capacity //submit user_3 app. This cant be scheduled since there is no capacity
submitApp(mockRM, parentQueue, USER3, USER3, 3, 1); submitApp(mockRM, parentQueue, USER3, USER3, 3, 1);
@ -77,7 +86,7 @@ public class TestQueueManagementDynamicEditPolicy extends
1.0f, 1.0f); 1.0f, 1.0f);
assertEquals(autoCreatedQueueManagementPolicy assertEquals(autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON); .getAbsoluteActivatedChildQueueCapacity(NO_LABEL), 0.2f, EPSILON);
//deactivate USER2 queue //deactivate USER2 queue
cs.killAllAppsInQueue(USER2); cs.killAllAppsInQueue(USER2);
@ -88,8 +97,8 @@ public class TestQueueManagementDynamicEditPolicy extends
mockRM.waitForState(user1AppId, RMAppState.KILLED); mockRM.waitForState(user1AppId, RMAppState.KILLED);
policy.editSchedule(); policy.editSchedule();
waitForPolicyState(0.1f, autoCreatedQueueManagementPolicy, NO_LABEL,
waitForPolicyState(0.1f, autoCreatedQueueManagementPolicy, 1000); 1000);
validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.5f, 0.1f, validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.5f, 0.1f,
1.0f, 1.0f); 1.0f, 1.0f);
@ -105,13 +114,12 @@ public class TestQueueManagementDynamicEditPolicy extends
} }
private void waitForPolicyState(float expectedVal, private void waitForPolicyState(float expectedVal,
GuaranteedOrZeroCapacityOverTimePolicy queueManagementPolicy, int GuaranteedOrZeroCapacityOverTimePolicy queueManagementPolicy, String
timesec) throws nodeLabel, int timesec) throws InterruptedException {
InterruptedException {
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
while (System.currentTimeMillis() - start < timesec * 1000) { while (System.currentTimeMillis() - start < timesec * 1000) {
if (Float.compare(expectedVal, queueManagementPolicy if (Float.compare(expectedVal, queueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity()) != 0) { .getAbsoluteActivatedChildQueueCapacity(nodeLabel)) > EPSILON) {
Thread.sleep(100); Thread.sleep(100);
} else { } else {
break; break;