YARN-7473. Implement Framework and policy for capacity management of auto created queues. (Suma Shivaprasad via wangda)

Change-Id: Icca7805fe12f6f7fb335effff4b121b6f7f6337b
This commit is contained in:
Wangda Tan 2017-12-08 15:10:16 -08:00
parent 74665e3a7d
commit b38643c9a8
35 changed files with 3535 additions and 805 deletions

View File

@ -79,7 +79,7 @@ public class SchedulingMonitor extends AbstractService {
}
private void schedulePreemptionChecker() {
handler = ses.scheduleAtFixedRate(new PreemptionChecker(),
handler = ses.scheduleAtFixedRate(new PolicyInvoker(),
0, monitorInterval, TimeUnit.MILLISECONDS);
}
@ -99,7 +99,7 @@ public class SchedulingMonitor extends AbstractService {
scheduleEditPolicy.editSchedule();
}
private class PreemptionChecker implements Runnable {
private class PolicyInvoker implements Runnable {
@Override
public void run() {
try {

View File

@ -37,6 +37,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuot
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.ManagedParentQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptableQueue;
@ -377,7 +380,9 @@ public class ProportionalCapacityPreemptionPolicy
}
private Set<String> getLeafQueueNames(TempQueuePerPartition q) {
if (q.children == null || q.children.isEmpty()) {
// If its a ManagedParentQueue, it might not have any children
if ((q.children == null || q.children.isEmpty())
&& !(q.parentQueue instanceof ManagedParentQueue)) {
return ImmutableSet.of(q.queueName);
}

View File

@ -21,6 +21,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.ParentQueue;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceUtils;
import org.apache.hadoop.yarn.util.resource.Resources;
@ -56,6 +59,7 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
final ArrayList<TempQueuePerPartition> children;
private Collection<TempAppPerPartition> apps;
LeafQueue leafQueue;
ParentQueue parentQueue;
boolean preemptionDisabled;
protected Resource pendingDeductReserved;
@ -90,6 +94,10 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
pendingDeductReserved = Resources.createResource(0);
}
if (ParentQueue.class.isAssignableFrom(queue.getClass())) {
parentQueue = (ParentQueue) queue;
}
this.normalizedGuarantee = new double[ResourceUtils
.getNumberOfKnownResourceTypes()];
this.children = new ArrayList<>();

View File

@ -28,10 +28,12 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedLeafQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.ReservationQueue;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.slf4j.Logger;
@ -92,8 +94,8 @@ public class CapacitySchedulerPlanFollower extends AbstractSchedulerPlanFollower
String planQueueName, Queue queue, String currResId) {
PlanQueue planQueue = (PlanQueue)queue;
try {
AutoCreatedLeafQueue resQueue =
new AutoCreatedLeafQueue(cs, currResId, planQueue);
ReservationQueue resQueue =
new ReservationQueue(cs, currResId, planQueue);
cs.addQueue(resQueue);
} catch (SchedulerDynamicEditException e) {
LOG.warn(
@ -112,8 +114,8 @@ public class CapacitySchedulerPlanFollower extends AbstractSchedulerPlanFollower
PlanQueue planQueue = (PlanQueue)queue;
if (cs.getQueue(defReservationId) == null) {
try {
AutoCreatedLeafQueue defQueue =
new AutoCreatedLeafQueue(cs, defReservationId, planQueue);
ReservationQueue defQueue =
new ReservationQueue(cs, defReservationId, planQueue);
cs.addQueue(defQueue);
} catch (SchedulerDynamicEditException e) {
LOG.warn(

View File

@ -457,7 +457,7 @@ public abstract class AbstractYarnScheduler
}
@Override
public void addQueue(Queue newQueue) throws YarnException {
public void addQueue(Queue newQueue) throws YarnException, IOException {
throw new YarnException(getClass().getSimpleName()
+ " does not support this operation");
}

View File

@ -272,7 +272,7 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
* @param newQueue the queue being added.
* @throws YarnException
*/
void addQueue(Queue newQueue) throws YarnException;
void addQueue(Queue newQueue) throws YarnException, IOException;
/**
* This method increase the entitlement for current queue (must respect

View File

@ -0,0 +1,113 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common
.QueueEntitlement;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
.NO_LABEL;
/**
* Abstract class for dynamic auto created queues managed by an implementation
* of AbstractManagedParentQueue
*/
public class AbstractAutoCreatedLeafQueue extends LeafQueue {
protected AbstractManagedParentQueue parent;
public AbstractAutoCreatedLeafQueue(CapacitySchedulerContext cs,
String queueName, AbstractManagedParentQueue parent, CSQueue old)
throws IOException {
super(cs, queueName, parent, old);
this.parent = parent;
}
private static final Logger LOG = LoggerFactory.getLogger(
AbstractAutoCreatedLeafQueue.class);
public AbstractAutoCreatedLeafQueue(CapacitySchedulerContext cs,
CapacitySchedulerConfiguration leafQueueConfigs, String queueName,
AbstractManagedParentQueue parent, CSQueue old) throws IOException {
super(cs, leafQueueConfigs, queueName, parent, old);
this.parent = parent;
}
/**
* This methods to change capacity for a queue and adjusts its
* absoluteCapacity
*
* @param entitlement the new entitlement for the queue (capacity,
* maxCapacity, etc..)
* @throws SchedulerDynamicEditException
*/
public void setEntitlement(QueueEntitlement entitlement)
throws SchedulerDynamicEditException {
setEntitlement(NO_LABEL, entitlement);
}
/**
* This methods to change capacity for a queue and adjusts its
* absoluteCapacity
*
* @param entitlement the new entitlement for the queue (capacity,
* maxCapacity, etc..)
* @throws SchedulerDynamicEditException
*/
public void setEntitlement(String nodeLabel, QueueEntitlement entitlement)
throws SchedulerDynamicEditException {
try {
writeLock.lock();
float capacity = entitlement.getCapacity();
if (capacity < 0 || capacity > 1.0f) {
throw new SchedulerDynamicEditException(
"Capacity demand is not in the [0,1] range: " + capacity);
}
setCapacity(nodeLabel, capacity);
setAbsoluteCapacity(nodeLabel,
getParent().getQueueCapacities().
getAbsoluteCapacity(nodeLabel)
* getQueueCapacities().getCapacity(nodeLabel));
// note: we currently set maxCapacity to capacity
// this might be revised later
setMaxCapacity(nodeLabel, entitlement.getMaxCapacity());
if (LOG.isDebugEnabled()) {
LOG.debug("successfully changed to " + capacity + " for queue " + this
.getQueueName());
}
//update queue used capacity etc
CSQueueUtils.updateQueueStatistics(resourceCalculator,
csContext.getClusterResource(),
this, labelManager, nodeLabel);
} finally {
writeLock.unlock();
}
}
protected void setupConfigurableCapacities(QueueCapacities queueCapacities) {
CSQueueUtils.updateAndCheckCapacitiesByLabel(getQueuePath(),
queueCapacities, parent == null ? null : parent.getQueueCapacities());
}
}

View File

@ -128,27 +128,34 @@ public abstract class AbstractCSQueue implements CSQueue {
public AbstractCSQueue(CapacitySchedulerContext cs,
String queueName, CSQueue parent, CSQueue old) throws IOException {
this(cs, cs.getConfiguration(), queueName, parent, old);
}
public AbstractCSQueue(CapacitySchedulerContext cs,
CapacitySchedulerConfiguration configuration, String queueName,
CSQueue parent, CSQueue old) {
this.labelManager = cs.getRMContext().getNodeLabelManager();
this.parent = parent;
this.queueName = queueName;
this.queuePath =
((parent == null) ? "" : (parent.getQueuePath() + ".")) + this.queueName;
this.queuePath = ((parent == null) ? "" : (parent.getQueuePath() + "."))
+ this.queueName;
this.resourceCalculator = cs.getResourceCalculator();
this.activitiesManager = cs.getActivitiesManager();
// must be called after parent and queueName is set
this.metrics =
old != null ? (CSQueueMetrics) old.getMetrics() : CSQueueMetrics
.forQueue(getQueuePath(), parent, cs.getConfiguration()
.getEnableUserMetrics(), cs.getConf());
this.metrics = old != null ?
(CSQueueMetrics) old.getMetrics() :
CSQueueMetrics.forQueue(getQueuePath(), parent,
configuration.getEnableUserMetrics(), cs.getConf());
this.csContext = cs;
this.minimumAllocation = csContext.getMinimumResourceCapability();
// initialize ResourceUsage
queueUsage = new ResourceUsage();
queueEntity = new PrivilegedEntity(EntityType.QUEUE, getQueuePath());
// initialize QueueCapacities
queueCapacities = new QueueCapacities(parent == null);
@ -159,11 +166,16 @@ public abstract class AbstractCSQueue implements CSQueue {
readLock = lock.readLock();
writeLock = lock.writeLock();
}
protected void setupConfigurableCapacities() {
setupConfigurableCapacities(csContext.getConfiguration());
}
protected void setupConfigurableCapacities(
CapacitySchedulerConfiguration configuration) {
CSQueueUtils.loadUpdateAndCheckCapacities(
getQueuePath(),
csContext.getConfiguration(),
configuration,
queueCapacities,
parent == null ? null : parent.getQueueCapacities());
}
@ -275,6 +287,29 @@ public abstract class AbstractCSQueue implements CSQueue {
}
}
/**
* Set maximum capacity
* @param maximumCapacity new max capacity
*/
void setMaxCapacity(String nodeLabel, float maximumCapacity) {
try {
writeLock.lock();
// Sanity check
CSQueueUtils.checkMaxCapacity(getQueueName(),
queueCapacities.getCapacity(nodeLabel), maximumCapacity);
float absMaxCapacity = CSQueueUtils.computeAbsoluteMaximumCapacity(
maximumCapacity, parent);
CSQueueUtils.checkAbsoluteCapacity(getQueueName(),
queueCapacities.getAbsoluteCapacity(nodeLabel), absMaxCapacity);
queueCapacities.setMaximumCapacity(maximumCapacity);
queueCapacities.setAbsoluteMaximumCapacity(absMaxCapacity);
} finally {
writeLock.unlock();
}
}
@Override
public String getDefaultNodeLabelExpression() {
return defaultLabelExpression;
@ -282,13 +317,20 @@ public abstract class AbstractCSQueue implements CSQueue {
void setupQueueConfigs(Resource clusterResource)
throws IOException {
setupQueueConfigs(clusterResource, csContext.getConfiguration());
}
protected void setupQueueConfigs(Resource clusterResource,
CapacitySchedulerConfiguration configuration) throws
IOException {
try {
writeLock.lock();
// get labels
this.accessibleLabels =
csContext.getConfiguration().getAccessibleNodeLabels(getQueuePath());
configuration.getAccessibleNodeLabels(getQueuePath());
this.defaultLabelExpression =
csContext.getConfiguration().getDefaultNodeLabelExpression(
configuration.getDefaultNodeLabelExpression(
getQueuePath());
this.resourceTypes = new HashSet<String>();
for (AbsoluteResourceType type : AbsoluteResourceType.values()) {
@ -308,7 +350,7 @@ public abstract class AbstractCSQueue implements CSQueue {
}
// After we setup labels, we can setup capacities
setupConfigurableCapacities();
setupConfigurableCapacities(configuration);
// Also fetch minimum/maximum resource constraint for this queue if
// configured.
@ -316,20 +358,20 @@ public abstract class AbstractCSQueue implements CSQueue {
updateConfigurableResourceRequirement(getQueuePath(), clusterResource);
this.maximumAllocation =
csContext.getConfiguration().getMaximumAllocationPerQueue(
configuration.getMaximumAllocationPerQueue(
getQueuePath());
// initialized the queue state based on previous state, configured state
// and its parent state.
QueueState previous = getState();
QueueState configuredState = csContext.getConfiguration()
QueueState configuredState = configuration
.getConfiguredState(getQueuePath());
QueueState parentState = (parent == null) ? null : parent.getState();
initializeQueueState(previous, configuredState, parentState);
authorizer = YarnAuthorizationProvider.getInstance(csContext.getConf());
this.acls = csContext.getConfiguration().getAcls(getQueuePath());
this.acls = configuration.getAcls(getQueuePath());
// Update metrics
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
@ -361,18 +403,21 @@ public abstract class AbstractCSQueue implements CSQueue {
this.reservationsContinueLooking =
csContext.getConfiguration().getReservationContinueLook();
this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(this);
this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(this,
configuration);
this.priority = csContext.getConfiguration().getQueuePriority(
this.priority = configuration.getQueuePriority(
getQueuePath());
this.userWeights = getUserWeightsFromHierarchy();
this.userWeights = getUserWeightsFromHierarchy(configuration);
} finally {
writeLock.unlock();
}
}
private Map<String, Float> getUserWeightsFromHierarchy() throws IOException {
private Map<String, Float> getUserWeightsFromHierarchy
(CapacitySchedulerConfiguration configuration) throws
IOException {
Map<String, Float> unionInheritedWeights = new HashMap<String, Float>();
CSQueue parentQ = getParent();
if (parentQ != null) {
@ -381,9 +426,8 @@ public abstract class AbstractCSQueue implements CSQueue {
}
// Insert this queue's user's weights, overriding parent's user's weights if
// there is overlap.
CapacitySchedulerConfiguration csConf = csContext.getConfiguration();
unionInheritedWeights.putAll(
csConf.getAllUserWeightsForQueue(getQueuePath()));
configuration.getAllUserWeightsForQueue(getQueuePath()));
return unionInheritedWeights;
}
@ -720,10 +764,11 @@ public abstract class AbstractCSQueue implements CSQueue {
*
* @return true if queue has preemption disabled, false otherwise
*/
private boolean isQueueHierarchyPreemptionDisabled(CSQueue q) {
CapacitySchedulerConfiguration csConf = csContext.getConfiguration();
private boolean isQueueHierarchyPreemptionDisabled(CSQueue q,
CapacitySchedulerConfiguration configuration) {
boolean systemWidePreemption =
csConf.getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
csContext.getConfiguration()
.getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS);
CSQueue parentQ = q.getParent();
@ -735,14 +780,14 @@ public abstract class AbstractCSQueue implements CSQueue {
// on, then q does not have preemption disabled (default=false, below)
// unless the preemption_disabled property is explicitly set.
if (parentQ == null) {
return csConf.getPreemptionDisabled(q.getQueuePath(), false);
return configuration.getPreemptionDisabled(q.getQueuePath(), false);
}
// If this is not the root queue, inherit the default value for the
// preemption_disabled property from the parent. Preemptability will be
// inherited from the parent's hierarchy unless explicitly overridden at
// this level.
return csConf.getPreemptionDisabled(q.getQueuePath(),
return configuration.getPreemptionDisabled(q.getQueuePath(),
parentQ.getPreemptionDisabled());
}

View File

@ -17,13 +17,21 @@
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common
.QueueEntitlement;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Comparator;
import java.util.Iterator;
import java.util.Map;
import java.util.SortedMap;
import java.util.TreeMap;
/**
* A container class for automatically created child leaf queues.
@ -35,13 +43,12 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
private static final Logger LOG = LoggerFactory.getLogger(
AbstractManagedParentQueue.class);
protected AutoCreatedLeafQueueTemplate leafQueueTemplate;
protected AutoCreatedLeafQueueConfig leafQueueTemplate;
protected AutoCreatedQueueManagementPolicy queueManagementPolicy = null;
public AbstractManagedParentQueue(CapacitySchedulerContext cs,
String queueName, CSQueue parent, CSQueue old) throws IOException {
super(cs, queueName, parent, old);
super.setupQueueConfigs(csContext.getClusterResource());
}
@Override
@ -53,52 +60,18 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
// Set new configs
setupQueueConfigs(clusterResource);
// run reinitialize on each existing queue, to trigger absolute cap
// recomputations
for (CSQueue res : this.getChildQueues()) {
res.reinitialize(res, clusterResource);
}
} finally {
writeLock.unlock();
}
}
/**
* Initialize leaf queue configs from template configurations specified on
* parent queue.
*/
protected AutoCreatedLeafQueueTemplate.Builder initializeLeafQueueConfigs
(String queuePath) {
CapacitySchedulerConfiguration conf = csContext.getConfiguration();
AutoCreatedLeafQueueTemplate.Builder leafQueueTemplateBuilder = new
AutoCreatedLeafQueueTemplate.Builder();
int maxApps = conf.getMaximumApplicationsPerQueue(queuePath);
if (maxApps < 0) {
maxApps = (int) (
CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS
* getAbsoluteCapacity());
}
int userLimit = conf.getUserLimit(queuePath);
float userLimitFactor = conf.getUserLimitFactor(queuePath);
leafQueueTemplateBuilder.userLimit(userLimit)
.userLimitFactor(userLimitFactor)
.maxApps(maxApps)
.maxAppsPerUser(
(int) (maxApps * (userLimit / 100.0f) * userLimitFactor));
return leafQueueTemplateBuilder;
}
/**
* Add the specified child queue.
* @param childQueue reference to the child queue to be added
* @throws SchedulerDynamicEditException
*/
public void addChildQueue(CSQueue childQueue)
throws SchedulerDynamicEditException {
throws SchedulerDynamicEditException, IOException {
try {
writeLock.lock();
if (childQueue.getCapacity() > 0) {
@ -193,84 +166,69 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
}
}
public static class AutoCreatedLeafQueueTemplate {
private QueueCapacities queueCapacities;
private int maxApps;
private int maxAppsPerUser;
private int userLimit;
private float userLimitFactor;
AutoCreatedLeafQueueTemplate(Builder builder) {
this.maxApps = builder.maxApps;
this.maxAppsPerUser = builder.maxAppsPerUser;
this.userLimit = builder.userLimit;
this.userLimitFactor = builder.userLimitFactor;
this.queueCapacities = builder.queueCapacities;
}
public static class Builder {
private int maxApps;
private int maxAppsPerUser;
private int userLimit;
private float userLimitFactor;
private QueueCapacities queueCapacities;
Builder maxApps(int maxApplications) {
this.maxApps = maxApplications;
return this;
}
Builder maxAppsPerUser(int maxApplicationsPerUser) {
this.maxAppsPerUser = maxApplicationsPerUser;
return this;
}
Builder userLimit(int usrLimit) {
this.userLimit = usrLimit;
return this;
}
Builder userLimitFactor(float ulf) {
this.userLimitFactor = ulf;
return this;
}
Builder capacities(QueueCapacities capacities) {
this.queueCapacities = capacities;
return this;
}
AutoCreatedLeafQueueTemplate build() {
return new AutoCreatedLeafQueueTemplate(this);
}
}
public int getUserLimit() {
return userLimit;
}
public float getUserLimitFactor() {
return userLimitFactor;
}
public QueueCapacities getQueueCapacities() {
return queueCapacities;
}
public int getMaxApps() {
return maxApps;
}
public int getMaxAppsPerUser() {
return maxAppsPerUser;
}
}
public AutoCreatedLeafQueueTemplate getLeafQueueTemplate() {
public AutoCreatedLeafQueueConfig getLeafQueueTemplate() {
return leafQueueTemplate;
}
public AutoCreatedQueueManagementPolicy
getAutoCreatedQueueManagementPolicy() {
return queueManagementPolicy;
}
protected SortedMap<String, String> getConfigurationsWithPrefix
(SortedMap<String, String> sortedConfigs, String prefix) {
return sortedConfigs.subMap( prefix, prefix + Character.MAX_VALUE );
}
protected SortedMap<String, String> sortCSConfigurations() {
SortedMap<String, String> sortedConfigs = new TreeMap(
new Comparator<String>() {
public int compare(String s1, String s2) {
return s1.compareToIgnoreCase(s2);
}
});
for (final Iterator<Map.Entry<String, String>> iterator =
csContext.getConfiguration().iterator(); iterator.hasNext(); ) {
final Map.Entry<String, String> confKeyValuePair = iterator.next();
sortedConfigs.put(confKeyValuePair.getKey(), confKeyValuePair.getValue());
}
return sortedConfigs;
}
protected CapacitySchedulerConfiguration initializeLeafQueueConfigs(String
configPrefix) {
CapacitySchedulerConfiguration leafQueueConfigs = new
CapacitySchedulerConfiguration(new Configuration(false), false);
SortedMap<String, String> sortedConfigs = sortCSConfigurations();
SortedMap<String, String> templateConfigs = getConfigurationsWithPrefix
(sortedConfigs, configPrefix);
for (final Iterator<Map.Entry<String, String>> iterator =
templateConfigs.entrySet().iterator(); iterator.hasNext(); ) {
Map.Entry<String, String> confKeyValuePair = iterator.next();
leafQueueConfigs.set(confKeyValuePair.getKey(),
confKeyValuePair.getValue());
}
return leafQueueConfigs;
}
protected void validateQueueEntitlementChange(AbstractAutoCreatedLeafQueue
leafQueue, QueueEntitlement entitlement)
throws SchedulerDynamicEditException {
float sumChilds = sumOfChildCapacities();
float newChildCap =
sumChilds - leafQueue.getCapacity() + entitlement.getCapacity();
if (!(newChildCap >= 0 && newChildCap < 1.0f + CSQueueUtils.EPSILON)) {
throw new SchedulerDynamicEditException(
"Sum of child queues should exceed 100% for auto creating parent "
+ "queue : " + queueName);
}
}
}

View File

@ -21,36 +21,27 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.AbstractManagedParentQueue.AutoCreatedLeafQueueTemplate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
/**
* Leaf queues which are auto created by an underkying implementation of
* Leaf queues which are auto created by an underlying implementation of
* AbstractManagedParentQueue. Eg: PlanQueue for reservations or
* ManagedParentQueue for auto created dynamic queues
*/
public class AutoCreatedLeafQueue extends LeafQueue {
public class AutoCreatedLeafQueue extends AbstractAutoCreatedLeafQueue {
private static final Logger LOG = LoggerFactory
.getLogger(AutoCreatedLeafQueue.class);
private AbstractManagedParentQueue parent;
public AutoCreatedLeafQueue(CapacitySchedulerContext cs, String queueName,
AbstractManagedParentQueue parent) throws IOException {
super(cs, queueName, parent, null);
AutoCreatedLeafQueueTemplate leafQueueTemplate =
parent.getLeafQueueTemplate();
updateApplicationAndUserLimits(leafQueueTemplate.getUserLimit(),
leafQueueTemplate.getUserLimitFactor(),
leafQueueTemplate.getMaxApps(),
leafQueueTemplate.getMaxAppsPerUser());
this.parent = parent;
ManagedParentQueue parent) throws IOException {
super(cs, parent.getLeafQueueConfigs(queueName),
queueName,
parent, null);
updateCapacitiesToZero();
}
@Override
@ -61,48 +52,75 @@ public class AutoCreatedLeafQueue extends LeafQueue {
validate(newlyParsedQueue);
super.reinitialize(newlyParsedQueue, clusterResource);
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
this, labelManager, null);
ManagedParentQueue managedParentQueue = (ManagedParentQueue) parent;
AutoCreatedLeafQueueTemplate leafQueueTemplate =
parent.getLeafQueueTemplate();
updateApplicationAndUserLimits(leafQueueTemplate.getUserLimit(),
leafQueueTemplate.getUserLimitFactor(),
leafQueueTemplate.getMaxApps(),
leafQueueTemplate.getMaxAppsPerUser());
super.reinitialize(newlyParsedQueue, clusterResource, managedParentQueue
.getLeafQueueConfigs(newlyParsedQueue.getQueueName()));
//Reset capacities to 0 since reinitialize above
// queueCapacities to initialize to configured capacity which might
// overcommit resources from parent queue
updateCapacitiesToZero();
} finally {
writeLock.unlock();
}
}
/**
* This methods to change capacity for a queue and adjusts its
* absoluteCapacity.
*
* @param entitlement the new entitlement for the queue (capacity,
* maxCapacity)
* @throws SchedulerDynamicEditException
*/
public void setEntitlement(QueueEntitlement entitlement)
throws SchedulerDynamicEditException {
public void reinitializeFromTemplate(AutoCreatedLeafQueueConfig
leafQueueTemplate) throws SchedulerDynamicEditException, IOException {
try {
writeLock.lock();
float capacity = entitlement.getCapacity();
// TODO:
// reinitialize only capacities for now since 0 capacity updates
// can cause
// abs capacity related config computations to be incorrect if we go
// through reinitialize
QueueCapacities capacities = leafQueueTemplate.getQueueCapacities();
//update abs capacities
setupConfigurableCapacities(capacities);
//reset capacities for the leaf queue
mergeCapacities(capacities);
//update queue used capacity for all the node labels
CSQueueUtils.updateQueueStatistics(resourceCalculator,
csContext.getClusterResource(),
this, labelManager, null);
//activate applications if any are pending
activateApplications();
} finally {
writeLock.unlock();
}
}
private void mergeCapacities(QueueCapacities capacities) {
for ( String nodeLabel : capacities.getExistingNodeLabels()) {
this.queueCapacities.setCapacity(nodeLabel,
capacities.getCapacity(nodeLabel));
this.queueCapacities.setAbsoluteCapacity(nodeLabel, capacities
.getAbsoluteCapacity(nodeLabel));
this.queueCapacities.setMaximumCapacity(nodeLabel, capacities
.getMaximumCapacity(nodeLabel));
this.queueCapacities.setAbsoluteMaximumCapacity(nodeLabel, capacities
.getAbsoluteMaximumCapacity(nodeLabel));
}
}
public void validateConfigurations(AutoCreatedLeafQueueConfig template)
throws SchedulerDynamicEditException {
QueueCapacities capacities = template.getQueueCapacities();
for (String label : capacities.getExistingNodeLabels()) {
float capacity = capacities.getCapacity(label);
if (capacity < 0 || capacity > 1.0f) {
throw new SchedulerDynamicEditException(
"Capacity demand is not in the [0,1] range: " + capacity);
}
setCapacity(capacity);
setAbsoluteCapacity(getParent().getAbsoluteCapacity() * getCapacity());
setMaxCapacity(entitlement.getMaxCapacity());
if (LOG.isDebugEnabled()) {
LOG.debug("successfully changed to " + capacity + " for queue " + this
.getQueueName());
}
} finally {
writeLock.unlock();
}
}
@ -113,22 +131,20 @@ public class AutoCreatedLeafQueue extends LeafQueue {
"Error trying to reinitialize " + getQueuePath() + " from "
+ newlyParsedQueue.getQueuePath());
}
}
@Override
protected void setupConfigurableCapacities() {
CSQueueUtils.updateAndCheckCapacitiesByLabel(getQueuePath(),
queueCapacities, parent == null ? null : parent.getQueueCapacities());
}
private void updateApplicationAndUserLimits(int userLimit,
float userLimitFactor,
int maxAppsForAutoCreatedQueues,
int maxAppsPerUserForAutoCreatedQueues) {
setUserLimit(userLimit);
setUserLimitFactor(userLimitFactor);
setMaxApplications(maxAppsForAutoCreatedQueues);
setMaxApplicationsPerUser(maxAppsPerUserForAutoCreatedQueues);
private void updateCapacitiesToZero() throws IOException {
try {
for( String nodeLabel : parent.getQueueCapacities().getExistingNodeLabels
()) {
//TODO - update to use getMaximumCapacity(nodeLabel) in YARN-7574
setEntitlement(nodeLabel, new QueueEntitlement(0.0f,
parent.getLeafQueueTemplate()
.getQueueCapacities()
.getMaximumCapacity()));
}
} catch (SchedulerDynamicEditException e) {
throw new IOException(e);
}
}
}

View File

@ -0,0 +1,66 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
/**
* Auto Created Leaf queue configurations, capacity
*/
public class AutoCreatedLeafQueueConfig {
/**
* Template queue capacities - contains configured and derived capacities
* like abs capacity which are used by auto queue creation policy to manage
* leaf queue capacities
*/
private QueueCapacities queueCapacities;
private CapacitySchedulerConfiguration leafQueueConfigs;
public AutoCreatedLeafQueueConfig(Builder builder) {
this.queueCapacities = builder.queueCapacities;
this.leafQueueConfigs = builder.leafQueueConfigs;
}
public static class Builder {
private QueueCapacities queueCapacities;
private CapacitySchedulerConfiguration leafQueueConfigs;
public Builder capacities(QueueCapacities capacities) {
this.queueCapacities = capacities;
return this;
}
public Builder configuration(CapacitySchedulerConfiguration conf) {
this.leafQueueConfigs = conf;
return this;
}
public AutoCreatedLeafQueueConfig build() {
return new AutoCreatedLeafQueueConfig(this);
}
}
public QueueCapacities getQueueCapacities() {
return queueCapacities;
}
public CapacitySchedulerConfiguration getLeafQueueConfigs() {
return leafQueueConfigs;
}
}

View File

@ -0,0 +1,64 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
import java.util.List;
public interface AutoCreatedQueueManagementPolicy {
/**
* Initialize policy
* @param schedulerContext Capacity Scheduler context
*/
void init(CapacitySchedulerContext schedulerContext, ParentQueue parentQueue);
/**
* Reinitialize policy state ( if required )
* @param schedulerContext Capacity Scheduler context
*/
void reinitialize(CapacitySchedulerContext schedulerContext,
ParentQueue parentQueue);
/**
* Get initial template for the specified leaf queue
* @param leafQueue the leaf queue
* @return initial leaf queue template configurations and capacities for
* auto created queue
*/
AutoCreatedLeafQueueConfig getInitialLeafQueueConfiguration(
AbstractAutoCreatedLeafQueue leafQueue)
throws SchedulerDynamicEditException;
/**
* Compute/Adjust child queue capacities
* for auto created leaf queues
*
* @return returns a list of suggested QueueEntitlementChange(s) which may
* or may not be be enforced by the scheduler
*/
List<QueueManagementChange> computeQueueManagementChanges()
throws SchedulerDynamicEditException;
/**
* Commit/Update state for the specified queue management changes.
*/
void commitQueueManagementChanges(
List<QueueManagementChange> queueManagementChanges)
throws SchedulerDynamicEditException;
}

View File

@ -230,7 +230,7 @@ public interface CSQueue extends SchedulerQueue<CSQueue> {
* @param newlyParsedQueue new queue to re-initalize from
* @param clusterResource resources in the cluster
*/
public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
throws IOException;
/**

View File

@ -29,9 +29,9 @@ import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.Sets;
class CSQueueUtils {
public class CSQueueUtils {
final static float EPSILON = 0.0001f;
public final static float EPSILON = 0.0001f;
/*
* Used only by tests
@ -123,12 +123,12 @@ class CSQueueUtils {
for (String label : configuredNodelabels) {
if (label.equals(CommonNodeLabelsManager.NO_LABEL)) {
queueCapacities.setCapacity(CommonNodeLabelsManager.NO_LABEL,
queueCapacities.setCapacity(label,
csConf.getNonLabeledQueueCapacity(queuePath) / 100);
queueCapacities.setMaximumCapacity(CommonNodeLabelsManager.NO_LABEL,
queueCapacities.setMaximumCapacity(label,
csConf.getNonLabeledQueueMaximumCapacity(queuePath) / 100);
queueCapacities.setMaxAMResourcePercentage(
CommonNodeLabelsManager.NO_LABEL,
label,
csConf.getMaximumAMResourcePercentPerPartition(queuePath, label));
} else {
queueCapacities.setCapacity(label,
@ -183,9 +183,32 @@ class CSQueueUtils {
if (Resources.greaterThan(rc, totalPartitionResource,
totalPartitionResource, Resources.none())) {
Resource queueGuranteedResource = childQueue
.getEffectiveCapacity(nodePartition);
//TODO : Modify below code to support Absolute Resource configurations
// (YARN-5881) for AutoCreatedLeafQueue
if (Float.compare(queueCapacities.getAbsoluteCapacity
(nodePartition), 0f) == 0
&& childQueue instanceof AutoCreatedLeafQueue) {
//If absolute capacity is 0 for a leaf queue (could be a managed leaf
// queue, then use the leaf queue's template capacity to compute
// guaranteed resource for used capacity)
// queueGuaranteed = totalPartitionedResource *
// absolute_capacity(partition)
ManagedParentQueue parentQueue = (ManagedParentQueue)
childQueue.getParent();
QueueCapacities leafQueueTemplateCapacities = parentQueue
.getLeafQueueTemplate()
.getQueueCapacities();
queueGuranteedResource = Resources.multiply(totalPartitionResource,
leafQueueTemplateCapacities.getAbsoluteCapacity
(nodePartition));
}
// make queueGuranteed >= minimum_allocation to avoid divided by 0.
queueGuranteedResource =
Resources.max(rc, totalPartitionResource, queueGuranteedResource,

View File

@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@ -129,6 +130,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemoved
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
.QueueManagementChangeEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ReleaseContainerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@ -138,6 +141,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimpleC
import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.Lock;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
@ -175,6 +179,8 @@ public class CapacityScheduler extends
private CSConfigurationProvider csConfProvider;
protected Clock monotonicClock;
@Override
public void setConf(Configuration conf) {
yarnConf = conf;
@ -1501,7 +1507,7 @@ public class CapacityScheduler extends
{
NodeLabelsUpdateSchedulerEvent labelUpdateEvent =
(NodeLabelsUpdateSchedulerEvent) event;
updateNodeLabelsAndQueueResource(labelUpdateEvent);
}
break;
@ -1613,6 +1619,25 @@ public class CapacityScheduler extends
}
}
break;
case MANAGE_QUEUE:
{
QueueManagementChangeEvent queueManagementChangeEvent =
(QueueManagementChangeEvent) event;
ParentQueue parentQueue = queueManagementChangeEvent.getParentQueue();
try {
final List<QueueManagementChange> queueManagementChanges =
queueManagementChangeEvent.getQueueManagementChanges();
((ManagedParentQueue) parentQueue)
.validateAndApplyQueueManagementChanges(queueManagementChanges);
} catch (SchedulerDynamicEditException sde) {
LOG.error("Queue Management Change event cannot be applied for "
+ "parent queue : " + parentQueue.getQueueName(), sde);
} catch (IOException ioe) {
LOG.error("Queue Management Change event cannot be applied for "
+ "parent queue : " + parentQueue.getQueueName(), ioe);
}
}
break;
default:
LOG.error("Invalid eventtype " + event.getType() + ". Ignoring!");
}
@ -1976,12 +2001,14 @@ public class CapacityScheduler extends
writeLock.lock();
LOG.info("Removing queue: " + queueName);
CSQueue q = this.getQueue(queueName);
if (!(q instanceof AutoCreatedLeafQueue)) {
if (!(AbstractAutoCreatedLeafQueue.class.isAssignableFrom(
q.getClass()))) {
throw new SchedulerDynamicEditException(
"The queue that we are asked " + "to remove (" + queueName
+ ") is not a AutoCreatedLeafQueue");
+ ") is not a AutoCreatedLeafQueue or ReservationQueue");
}
AutoCreatedLeafQueue disposableLeafQueue = (AutoCreatedLeafQueue) q;
AbstractAutoCreatedLeafQueue disposableLeafQueue =
(AbstractAutoCreatedLeafQueue) q;
// at this point we should have no more apps
if (disposableLeafQueue.getNumApplications() > 0) {
throw new SchedulerDynamicEditException(
@ -1994,8 +2021,8 @@ public class CapacityScheduler extends
((AbstractManagedParentQueue) disposableLeafQueue.getParent())
.removeChildQueue(q);
this.queueManager.removeQueue(queueName);
LOG.info("Removal of AutoCreatedLeafQueue "
+ queueName + " has succeeded");
LOG.info(
"Removal of AutoCreatedLeafQueue " + queueName + " has succeeded");
} finally {
writeLock.unlock();
}
@ -2003,22 +2030,27 @@ public class CapacityScheduler extends
@Override
public void addQueue(Queue queue)
throws SchedulerDynamicEditException {
throws SchedulerDynamicEditException, IOException {
try {
writeLock.lock();
if (!(queue instanceof AutoCreatedLeafQueue)) {
if (queue == null) {
throw new SchedulerDynamicEditException(
"Queue " + queue.getQueueName() + " is not a AutoCreatedLeafQueue");
"Queue specified is null. Should be an implementation of "
+ "AbstractAutoCreatedLeafQueue");
} else if (!(AbstractAutoCreatedLeafQueue.class
.isAssignableFrom(queue.getClass()))) {
throw new SchedulerDynamicEditException(
"Queue is not an implementation of "
+ "AbstractAutoCreatedLeafQueue : " + queue.getClass());
}
AutoCreatedLeafQueue newQueue = (AutoCreatedLeafQueue) queue;
AbstractAutoCreatedLeafQueue newQueue =
(AbstractAutoCreatedLeafQueue) queue;
if (newQueue.getParent() == null
|| !(AbstractManagedParentQueue.class.
if (newQueue.getParent() == null || !(AbstractManagedParentQueue.class.
isAssignableFrom(newQueue.getParent().getClass()))) {
throw new SchedulerDynamicEditException(
"ParentQueue for " + newQueue.getQueueName()
+ " is not properly set"
"ParentQueue for " + newQueue + " is not properly set"
+ " (should be set and be a PlanQueue or ManagedParentQueue)");
}
@ -2027,6 +2059,7 @@ public class CapacityScheduler extends
String queuename = newQueue.getQueueName();
parentPlan.addChildQueue(newQueue);
this.queueManager.addQueue(queuename, newQueue);
LOG.info("Creation of AutoCreatedLeafQueue " + newQueue + " succeeded");
} finally {
writeLock.unlock();
@ -2039,48 +2072,32 @@ public class CapacityScheduler extends
try {
writeLock.lock();
LeafQueue queue = this.queueManager.getAndCheckLeafQueue(inQueue);
AbstractManagedParentQueue parent = (AbstractManagedParentQueue) queue
.getParent();
AbstractManagedParentQueue parent =
(AbstractManagedParentQueue) queue.getParent();
if (!(queue instanceof AutoCreatedLeafQueue)) {
if (!(AbstractAutoCreatedLeafQueue.class.isAssignableFrom(
queue.getClass()))) {
throw new SchedulerDynamicEditException(
"Entitlement can not be" + " modified dynamically since queue "
+ inQueue + " is not a AutoCreatedLeafQueue");
}
if (parent == null
|| !(AbstractManagedParentQueue.class.isAssignableFrom(
parent.getClass()))) {
if (parent == null || !(AbstractManagedParentQueue.class.isAssignableFrom(
parent.getClass()))) {
throw new SchedulerDynamicEditException(
"The parent of AutoCreatedLeafQueue " + inQueue
+ " must be a PlanQueue/ManagedParentQueue");
}
AutoCreatedLeafQueue newQueue = (AutoCreatedLeafQueue) queue;
AbstractAutoCreatedLeafQueue newQueue =
(AbstractAutoCreatedLeafQueue) queue;
parent.validateQueueEntitlementChange(newQueue, entitlement);
float sumChilds = parent.sumOfChildCapacities();
float newChildCap =
sumChilds - queue.getCapacity() + entitlement.getCapacity();
newQueue.setEntitlement(entitlement);
if (newChildCap >= 0 && newChildCap < 1.0f + CSQueueUtils.EPSILON) {
// note: epsilon checks here are not ok, as the epsilons might
// accumulate and become a problem in aggregate
if (Math.abs(entitlement.getCapacity() - queue.getCapacity()) == 0
&& Math.abs(
entitlement.getMaxCapacity() - queue.getMaximumCapacity()) == 0) {
return;
}
newQueue.setEntitlement(entitlement);
} else{
throw new SchedulerDynamicEditException(
"Sum of child queues should exceed 100% for auto creating parent "
+ "queue : " + parent.getQueueName());
}
LOG.info(
"Set entitlement for AutoCreatedLeafQueue " + inQueue
+ " to " + queue.getCapacity() +
" request was (" + entitlement.getCapacity()
+ ")");
LOG.info("Set entitlement for AutoCreatedLeafQueue " + inQueue + " to "
+ queue.getCapacity() + " request was (" + entitlement.getCapacity()
+ ")");
} finally {
writeLock.unlock();
}
@ -2718,7 +2735,6 @@ public class CapacityScheduler extends
addQueue(autoCreatedLeafQueue);
//TODO - Set entitlement through capacity management policy
} else{
throw new SchedulerDynamicEditException(
"Could not auto-create leaf queue for " + leafQueueName

View File

@ -923,6 +923,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
return getInt(NODE_LOCALITY_DELAY, DEFAULT_NODE_LOCALITY_DELAY);
}
@VisibleForTesting
public void setNodeLocalityDelay(int nodeLocalityDelay) {
setInt(NODE_LOCALITY_DELAY, nodeLocalityDelay);
}
public int getRackLocalityAdditionalDelay() {
return getInt(RACK_LOCALITY_ADDITIONAL_DELAY,
DEFAULT_RACK_LOCALITY_ADDITIONAL_DELAY);
@ -1401,6 +1406,10 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
return maxApplicationsPerQueue;
}
public void setGlobalMaximumApplicationsPerQueue(int val) {
setInt(QUEUE_GLOBAL_MAX_APPLICATION, val);
}
/**
* Ordering policy inside a parent queue to sort queues
*/
@ -1620,9 +1629,13 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
@Private
public static final boolean DEFAULT_AUTO_CREATE_CHILD_QUEUE_ENABLED = false;
@Private
private static final String AUTO_CREATE_CHILD_QUEUE_PREFIX =
"auto-create-child-queue.";
@Private
public static final String AUTO_CREATE_CHILD_QUEUE_ENABLED =
"auto-create-child-queue.enabled";
AUTO_CREATE_CHILD_QUEUE_PREFIX + "enabled";
@Private
public static final String AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX =
@ -1722,8 +1735,83 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
}
@Private
public static final String AUTO_CREATED_QUEUE_MANAGEMENT_POLICY =
AUTO_CREATE_CHILD_QUEUE_PREFIX + "management-policy";
@Private
public static final String DEFAULT_AUTO_CREATED_QUEUE_MANAGEMENT_POLICY =
"org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity"
+ ".queuemanagement."
+ "GuaranteedOrZeroCapacityOverTimePolicy";
@Private
private static final String QUEUE_MANAGEMENT_CONFIG_PREFIX =
"yarn.resourcemanager.monitor.capacity.queue-management.";
/**
* Time in milliseconds between invocations of this policy
*/
@Private
public static final String QUEUE_MANAGEMENT_MONITORING_INTERVAL =
QUEUE_MANAGEMENT_CONFIG_PREFIX + "monitoring-interval";
@Private
public static final long DEFAULT_QUEUE_MANAGEMENT_MONITORING_INTERVAL =
1500L;
/**
* Queue Management computation policy for Auto Created queues
* @param queue The queue's path
* @return Configured policy class name
*/
@Private
public String getAutoCreatedQueueManagementPolicy(String queue) {
String autoCreatedQueueManagementPolicy =
get(getQueuePrefix(queue) + AUTO_CREATED_QUEUE_MANAGEMENT_POLICY,
DEFAULT_AUTO_CREATED_QUEUE_MANAGEMENT_POLICY);
return autoCreatedQueueManagementPolicy;
}
/**
* Get The policy class configured to manage capacities for auto created leaf
* queues under the specified parent
*
* @param queueName The parent queue's name
* @return The policy class configured to manage capacities for auto created
* leaf queues under the specified parent queue
*/
@Private
protected AutoCreatedQueueManagementPolicy
getAutoCreatedQueueManagementPolicyClass(
String queueName) {
String queueManagementPolicyClassName =
getAutoCreatedQueueManagementPolicy(queueName);
LOG.info("Using Auto Created Queue Management Policy: "
+ queueManagementPolicyClassName + " for queue: " + queueName);
try {
Class<?> queueManagementPolicyClazz = getClassByName(
queueManagementPolicyClassName);
if (AutoCreatedQueueManagementPolicy.class.isAssignableFrom(
queueManagementPolicyClazz)) {
return (AutoCreatedQueueManagementPolicy) ReflectionUtils.newInstance(
queueManagementPolicyClazz, this);
} else{
throw new YarnRuntimeException(
"Class: " + queueManagementPolicyClassName + " not instance of "
+ AutoCreatedQueueManagementPolicy.class.getCanonicalName());
}
} catch (ClassNotFoundException e) {
throw new YarnRuntimeException(
"Could not instantiate " + "AutoCreatedQueueManagementPolicy: "
+ queueManagementPolicyClassName + " for queue: " + queueName,
e);
}
}
@VisibleForTesting
public void setAutoCreatedLeafQueueTemplateCapacity(String queuePath,
@Private
public void setAutoCreatedLeafQueueConfigCapacity(String queuePath,
float val) {
String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
queuePath);
@ -1732,13 +1820,31 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
@Private
@VisibleForTesting
public void setAutoCreatedLeafQueueTemplateMaxCapacity(String queuePath,
public void setAutoCreatedLeafQueueConfigMaxCapacity(String queuePath,
float val) {
String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
queuePath);
setMaximumCapacity(leafQueueConfPrefix, val);
}
@VisibleForTesting
@Private
public void setAutoCreatedLeafQueueConfigUserLimit(String queuePath,
int val) {
String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
queuePath);
setUserLimit(leafQueueConfPrefix, val);
}
@VisibleForTesting
@Private
public void setAutoCreatedLeafQueueConfigUserLimitFactor(String queuePath,
float val) {
String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
queuePath);
setUserLimitFactor(leafQueueConfPrefix, val);
}
public static String getUnits(String resourceValue) {
String units;
for (int i = 0; i < resourceValue.length(); i++) {

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preempti
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
/**
@ -94,4 +95,11 @@ public interface CapacitySchedulerContext {
* @return if configuration is mutable
*/
boolean isConfigurationMutable();
/**
* Get clock from scheduler
* @return Clock
*/
Clock getClock();
}

View File

@ -239,7 +239,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
queueName + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
List<CSQueue> childQueues = new ArrayList<>();
AutoCreatedLeafQueue resQueue = new AutoCreatedLeafQueue(csContext,
ReservationQueue resQueue = new ReservationQueue(csContext,
defReservationId, (PlanQueue) queue);
try {
resQueue.setEntitlement(new QueueEntitlement(1.0f, 1.0f));
@ -312,7 +312,8 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
Map<String, CSQueue> newQueues) throws IOException {
// check that all static queues are included in the newQueues list
for (Map.Entry<String, CSQueue> e : queues.entrySet()) {
if (!(e.getValue() instanceof AutoCreatedLeafQueue)) {
if (!(AbstractAutoCreatedLeafQueue.class.isAssignableFrom(e.getValue()
.getClass()))) {
String queueName = e.getKey();
CSQueue oldQueue = e.getValue();
CSQueue newQueue = newQueues.get(queueName);
@ -394,7 +395,6 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
String queueName = e.getKey();
CSQueue existingQueue = e.getValue();
//TODO - Handle case when auto create is disabled on parent queues
if (!newQueues.containsKey(queueName) && !(
existingQueue instanceof AutoCreatedLeafQueue && conf
.isAutoCreateChildQueueEnabled(

View File

@ -138,7 +138,14 @@ public class LeafQueue extends AbstractCSQueue {
@SuppressWarnings({ "unchecked", "rawtypes" })
public LeafQueue(CapacitySchedulerContext cs,
String queueName, CSQueue parent, CSQueue old) throws IOException {
super(cs, queueName, parent, old);
this(cs, cs.getConfiguration(), queueName, parent, old);
}
public LeafQueue(CapacitySchedulerContext cs,
CapacitySchedulerConfiguration configuration,
String queueName, CSQueue parent, CSQueue old) throws
IOException {
super(cs, configuration, queueName, parent, old);
this.scheduler = cs;
this.usersManager = new UsersManager(metrics, this, labelManager, scheduler,
@ -149,17 +156,25 @@ public class LeafQueue extends AbstractCSQueue {
if(LOG.isDebugEnabled()) {
LOG.debug("LeafQueue:" + " name=" + queueName
+ ", fullname=" + getQueuePath());
+ ", fullname=" + getQueuePath());
}
setupQueueConfigs(cs.getClusterResource());
setupQueueConfigs(cs.getClusterResource(), configuration);
}
protected void setupQueueConfigs(Resource clusterResource)
throws IOException {
setupQueueConfigs(clusterResource, csContext.getConfiguration());
}
protected void setupQueueConfigs(Resource clusterResource,
CapacitySchedulerConfiguration conf) throws
IOException {
try {
writeLock.lock();
super.setupQueueConfigs(clusterResource);
CapacitySchedulerConfiguration schedConf = csContext.getConfiguration();
super.setupQueueConfigs(clusterResource, conf);
this.lastClusterResource = clusterResource;
@ -173,8 +188,6 @@ public class LeafQueue extends AbstractCSQueue {
// absoluteMaxAvailCapacity during headroom/userlimit/allocation events)
setQueueResourceLimitsInfo(clusterResource);
CapacitySchedulerConfiguration conf = csContext.getConfiguration();
setOrderingPolicy(
conf.<FiCaSchedulerApp>getAppOrderingPolicy(getQueuePath()));
@ -183,11 +196,13 @@ public class LeafQueue extends AbstractCSQueue {
maxApplications = conf.getMaximumApplicationsPerQueue(getQueuePath());
if (maxApplications < 0) {
int maxGlobalPerQueueApps = conf.getGlobalMaximumApplicationsPerQueue();
int maxGlobalPerQueueApps = schedConf
.getGlobalMaximumApplicationsPerQueue();
if (maxGlobalPerQueueApps > 0) {
maxApplications = maxGlobalPerQueueApps;
} else {
int maxSystemApps = conf.getMaximumSystemApplications();
int maxSystemApps = schedConf.
getMaximumSystemApplications();
maxApplications =
(int) (maxSystemApps * queueCapacities.getAbsoluteCapacity());
}
@ -218,9 +233,11 @@ public class LeafQueue extends AbstractCSQueue {
.join(getAccessibleNodeLabels().iterator(), ',')));
}
nodeLocalityDelay = conf.getNodeLocalityDelay();
rackLocalityAdditionalDelay = conf.getRackLocalityAdditionalDelay();
rackLocalityFullReset = conf.getRackLocalityFullReset();
nodeLocalityDelay = schedConf.getNodeLocalityDelay();
rackLocalityAdditionalDelay = schedConf
.getRackLocalityAdditionalDelay();
rackLocalityFullReset = schedConf
.getRackLocalityFullReset();
// re-init this since max allocation could have changed
this.minimumAllocationFactor = Resources.ratio(resourceCalculator,
@ -507,10 +524,11 @@ public class LeafQueue extends AbstractCSQueue {
}
}
@Override
public void reinitialize(
CSQueue newlyParsedQueue, Resource clusterResource)
throws IOException {
protected void reinitialize(
CSQueue newlyParsedQueue, Resource clusterResource,
CapacitySchedulerConfiguration configuration) throws
IOException {
try {
writeLock.lock();
// Sanity check
@ -535,7 +553,7 @@ public class LeafQueue extends AbstractCSQueue {
+ newMax);
}
setupQueueConfigs(clusterResource);
setupQueueConfigs(clusterResource, configuration);
// queue metrics are updated, more resource may be available
// activate the pending applications if possible
@ -546,6 +564,14 @@ public class LeafQueue extends AbstractCSQueue {
}
}
@Override
public void reinitialize(
CSQueue newlyParsedQueue, Resource clusterResource)
throws IOException {
reinitialize(newlyParsedQueue, clusterResource,
csContext.getConfiguration());
}
@Override
public void submitApplicationAttempt(FiCaSchedulerApp application,
String userName) {
@ -731,7 +757,7 @@ public class LeafQueue extends AbstractCSQueue {
}
}
private void activateApplications() {
protected void activateApplications() {
try {
writeLock.lock();
// limit of allowed resource usage for application masters
@ -1991,10 +2017,18 @@ public class LeafQueue extends AbstractCSQueue {
queueCapacities.setCapacity(capacity);
}
public void setCapacity(String nodeLabel, float capacity) {
queueCapacities.setCapacity(nodeLabel, capacity);
}
public void setAbsoluteCapacity(float absoluteCapacity) {
queueCapacities.setAbsoluteCapacity(absoluteCapacity);
}
public void setAbsoluteCapacity(String nodeLabel, float absoluteCapacity) {
queueCapacities.setAbsoluteCapacity(nodeLabel, absoluteCapacity);
}
public void setMaxApplicationsPerUser(int maxApplicationsPerUser) {
this.maxApplicationsPerUser = maxApplicationsPerUser;
}
@ -2002,7 +2036,12 @@ public class LeafQueue extends AbstractCSQueue {
public void setMaxApplications(int maxApplications) {
this.maxApplications = maxApplications;
}
public void setMaxAMResourcePerQueuePercent(
float maxAMResourcePerQueuePercent) {
this.maxAMResourcePerQueuePercent = maxAMResourcePerQueuePercent;
}
public OrderingPolicy<FiCaSchedulerApp>
getOrderingPolicy() {
return orderingPolicy;

View File

@ -17,13 +17,23 @@
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
.FiCaSchedulerApp;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
* Auto Creation enabled Parent queue. This queue initially does not have any
@ -44,54 +54,125 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
final String queueName, final CSQueue parent, final CSQueue old)
throws IOException {
super(cs, queueName, parent, old);
String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(
csContext.getConfiguration());
this.leafQueueTemplate = initializeLeafQueueConfigs(
leafQueueTemplateConfPrefix).build();
shouldFailAutoCreationWhenGuaranteedCapacityExceeded =
csContext.getConfiguration()
.getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
getQueuePath());
leafQueueTemplate = initializeLeafQueueConfigs().build();
StringBuffer queueInfo = new StringBuffer();
queueInfo.append("Created Managed Parent Queue: ").append(queueName).append(
"]\nwith capacity: [").append(super.getCapacity()).append(
"]\nwith max capacity: [").append(super.getMaximumCapacity()).append(
"\nwith max apps: [").append(leafQueueTemplate.getMaxApps()).append(
"]\nwith max apps per user: [").append(
leafQueueTemplate.getMaxAppsPerUser()).append("]\nwith user limit: [")
.append(leafQueueTemplate.getUserLimit()).append(
"]\nwith user limit factor: [").append(
leafQueueTemplate.getUserLimitFactor()).append("].");
"].");
LOG.info(queueInfo.toString());
initializeQueueManagementPolicy();
}
@Override
public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
throws IOException {
validate(newlyParsedQueue);
super.reinitialize(newlyParsedQueue, clusterResource);
String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(
csContext.getConfiguration());
this.leafQueueTemplate = initializeLeafQueueConfigs(
leafQueueTemplateConfPrefix).build();
try {
writeLock.lock();
validate(newlyParsedQueue);
shouldFailAutoCreationWhenGuaranteedCapacityExceeded =
csContext.getConfiguration()
.getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
getQueuePath());
//validate if capacity is exceeded for child queues
if (shouldFailAutoCreationWhenGuaranteedCapacityExceeded) {
float childCap = sumOfChildCapacities();
if (getCapacity() < childCap) {
throw new IOException(
"Total of Auto Created leaf queues guaranteed capacity : "
+ childCap + " exceeds Parent queue's " + getQueuePath()
+ " guaranteed capacity " + getCapacity() + ""
+ ".Cannot enforce policy to auto"
+ " create queues beyond parent queue's capacity");
}
}
leafQueueTemplate = initializeLeafQueueConfigs().build();
super.reinitialize(newlyParsedQueue, clusterResource);
// run reinitialize on each existing queue, to trigger absolute cap
// recomputations
for (CSQueue res : this.getChildQueues()) {
res.reinitialize(res, clusterResource);
}
//clear state in policy
reinitializeQueueManagementPolicy();
//reassign capacities according to policy
final List<QueueManagementChange> queueManagementChanges =
queueManagementPolicy.computeQueueManagementChanges();
validateAndApplyQueueManagementChanges(queueManagementChanges);
StringBuffer queueInfo = new StringBuffer();
queueInfo.append("Reinitialized Managed Parent Queue: ").append(queueName)
.append("]\nwith capacity: [").append(super.getCapacity()).append(
"]\nwith max capacity: [").append(super.getMaximumCapacity()).append(
"].");
LOG.info(queueInfo.toString());
} catch (YarnException ye) {
LOG.error("Exception while computing policy changes for leaf queue : "
+ getQueueName(), ye);
throw new IOException(ye);
} finally {
writeLock.unlock();
}
}
@Override
protected AutoCreatedLeafQueueTemplate.Builder initializeLeafQueueConfigs(
String queuePath) {
AutoCreatedLeafQueueTemplate.Builder leafQueueTemplate =
super.initializeLeafQueueConfigs(queuePath);
CapacitySchedulerConfiguration conf = csContext.getConfiguration();
String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(conf);
QueueCapacities queueCapacities = new QueueCapacities(false);
CSQueueUtils.loadUpdateAndCheckCapacities(leafQueueTemplateConfPrefix,
csContext.getConfiguration(), queueCapacities, getQueueCapacities());
leafQueueTemplate.capacities(queueCapacities);
shouldFailAutoCreationWhenGuaranteedCapacityExceeded =
conf.getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
private void initializeQueueManagementPolicy() {
queueManagementPolicy =
csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass(
getQueuePath());
return leafQueueTemplate;
queueManagementPolicy.init(csContext, this);
}
private void reinitializeQueueManagementPolicy() {
AutoCreatedQueueManagementPolicy managementPolicy =
csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass(
getQueuePath());
if (!(managementPolicy.getClass().equals(
this.queueManagementPolicy.getClass()))) {
queueManagementPolicy = managementPolicy;
queueManagementPolicy.init(csContext, this);
} else{
queueManagementPolicy.reinitialize(csContext, this);
}
}
protected AutoCreatedLeafQueueConfig.Builder initializeLeafQueueConfigs() {
AutoCreatedLeafQueueConfig.Builder builder =
new AutoCreatedLeafQueueConfig.Builder();
String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(
csContext.getConfiguration());
//Load template configuration
builder.configuration(
super.initializeLeafQueueConfigs(leafQueueTemplateConfPrefix));
//Load template capacities
QueueCapacities queueCapacities = new QueueCapacities(false);
CSQueueUtils.loadUpdateAndCheckCapacities(csContext.getConfiguration()
.getAutoCreatedQueueTemplateConfPrefix(getQueuePath()),
csContext.getConfiguration(), queueCapacities, getQueueCapacities());
builder.capacities(queueCapacities);
return builder;
}
protected void validate(final CSQueue newlyParsedQueue) throws IOException {
@ -106,7 +187,7 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
@Override
public void addChildQueue(CSQueue childQueue)
throws SchedulerDynamicEditException {
throws SchedulerDynamicEditException, IOException {
try {
writeLock.lock();
@ -138,21 +219,164 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
throw new SchedulerDynamicEditException(
"Cannot auto create leaf queue " + leafQueueName + ". Child "
+ "queues capacities have reached parent queue : "
+ parentQueue.getQueuePath() + " guaranteed capacity");
+ parentQueue.getQueuePath() + "'s guaranteed capacity");
}
}
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue;
super.addChildQueue(leafQueue);
//TODO - refresh policy queue after capacity management is added
final AutoCreatedLeafQueueConfig initialLeafQueueTemplate =
queueManagementPolicy.getInitialLeafQueueConfiguration(leafQueue);
leafQueue.reinitializeFromTemplate(initialLeafQueueTemplate);
} finally {
writeLock.unlock();
}
}
public List<FiCaSchedulerApp> getScheduleableApplications() {
try {
readLock.lock();
List<FiCaSchedulerApp> apps = new ArrayList<>();
for (CSQueue childQueue : getChildQueues()) {
apps.addAll(((LeafQueue) childQueue).getApplications());
}
return Collections.unmodifiableList(apps);
} finally {
readLock.unlock();
}
}
public List<FiCaSchedulerApp> getPendingApplications() {
try {
readLock.lock();
List<FiCaSchedulerApp> apps = new ArrayList<>();
for (CSQueue childQueue : getChildQueues()) {
apps.addAll(((LeafQueue) childQueue).getPendingApplications());
}
return Collections.unmodifiableList(apps);
} finally {
readLock.unlock();
}
}
public List<FiCaSchedulerApp> getAllApplications() {
try {
readLock.lock();
List<FiCaSchedulerApp> apps = new ArrayList<>();
for (CSQueue childQueue : getChildQueues()) {
apps.addAll(((LeafQueue) childQueue).getAllApplications());
}
return Collections.unmodifiableList(apps);
} finally {
readLock.unlock();
}
}
public String getLeafQueueConfigPrefix(CapacitySchedulerConfiguration conf) {
return CapacitySchedulerConfiguration.PREFIX + conf
.getAutoCreatedQueueTemplateConfPrefix(getQueuePath());
}
public boolean shouldFailAutoCreationWhenGuaranteedCapacityExceeded() {
return shouldFailAutoCreationWhenGuaranteedCapacityExceeded;
}
/**
* Asynchronously called from scheduler to apply queue management changes
*
* @param queueManagementChanges
*/
public void validateAndApplyQueueManagementChanges(
List<QueueManagementChange> queueManagementChanges)
throws IOException, SchedulerDynamicEditException {
try {
writeLock.lock();
validateQueueManagementChanges(queueManagementChanges);
applyQueueManagementChanges(queueManagementChanges);
AutoCreatedQueueManagementPolicy policy =
getAutoCreatedQueueManagementPolicy();
//acquires write lock on policy
policy.commitQueueManagementChanges(queueManagementChanges);
} finally {
writeLock.unlock();
}
}
private String getLeafQueueConfigPrefix(CapacitySchedulerConfiguration conf) {
return conf.getAutoCreatedQueueTemplateConfPrefix(getQueuePath());
public void validateQueueManagementChanges(
List<QueueManagementChange> queueManagementChanges)
throws SchedulerDynamicEditException {
for (QueueManagementChange queueManagementChange : queueManagementChanges) {
CSQueue childQueue = queueManagementChange.getQueue();
if (!(childQueue instanceof AutoCreatedLeafQueue)) {
throw new SchedulerDynamicEditException(
"queue should be " + "AutoCreatedLeafQueue. Found " + childQueue
.getClass());
}
if (!(AbstractManagedParentQueue.class.
isAssignableFrom(childQueue.getParent().getClass()))) {
LOG.error("Queue " + getQueueName()
+ " is not an instance of PlanQueue or ManagedParentQueue." + " "
+ "Ignoring update " + queueManagementChanges);
throw new SchedulerDynamicEditException(
"Queue " + getQueueName() + " is not a AutoEnabledParentQueue."
+ " Ignoring update " + queueManagementChanges);
}
switch (queueManagementChange.getQueueAction()){
case UPDATE_QUEUE:
AutoCreatedLeafQueueConfig template =
queueManagementChange.getUpdatedQueueTemplate();
((AutoCreatedLeafQueue) childQueue).validateConfigurations(template);
break;
}
}
}
private void applyQueueManagementChanges(
List<QueueManagementChange> queueManagementChanges)
throws SchedulerDynamicEditException, IOException {
for (QueueManagementChange queueManagementChange : queueManagementChanges) {
switch (queueManagementChange.getQueueAction()){
case UPDATE_QUEUE:
AutoCreatedLeafQueue childQueueToBeUpdated =
(AutoCreatedLeafQueue) queueManagementChange.getQueue();
//acquires write lock on leaf queue
childQueueToBeUpdated.reinitializeFromTemplate(
queueManagementChange.getUpdatedQueueTemplate());
break;
}
}
}
public CapacitySchedulerConfiguration getLeafQueueConfigs(
String leafQueueName) {
return getLeafQueueConfigs(getLeafQueueTemplate().getLeafQueueConfigs(),
leafQueueName);
}
public CapacitySchedulerConfiguration getLeafQueueConfigs(
CapacitySchedulerConfiguration templateConfig, String leafQueueName) {
CapacitySchedulerConfiguration leafQueueConfigTemplate = new
CapacitySchedulerConfiguration(new Configuration(false), false);
for (final Iterator<Map.Entry<String, String>> iterator =
templateConfig.iterator(); iterator.hasNext(); ) {
Map.Entry<String, String> confKeyValuePair = iterator.next();
final String name = confKeyValuePair.getKey().replaceFirst(
CapacitySchedulerConfiguration
.AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX,
leafQueueName);
leafQueueConfigTemplate.set(name, confKeyValuePair.getValue());
}
return leafQueueConfigTemplate;
}
}

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -35,61 +36,132 @@ public class PlanQueue extends AbstractManagedParentQueue {
private static final Logger LOG = LoggerFactory.getLogger(PlanQueue.class);
private int maxAppsForReservation;
private int maxAppsPerUserForReservation;
private int userLimit;
private float userLimitFactor;
protected CapacitySchedulerContext schedulerContext;
private boolean showReservationsAsQueues;
public PlanQueue(CapacitySchedulerContext cs, String queueName,
CSQueue parent, CSQueue old) throws IOException {
super(cs, queueName, parent, old);
this.leafQueueTemplate = initializeLeafQueueConfigs(getQueuePath()).build();
this.schedulerContext = cs;
// Set the reservation queue attributes for the Plan
CapacitySchedulerConfiguration conf = cs.getConfiguration();
String queuePath = super.getQueuePath();
int maxAppsForReservation = conf.getMaximumApplicationsPerQueue(queuePath);
showReservationsAsQueues = conf.getShowReservationAsQueues(queuePath);
if (maxAppsForReservation < 0) {
maxAppsForReservation =
(int) (CapacitySchedulerConfiguration.
DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS * super
.getAbsoluteCapacity());
}
int userLimit = conf.getUserLimit(queuePath);
float userLimitFactor = conf.getUserLimitFactor(queuePath);
int maxAppsPerUserForReservation =
(int) (maxAppsForReservation * (userLimit / 100.0f) * userLimitFactor);
updateQuotas(userLimit, userLimitFactor, maxAppsForReservation,
maxAppsPerUserForReservation);
StringBuffer queueInfo = new StringBuffer();
queueInfo.append("Created Plan Queue: ").append(queueName).append(
"]\nwith capacity: [").append(super.getCapacity()).append(
"]\nwith max capacity: [").append(super.getMaximumCapacity()).append(
"\nwith max apps: [").append(leafQueueTemplate.getMaxApps()).append(
"]\nwith max apps per user: [").append(
leafQueueTemplate.getMaxAppsPerUser()).append("]\nwith user limit: [")
.append(leafQueueTemplate.getUserLimit()).append(
"]\nwith user limit factor: [").append(
leafQueueTemplate.getUserLimitFactor()).append("].");
queueInfo.append("Created Plan Queue: ").append(queueName)
.append("\nwith capacity: [").append(super.getCapacity())
.append("]\nwith max capacity: [").append(super.getMaximumCapacity())
.append("\nwith max reservation apps: [").append(maxAppsForReservation)
.append("]\nwith max reservation apps per user: [")
.append(maxAppsPerUserForReservation).append("]\nwith user limit: [")
.append(userLimit).append("]\nwith user limit factor: [")
.append(userLimitFactor).append("].");
LOG.info(queueInfo.toString());
}
@Override
public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
throws IOException {
validate(newlyParsedQueue);
super.reinitialize(newlyParsedQueue, clusterResource);
this.leafQueueTemplate = initializeLeafQueueConfigs(getQueuePath()).build();
public void reinitialize(CSQueue newlyParsedQueue,
Resource clusterResource) throws IOException {
try {
writeLock.lock();
// Sanity check
if (!(newlyParsedQueue instanceof PlanQueue) || !newlyParsedQueue
.getQueuePath().equals(getQueuePath())) {
throw new IOException(
"Trying to reinitialize " + getQueuePath() + " from "
+ newlyParsedQueue.getQueuePath());
}
PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
if (newlyParsedParentQueue.getChildQueues().size() != 1) {
throw new IOException(
"Reservable Queue should not have sub-queues in the"
+ "configuration expect the default reservation queue");
}
// Set new configs
setupQueueConfigs(clusterResource);
updateQuotas(newlyParsedParentQueue.userLimit,
newlyParsedParentQueue.userLimitFactor,
newlyParsedParentQueue.maxAppsForReservation,
newlyParsedParentQueue.maxAppsPerUserForReservation);
// run reinitialize on each existing queue, to trigger absolute cap
// recomputations
for (CSQueue res : this.getChildQueues()) {
res.reinitialize(res, clusterResource);
}
showReservationsAsQueues =
newlyParsedParentQueue.showReservationsAsQueues;
} finally {
writeLock.unlock();
}
}
@Override
protected AutoCreatedLeafQueueTemplate.Builder initializeLeafQueueConfigs
(String queuePath) {
AutoCreatedLeafQueueTemplate.Builder leafQueueTemplate = super
.initializeLeafQueueConfigs
(queuePath);
showReservationsAsQueues = csContext.getConfiguration()
.getShowReservationAsQueues(queuePath);
return leafQueueTemplate;
private void updateQuotas(int userLimit, float userLimitFactor,
int maxAppsForReservation, int maxAppsPerUserForReservation) {
this.userLimit = userLimit;
this.userLimitFactor = userLimitFactor;
this.maxAppsForReservation = maxAppsForReservation;
this.maxAppsPerUserForReservation = maxAppsPerUserForReservation;
}
protected void validate(final CSQueue newlyParsedQueue) throws IOException {
// Sanity check
if (!(newlyParsedQueue instanceof PlanQueue) || !newlyParsedQueue
.getQueuePath().equals(getQueuePath())) {
throw new IOException(
"Trying to reinitialize " + getQueuePath() + " from "
+ newlyParsedQueue.getQueuePath());
}
/**
* Number of maximum applications for each of the reservations in this Plan.
*
* @return maxAppsForreservation
*/
public int getMaxApplicationsForReservations() {
return maxAppsForReservation;
}
PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
/**
* Number of maximum applications per user for each of the reservations in
* this Plan.
*
* @return maxAppsPerUserForreservation
*/
public int getMaxApplicationsPerUserForReservation() {
return maxAppsPerUserForReservation;
}
if (newlyParsedParentQueue.getChildQueues().size() != 1) {
throw new IOException(
"Reservable Queue should not have sub-queues in the"
+ "configuration expect the default reservation queue");
}
/**
* User limit value for each of the reservations in this Plan.
*
* @return userLimit
*/
public int getUserLimitForReservation() {
return userLimit;
}
/**
* User limit factor value for each of the reservations in this Plan.
*
* @return userLimitFactor
*/
public float getUserLimitFactor() {
return userLimitFactor;
}
/**
@ -98,4 +170,4 @@ public class PlanQueue extends AbstractManagedParentQueue {
public boolean showReservationsAsQueues() {
return showReservationsAsQueues;
}
}
}

View File

@ -0,0 +1,148 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.records.QueueState;
/**
* Encapsulates Queue entitlement and state updates needed
* for adjusting capacity dynamically
*
*/
@Private
@Unstable
public abstract class QueueManagementChange {
private final CSQueue queue;
/**
* Updating the queue may involve entitlement updates
* and/or QueueState changes
*
* QueueAction can potentially be enhanced
* for adding, removing queues for queue management
*/
public enum QueueAction {
UPDATE_QUEUE
}
private AutoCreatedLeafQueueConfig
queueTemplateUpdate;
private final QueueAction queueAction;
/**
* Updated Queue state with the new entitlement
*/
private QueueState transitionToQueueState;
public QueueManagementChange(final CSQueue queue,
final QueueAction queueAction) {
this.queue = queue;
this.queueAction = queueAction;
}
public QueueManagementChange(final CSQueue queue,
final QueueAction queueAction, QueueState targetQueueState,
final AutoCreatedLeafQueueConfig
queueTemplateUpdates) {
this(queue, queueAction, queueTemplateUpdates);
this.transitionToQueueState = targetQueueState;
}
public QueueManagementChange(final CSQueue queue,
final QueueAction queueAction,
final AutoCreatedLeafQueueConfig
queueTemplateUpdates) {
this(queue, queueAction);
this.queueTemplateUpdate = queueTemplateUpdates;
}
public QueueState getTransitionToQueueState() {
return transitionToQueueState;
}
public CSQueue getQueue() {
return queue;
}
public AutoCreatedLeafQueueConfig getUpdatedQueueTemplate() {
return queueTemplateUpdate;
}
public QueueAction getQueueAction() {
return queueAction;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (!(o instanceof QueueManagementChange))
return false;
QueueManagementChange that = (QueueManagementChange) o;
if (queue != null ? !queue.equals(that.queue) : that.queue != null)
return false;
if (queueTemplateUpdate != null ? !queueTemplateUpdate.equals(
that.queueTemplateUpdate) : that.queueTemplateUpdate != null)
return false;
if (queueAction != that.queueAction)
return false;
return transitionToQueueState == that.transitionToQueueState;
}
@Override
public int hashCode() {
int result = queue != null ? queue.hashCode() : 0;
result = 31 * result + (queueTemplateUpdate != null ?
queueTemplateUpdate.hashCode() :
0);
result = 31 * result + (queueAction != null ? queueAction.hashCode() : 0);
result = 31 * result + (transitionToQueueState != null ?
transitionToQueueState.hashCode() :
0);
return result;
}
@Override
public String toString() {
return "QueueManagementChange{" + "queue=" + queue
+ ", updatedEntitlementsByPartition=" + queueTemplateUpdate
+ ", queueAction=" + queueAction + ", transitionToQueueState="
+ transitionToQueueState + '}';
}
public static class UpdateQueue extends QueueManagementChange {
public UpdateQueue(final CSQueue queue, QueueState targetQueueState,
final AutoCreatedLeafQueueConfig
queueTemplateUpdate) {
super(queue, QueueAction.UPDATE_QUEUE, targetQueueState,
queueTemplateUpdate);
}
public UpdateQueue(final CSQueue queue,
final AutoCreatedLeafQueueConfig
queueTemplateUpdate) {
super(queue, QueueAction.UPDATE_QUEUE, queueTemplateUpdate);
}
}
}

View File

@ -0,0 +1,272 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
.QueueManagementChangeEvent;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import java.text.MessageFormat;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* Queue Management scheduling policy for managed parent queues which enable
* auto child queue creation
*/
public class QueueManagementDynamicEditPolicy implements SchedulingEditPolicy {
private static final Log LOG =
LogFactory.getLog(QueueManagementDynamicEditPolicy.class);
private Clock clock;
// Pointer to other RM components
private RMContext rmContext;
private ResourceCalculator rc;
private CapacityScheduler scheduler;
private RMNodeLabelsManager nlm;
private long monitoringInterval;
private Set<String> managedParentQueues = new HashSet<>();
/**
* Instantiated by CapacitySchedulerConfiguration
*/
public QueueManagementDynamicEditPolicy() {
clock = SystemClock.getInstance();
}
@SuppressWarnings("unchecked")
@VisibleForTesting
public QueueManagementDynamicEditPolicy(RMContext context,
CapacityScheduler scheduler) {
init(context.getYarnConfiguration(), context, scheduler);
}
@SuppressWarnings("unchecked")
@VisibleForTesting
public QueueManagementDynamicEditPolicy(RMContext context,
CapacityScheduler scheduler, Clock clock) {
init(context.getYarnConfiguration(), context, scheduler);
this.clock = clock;
}
@Override
public void init(final Configuration config, final RMContext context,
final ResourceScheduler sched) {
LOG.info("Queue Management Policy monitor:" + this.
getClass().getCanonicalName());
assert null == scheduler : "Unexpected duplicate call to init";
if (!(sched instanceof CapacityScheduler)) {
throw new YarnRuntimeException("Class " +
sched.getClass().getCanonicalName() + " not instance of " +
CapacityScheduler.class.getCanonicalName());
}
rmContext = context;
scheduler = (CapacityScheduler) sched;
clock = scheduler.getClock();
rc = scheduler.getResourceCalculator();
nlm = scheduler.getRMContext().getNodeLabelManager();
CapacitySchedulerConfiguration csConfig = scheduler.getConfiguration();
monitoringInterval = csConfig.getLong(
CapacitySchedulerConfiguration.QUEUE_MANAGEMENT_MONITORING_INTERVAL,
CapacitySchedulerConfiguration.
DEFAULT_QUEUE_MANAGEMENT_MONITORING_INTERVAL);
initQueues();
}
/**
* Reinitializes queues(Called on scheduler.reinitialize)
* @param config Configuration
* @param context The resourceManager's context
* @param sched The scheduler
*/
public void reinitialize(final Configuration config, final RMContext context,
final ResourceScheduler sched) {
//TODO - Wire with scheduler reinitialize and remove initQueues below?
initQueues();
}
private void initQueues() {
managedParentQueues.clear();
for (Map.Entry<String, CSQueue> queues : scheduler
.getCapacitySchedulerQueueManager()
.getQueues().entrySet()) {
String queueName = queues.getKey();
CSQueue queue = queues.getValue();
if ( queue instanceof ManagedParentQueue) {
managedParentQueues.add(queueName);
}
}
}
@Override
public void editSchedule() {
long startTs = clock.getTime();
initQueues();
manageAutoCreatedLeafQueues();
if (LOG.isDebugEnabled()) {
LOG.debug("Total time used=" + (clock.getTime() - startTs) + " ms.");
}
}
@VisibleForTesting
List<QueueManagementChange> manageAutoCreatedLeafQueues()
{
List<QueueManagementChange> queueManagementChanges = new ArrayList<>();
// All partitions to look at
//Proceed only if there are queues to process
if (managedParentQueues.size() > 0) {
for (String parentQueueName : managedParentQueues) {
ManagedParentQueue parentQueue =
(ManagedParentQueue) scheduler.getCapacitySchedulerQueueManager().
getQueue(parentQueueName);
queueManagementChanges.addAll(
computeQueueManagementChanges
(parentQueue));
}
}
return queueManagementChanges;
}
@VisibleForTesting
List<QueueManagementChange> computeQueueManagementChanges
(ManagedParentQueue parentQueue) {
List<QueueManagementChange> queueManagementChanges =
Collections.emptyList();
if (!parentQueue.shouldFailAutoCreationWhenGuaranteedCapacityExceeded()) {
AutoCreatedQueueManagementPolicy policyClazz =
parentQueue.getAutoCreatedQueueManagementPolicy();
long startTime = 0;
try {
if (LOG.isDebugEnabled()) {
LOG.debug(MessageFormat
.format("Trying to use {0} to compute preemption "
+ "candidates",
policyClazz.getClass().getName()));
startTime = clock.getTime();
}
queueManagementChanges = policyClazz.computeQueueManagementChanges();
//Scheduler update is asynchronous
if (queueManagementChanges.size() > 0) {
QueueManagementChangeEvent queueManagementChangeEvent =
new QueueManagementChangeEvent(parentQueue,
queueManagementChanges);
scheduler.getRMContext().getDispatcher().getEventHandler().handle(
queueManagementChangeEvent);
}
if (LOG.isDebugEnabled()) {
LOG.debug(MessageFormat.format("{0} uses {1} millisecond"
+ " to run",
policyClazz.getClass().getName(), clock.getTime()
- startTime));
if (queueManagementChanges.size() > 0) {
LOG.debug(" Updated queue management updates for parent queue"
+ " ["
+ parentQueue.getQueueName() + ": [\n" + queueManagementChanges
.toString() + "\n]");
}
}
} catch (YarnException e) {
LOG.error(
"Could not compute child queue management updates for parent "
+ "queue "
+ parentQueue.getQueueName(), e);
}
} else{
if (LOG.isDebugEnabled()) {
LOG.debug(
"Skipping queue management updates for parent queue "
+ parentQueue
.getQueuePath() + " "
+ "since configuration for auto creating queue's beyond "
+ "parent's "
+ "guaranteed capacity is disabled");
}
}
return queueManagementChanges;
}
@Override
public long getMonitoringInterval() {
return monitoringInterval;
}
@Override
public String getPolicyName() {
return "QueueManagementDynamicEditPolicy";
}
public ResourceCalculator getResourceCalculator() {
return rc;
}
public RMContext getRmContext() {
return rmContext;
}
public ResourceCalculator getRC() {
return rc;
}
public CapacityScheduler getScheduler() {
return scheduler;
}
public Set<String> getManagedParentQueues() {
return managedParentQueues;
}
}

View File

@ -0,0 +1,91 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.io.IOException;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This represents a dynamic {@link LeafQueue} managed by the
* {@link ReservationSystem}
*
*/
public class ReservationQueue extends AbstractAutoCreatedLeafQueue {
private static final Logger LOG = LoggerFactory
.getLogger(ReservationQueue.class);
private PlanQueue parent;
public ReservationQueue(CapacitySchedulerContext cs, String queueName,
PlanQueue parent) throws IOException {
super(cs, queueName, parent, null);
// the following parameters are common to all reservation in the plan
updateQuotas(parent.getUserLimitForReservation(),
parent.getUserLimitFactor(),
parent.getMaxApplicationsForReservations(),
parent.getMaxApplicationsPerUserForReservation());
this.parent = parent;
}
@Override
public void reinitialize(CSQueue newlyParsedQueue,
Resource clusterResource) throws IOException {
try {
writeLock.lock();
// Sanity check
if (!(newlyParsedQueue instanceof ReservationQueue) || !newlyParsedQueue
.getQueuePath().equals(getQueuePath())) {
throw new IOException(
"Trying to reinitialize " + getQueuePath() + " from "
+ newlyParsedQueue.getQueuePath());
}
super.reinitialize(newlyParsedQueue, clusterResource);
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
this, labelManager, null);
updateQuotas(parent.getUserLimitForReservation(),
parent.getUserLimitFactor(),
parent.getMaxApplicationsForReservations(),
parent.getMaxApplicationsPerUserForReservation());
} finally {
writeLock.unlock();
}
}
private void updateQuotas(int userLimit, float userLimitFactor,
int maxAppsForReservation, int maxAppsPerUserForReservation) {
setUserLimit(userLimit);
setUserLimitFactor(userLimitFactor);
setMaxApplications(maxAppsForReservation);
maxApplicationsPerUser = maxAppsPerUserForReservation;
}
@Override
protected void setupConfigurableCapacities(CapacitySchedulerConfiguration
configuration) {
super.setupConfigurableCapacities(queueCapacities);
}
}

View File

@ -0,0 +1,745 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.queuemanagement;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.AbstractAutoCreatedLeafQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.AutoCreatedLeafQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.AutoCreatedLeafQueueConfig;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.AutoCreatedQueueManagementPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.CapacitySchedulerContext;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.LeafQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.ManagedParentQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.ParentQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.QueueCapacities;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.QueueManagementChange;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
.FiCaSchedulerApp;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.MonotonicClock;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
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
.capacity.CSQueueUtils.EPSILON;
/**
* Capacity Management policy for auto created leaf queues
* <p>
* Assigns capacity if available to leaf queues based on application
* submission order i.e leaf queues are assigned capacity in FCFS order based
* on application submission time. Updates leaf queue capacities to 0 when
* there are no pending or running apps under that queue.
*/
public class GuaranteedOrZeroCapacityOverTimePolicy
implements AutoCreatedQueueManagementPolicy {
private CapacitySchedulerContext scheduler;
private ManagedParentQueue managedParentQueue;
private static final Log LOG = LogFactory.getLog(
GuaranteedOrZeroCapacityOverTimePolicy.class);
private AutoCreatedLeafQueueConfig ZERO_CAPACITY_ENTITLEMENT;
private ReentrantReadWriteLock.WriteLock writeLock;
private ReentrantReadWriteLock.ReadLock readLock;
private ParentQueueState parentQueueState = new ParentQueueState();
private AutoCreatedLeafQueueConfig leafQueueTemplate;
private QueueCapacities leafQueueTemplateCapacities;
private Map<String, LeafQueueState> leafQueueStateMap = new HashMap<>();
private Clock clock = new MonotonicClock();
private class LeafQueueState {
private AtomicBoolean isActive = new AtomicBoolean(false);
private long mostRecentActivationTime;
private long mostRecentDeactivationTime;
public long getMostRecentActivationTime() {
return mostRecentActivationTime;
}
public long getMostRecentDeactivationTime() {
return mostRecentDeactivationTime;
}
/**
* Is the queue currently active or deactivated?
*
* @return true if Active else false
*/
public boolean isActive() {
return isActive.get();
}
private boolean activate() {
boolean ret = isActive.compareAndSet(false, true);
mostRecentActivationTime = clock.getTime();
return ret;
}
private boolean deactivate() {
boolean ret = isActive.compareAndSet(true, false);
mostRecentDeactivationTime = clock.getTime();
return ret;
}
}
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 Map<String, Float> totalAbsoluteActivatedChildQueueCapacityByLabel =
new HashMap<String, Float>();
private float getAbsoluteActivatedChildQueueCapacity() {
return getAbsoluteActivatedChildQueueCapacity(NO_LABEL);
}
private float getAbsoluteActivatedChildQueueCapacity(String nodeLabel) {
try {
readLock.lock();
Float totalActivatedCapacity = getByLabel(nodeLabel);
if (totalActivatedCapacity != null) {
return totalActivatedCapacity;
} else{
return 0;
}
} finally {
readLock.unlock();
}
}
private void incAbsoluteActivatedChildCapacity(String nodeLabel,
float childQueueCapacity) {
try {
writeLock.lock();
Float activatedChildCapacity = getByLabel(nodeLabel);
if (activatedChildCapacity != null) {
setByLabel(nodeLabel, activatedChildCapacity + childQueueCapacity);
} else{
setByLabel(nodeLabel, childQueueCapacity);
}
} finally {
writeLock.unlock();
}
}
private void decAbsoluteActivatedChildCapacity(String nodeLabel,
float childQueueCapacity) {
try {
writeLock.lock();
Float activatedChildCapacity = getByLabel(nodeLabel);
if (activatedChildCapacity != null) {
setByLabel(nodeLabel, activatedChildCapacity - childQueueCapacity);
} else{
setByLabel(nodeLabel, childQueueCapacity);
}
} finally {
writeLock.unlock();
}
}
Float getByLabel(String label) {
return totalAbsoluteActivatedChildQueueCapacityByLabel.get(label);
}
Float setByLabel(String label, float val) {
return totalAbsoluteActivatedChildQueueCapacityByLabel.put(label, val);
}
void clear() {
totalAbsoluteActivatedChildQueueCapacityByLabel.clear();
}
}
/**
* Comparator that orders applications by their submit time
*/
private class PendingApplicationComparator
implements Comparator<FiCaSchedulerApp> {
@Override
public int compare(FiCaSchedulerApp app1, FiCaSchedulerApp app2) {
RMApp rmApp1 = scheduler.getRMContext().getRMApps().get(
app1.getApplicationId());
RMApp rmApp2 = scheduler.getRMContext().getRMApps().get(
app2.getApplicationId());
if (rmApp1 != null && rmApp2 != null) {
return Long.compare(rmApp1.getSubmitTime(), rmApp2.getSubmitTime());
} else if (rmApp1 != null) {
return -1;
} else if (rmApp2 != null) {
return 1;
} else{
return 0;
}
}
}
private PendingApplicationComparator applicationComparator =
new PendingApplicationComparator();
@Override
public void init(final CapacitySchedulerContext schedulerContext,
final ParentQueue parentQueue) {
this.scheduler = schedulerContext;
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
readLock = lock.readLock();
writeLock = lock.writeLock();
if (!(parentQueue instanceof ManagedParentQueue)) {
throw new IllegalArgumentException(
"Expected instance of type " + ManagedParentQueue.class);
}
this.managedParentQueue = (ManagedParentQueue) parentQueue;
initializeLeafQueueTemplate(this.managedParentQueue);
LOG.info(
"Initialized queue management policy for parent queue " + parentQueue
.getQueueName() + " with leaf queue template capacities : ["
+ leafQueueTemplate.getQueueCapacities() + "]");
}
private void initializeLeafQueueTemplate(ManagedParentQueue parentQueue) {
leafQueueTemplate = parentQueue.getLeafQueueTemplate();
leafQueueTemplateCapacities = leafQueueTemplate.getQueueCapacities();
ZERO_CAPACITY_ENTITLEMENT = buildTemplate(0.0f,
leafQueueTemplateCapacities.getMaximumCapacity());
}
@Override
public List<QueueManagementChange> computeQueueManagementChanges()
throws SchedulerDynamicEditException {
//TODO : Add support for node labels on leaf queue template configurations
//synch/add missing leaf queue(s) if any to state
updateLeafQueueState();
try {
readLock.lock();
List<QueueManagementChange> queueManagementChanges = new ArrayList<>();
// check if any leaf queues need to be deactivated based on pending
// applications and
float parentAbsoluteCapacity =
managedParentQueue.getQueueCapacities().getAbsoluteCapacity();
float leafQueueTemplateAbsoluteCapacity =
leafQueueTemplateCapacities.getAbsoluteCapacity();
Map<String, QueueCapacities> deactivatedLeafQueues =
deactivateLeafQueuesIfInActive(managedParentQueue, queueManagementChanges);
float deactivatedCapacity = getTotalDeactivatedCapacity(
deactivatedLeafQueues);
float sumOfChildQueueActivatedCapacity = parentQueueState.
getAbsoluteActivatedChildQueueCapacity();
//Check if we need to activate anything at all?
float availableCapacity = getAvailableCapacity(parentAbsoluteCapacity,
deactivatedCapacity, sumOfChildQueueActivatedCapacity);
if (LOG.isDebugEnabled()) {
LOG.debug(
"Parent queue : " + managedParentQueue.getQueueName() + " absCapacity = "
+ parentAbsoluteCapacity + ", leafQueueAbsoluteCapacity = "
+ leafQueueTemplateAbsoluteCapacity + ", deactivatedCapacity = "
+ deactivatedCapacity + " , absChildActivatedCapacity = "
+ sumOfChildQueueActivatedCapacity + ", availableCapacity = "
+ availableCapacity);
}
if (availableCapacity >= leafQueueTemplateAbsoluteCapacity) {
//sort applications across leaf queues by submit time
List<FiCaSchedulerApp> pendingApps = getSortedPendingApplications();
if (pendingApps.size() > 0) {
int maxLeafQueuesTobeActivated = getMaxLeavesToBeActivated(
availableCapacity, leafQueueTemplateAbsoluteCapacity,
pendingApps.size());
if (LOG.isDebugEnabled()) {
LOG.debug("Found " + maxLeafQueuesTobeActivated
+ " leaf queues to be activated with " + pendingApps.size()
+ " apps ");
}
LinkedHashSet<String> leafQueuesToBeActivated = getSortedLeafQueues(
pendingApps, maxLeafQueuesTobeActivated,
deactivatedLeafQueues.keySet());
//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 + "]");
}
}
}
}
return queueManagementChanges;
} finally {
readLock.unlock();
}
}
private float getTotalDeactivatedCapacity(
Map<String, QueueCapacities> deactivatedLeafQueues) {
float deactivatedCapacity = 0;
for (Iterator<Map.Entry<String, QueueCapacities>> iterator =
deactivatedLeafQueues.entrySet().iterator(); iterator.hasNext(); ) {
Map.Entry<String, QueueCapacities> deactivatedQueueCapacity =
iterator.next();
deactivatedCapacity +=
deactivatedQueueCapacity.getValue().getAbsoluteCapacity();
}
return deactivatedCapacity;
}
@VisibleForTesting
void updateLeafQueueState() {
try {
writeLock.lock();
Set<String> newQueues = new HashSet<>();
for (CSQueue newQueue : managedParentQueue.getChildQueues()) {
if (newQueue instanceof LeafQueue) {
addLeafQueueStateIfNotExists((LeafQueue) newQueue);
newQueues.add(newQueue.getQueueName());
}
}
for (Iterator<Map.Entry<String, LeafQueueState>> itr =
leafQueueStateMap.entrySet().iterator(); itr.hasNext(); ) {
Map.Entry<String, LeafQueueState> e = itr.next();
String queueName = e.getKey();
if (!newQueues.contains(queueName)) {
itr.remove();
}
}
} finally {
writeLock.unlock();
}
}
private LinkedHashSet<String> getSortedLeafQueues(
final List<FiCaSchedulerApp> pendingApps, int leafQueuesNeeded,
Set<String> deactivatedQueues) throws SchedulerDynamicEditException {
LinkedHashSet<String> leafQueues = new LinkedHashSet<>(leafQueuesNeeded);
int ctr = 0;
for (FiCaSchedulerApp app : pendingApps) {
AutoCreatedLeafQueue leafQueue =
(AutoCreatedLeafQueue) app.getCSLeafQueue();
String leafQueueName = leafQueue.getQueueName();
//Check if leafQueue is not active already and has any pending apps
if (ctr < leafQueuesNeeded) {
if (!isActive(leafQueue)) {
if (!deactivatedQueues.contains(leafQueueName)) {
if (addLeafQueueIfNotExists(leafQueues, leafQueueName)) {
ctr++;
}
}
}
} else{
break;
}
}
return leafQueues;
}
private boolean addLeafQueueIfNotExists(Set<String> leafQueues,
String leafQueueName) {
boolean ret = false;
if (!leafQueues.contains(leafQueueName)) {
ret = leafQueues.add(leafQueueName);
}
return ret;
}
@VisibleForTesting
public boolean isActive(final AutoCreatedLeafQueue leafQueue)
throws SchedulerDynamicEditException {
try {
readLock.lock();
LeafQueueState leafQueueStatus = getLeafQueueState(leafQueue);
return leafQueueStatus.isActive();
} finally {
readLock.unlock();
}
}
private Map<String, QueueCapacities> deactivateLeafQueuesIfInActive(
ParentQueue parentQueue,
List<QueueManagementChange> queueManagementChanges)
throws SchedulerDynamicEditException {
Map<String, QueueCapacities> deactivatedQueues = new HashMap<>();
for (CSQueue childQueue : parentQueue.getChildQueues()) {
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue;
if (isActive(leafQueue) && !hasPendingApps(leafQueue)) {
queueManagementChanges.add(
new QueueManagementChange.UpdateQueue(leafQueue,
ZERO_CAPACITY_ENTITLEMENT));
deactivatedQueues.put(leafQueue.getQueueName(),
leafQueueTemplateCapacities);
} else{
if (LOG.isDebugEnabled()) {
LOG.debug(" Leaf queue has pending applications : " + leafQueue
.getNumApplications() + ".Skipping deactivation for "
+ leafQueue);
}
}
}
if (LOG.isDebugEnabled()) {
if (deactivatedQueues.size() > 0) {
LOG.debug("Deactivated leaf queues : " + deactivatedQueues);
}
}
return deactivatedQueues;
}
private void computeQueueManagementChanges(
Set<String> leafQueuesToBeActivated,
List<QueueManagementChange> queueManagementChanges,
final float availableCapacity,
final float leafQueueTemplateAbsoluteCapacity) {
float curAvailableCapacity = availableCapacity;
for (String curLeafQueue : leafQueuesToBeActivated) {
// Activate queues if capacity is available
if (curAvailableCapacity >= leafQueueTemplateAbsoluteCapacity) {
AutoCreatedLeafQueue leafQueue =
(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);
}
}
}
}
@VisibleForTesting
public int getMaxLeavesToBeActivated(float availableCapacity,
float childQueueAbsoluteCapacity, int numPendingApps)
throws SchedulerDynamicEditException {
if (childQueueAbsoluteCapacity > 0) {
int numLeafQueuesNeeded = (int) Math.floor(
availableCapacity / childQueueAbsoluteCapacity);
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");
}
}
private float getAvailableCapacity(float parentAbsCapacity,
float deactivatedAbsCapacity, float totalChildQueueActivatedCapacity) {
return parentAbsCapacity - totalChildQueueActivatedCapacity
+ deactivatedAbsCapacity + EPSILON;
}
/**
* Commit queue management changes - which involves updating required state
* on parent/underlying leaf queues
*
* @param queueManagementChanges Queue Management changes to commit
* @throws SchedulerDynamicEditException when validation fails
*/
@Override
public void commitQueueManagementChanges(
List<QueueManagementChange> queueManagementChanges)
throws SchedulerDynamicEditException {
try {
writeLock.lock();
for (QueueManagementChange queueManagementChange :
queueManagementChanges) {
AutoCreatedLeafQueueConfig updatedQueueTemplate =
queueManagementChange.getUpdatedQueueTemplate();
CSQueue queue = queueManagementChange.getQueue();
if (!(queue instanceof AutoCreatedLeafQueue)) {
throw new SchedulerDynamicEditException(
"Expected queue management change for AutoCreatedLeafQueue. "
+ "Found " + queue.getClass().getName());
}
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) queue;
if (updatedQueueTemplate.getQueueCapacities().getCapacity() > 0) {
if (isActive(leafQueue)) {
if (LOG.isDebugEnabled()) {
LOG.debug(
"Queue is already active. Skipping activation : " + queue
.getQueuePath());
}
} else{
activate(leafQueue);
}
} else{
if (!isActive(leafQueue)) {
if (LOG.isDebugEnabled()) {
LOG.debug(
"Queue is already de-activated. " + "Skipping de-activation "
+ ": " + leafQueue.getQueuePath());
}
} else{
deactivate(leafQueue);
}
}
}
} finally {
writeLock.unlock();
}
}
private void activate(final AutoCreatedLeafQueue leafQueue)
throws SchedulerDynamicEditException {
try {
writeLock.lock();
getLeafQueueState(leafQueue).activate();
parentQueueState.incAbsoluteActivatedChildCapacity(NO_LABEL,
leafQueueTemplateCapacities.getAbsoluteCapacity());
} finally {
writeLock.unlock();
}
}
private void deactivate(final AutoCreatedLeafQueue leafQueue)
throws SchedulerDynamicEditException {
try {
writeLock.lock();
getLeafQueueState(leafQueue).deactivate();
for (String nodeLabel : managedParentQueue.getQueueCapacities()
.getExistingNodeLabels()) {
parentQueueState.decAbsoluteActivatedChildCapacity(nodeLabel,
leafQueueTemplateCapacities.getAbsoluteCapacity());
}
} finally {
writeLock.unlock();
}
}
public boolean hasPendingApps(final AutoCreatedLeafQueue leafQueue) {
return leafQueue.getNumApplications() > 0;
}
@Override
public void reinitialize(CapacitySchedulerContext schedulerContext,
final ParentQueue parentQueue) {
if (!(parentQueue instanceof ManagedParentQueue)) {
throw new IllegalStateException(
"Expected instance of type " + ManagedParentQueue.class + " found "
+ " : " + parentQueue.getClass());
}
if (this.managedParentQueue != null && !parentQueue.getQueuePath().equals(
this.managedParentQueue.getQueuePath())) {
throw new IllegalStateException(
"Expected parent queue path to match " + this.managedParentQueue
.getQueuePath() + " found : " + parentQueue.getQueuePath());
}
this.managedParentQueue = (ManagedParentQueue) parentQueue;
initializeLeafQueueTemplate(this.managedParentQueue);
//clear state
parentQueueState.clear();
clearLeafQueueState();
LOG.info(
"Reinitialized queue management policy for parent queue "
+ parentQueue.getQueueName() +" with leaf queue template "
+ "capacities : ["
+ leafQueueTemplate.getQueueCapacities() + "]");
}
@Override
public AutoCreatedLeafQueueConfig getInitialLeafQueueConfiguration(
AbstractAutoCreatedLeafQueue leafQueue)
throws SchedulerDynamicEditException {
if ( !(leafQueue instanceof AutoCreatedLeafQueue)) {
throw new SchedulerDynamicEditException("Not an instance of "
+ "AutoCreatedLeafQueue : " + leafQueue.getClass());
}
AutoCreatedLeafQueue autoCreatedLeafQueue =
(AutoCreatedLeafQueue) leafQueue;
AutoCreatedLeafQueueConfig template = ZERO_CAPACITY_ENTITLEMENT;
try {
writeLock.lock();
if (!addLeafQueueStateIfNotExists(leafQueue)) {
LOG.error("Leaf queue already exists in state : " + getLeafQueueState(
leafQueue));
throw new SchedulerDynamicEditException(
"Leaf queue already exists in state : " + getLeafQueueState(
leafQueue));
}
float availableCapacity = getAvailableCapacity(
managedParentQueue.getQueueCapacities().getAbsoluteCapacity(), 0,
parentQueueState.getAbsoluteActivatedChildQueueCapacity());
if (availableCapacity >= leafQueueTemplateCapacities
.getAbsoluteCapacity()) {
activate(autoCreatedLeafQueue);
template = buildTemplate(leafQueueTemplateCapacities.getCapacity(),
leafQueueTemplateCapacities.getMaximumCapacity());
}
} finally {
writeLock.unlock();
}
return template;
}
@VisibleForTesting
LeafQueueState getLeafQueueState(LeafQueue queue)
throws SchedulerDynamicEditException {
try {
readLock.lock();
String queueName = queue.getQueueName();
if (!containsLeafQueue(queueName)) {
throw new SchedulerDynamicEditException(
"Could not find leaf queue in " + "state " + queueName);
} else{
return leafQueueStateMap.get(queueName);
}
} finally {
readLock.unlock();
}
}
@VisibleForTesting
public float getAbsoluteActivatedChildQueueCapacity() {
return parentQueueState.getAbsoluteActivatedChildQueueCapacity();
}
private List<FiCaSchedulerApp> getSortedPendingApplications() {
List<FiCaSchedulerApp> apps = new ArrayList<>(
managedParentQueue.getAllApplications());
Collections.sort(apps, applicationComparator);
return apps;
}
private AutoCreatedLeafQueueConfig buildTemplate(float capacity,
float maxCapacity) {
AutoCreatedLeafQueueConfig.Builder templateBuilder =
new AutoCreatedLeafQueueConfig.Builder();
QueueCapacities capacities = new QueueCapacities(false);
templateBuilder.capacities(capacities);
for (String nodeLabel : managedParentQueue.getQueueCapacities()
.getExistingNodeLabels()) {
capacities.setCapacity(nodeLabel, capacity);
capacities.setMaximumCapacity(nodeLabel, maxCapacity);
}
return new AutoCreatedLeafQueueConfig(templateBuilder);
}
}

View File

@ -43,4 +43,26 @@ public class QueueEntitlement {
public void setCapacity(float capacity) {
this.capacity = capacity;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (!(o instanceof QueueEntitlement))
return false;
QueueEntitlement that = (QueueEntitlement) o;
if (Float.compare(that.capacity, capacity) != 0)
return false;
return Float.compare(that.maxCapacity, maxCapacity) == 0;
}
@Override
public int hashCode() {
int result = (capacity != +0.0f ? Float.floatToIntBits(capacity) : 0);
result = 31 * result + (maxCapacity != +0.0f ? Float.floatToIntBits(
maxCapacity) : 0);
return result;
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.ParentQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.QueueManagementChange;
import java.util.List;
/**
* Event to update scheduler of any queue management changes
*/
public class QueueManagementChangeEvent extends SchedulerEvent {
private ParentQueue parentQueue;
private List<QueueManagementChange> queueManagementChanges;
public QueueManagementChangeEvent(ParentQueue parentQueue,
List<QueueManagementChange> queueManagementChanges) {
super(SchedulerEventType.MANAGE_QUEUE);
this.parentQueue = parentQueue;
this.queueManagementChanges = queueManagementChanges;
}
public ParentQueue getParentQueue() {
return parentQueue;
}
public List<QueueManagementChange> getQueueManagementChanges() {
return queueManagementChanges;
}
}

View File

@ -51,5 +51,8 @@ public enum SchedulerEventType {
MARK_CONTAINER_FOR_KILLABLE,
// Cancel a killable container
MARK_CONTAINER_FOR_NONKILLABLE
MARK_CONTAINER_FOR_NONKILLABLE,
//Queue Management Change
MANAGE_QUEUE
}

View File

@ -0,0 +1,579 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.commons.lang.math.RandomUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.placement
.ApplicationPlacementContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement
.UserGroupMappingPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common
.QueueEntitlement;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
.SchedulerEvent;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
.NO_LABEL;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CSQueueUtils.EPSILON;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CapacitySchedulerConfiguration.DOT;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CapacitySchedulerConfiguration.FAIR_APP_ORDERING_POLICY;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.fail;
public class TestCapacitySchedulerAutoCreatedQueueBase {
private static final Log LOG = LogFactory.getLog(
TestCapacitySchedulerAutoCreatedQueueBase.class);
public final int GB = 1024;
public final static ContainerUpdates NULL_UPDATE_REQUESTS =
new ContainerUpdates();
public static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
public static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
public static final String C = CapacitySchedulerConfiguration.ROOT + ".c";
public static final String D = CapacitySchedulerConfiguration.ROOT + ".d";
public static final String A1 = A + ".a1";
public static final String A2 = A + ".a2";
public static final String B1 = B + ".b1";
public static final String B2 = B + ".b2";
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 B_CAPACITY = 40f;
public static final float C_CAPACITY = 20f;
public static final float D_CAPACITY = 20f;
public static final float A1_CAPACITY = 30;
public static final float A2_CAPACITY = 70;
public static final float B1_CAPACITY = 60f;
public static final float B2_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 String USER = "user_";
public static final String USER0 = USER + 0;
public static final String USER1 = USER + 1;
public static final String USER3 = USER + 3;
public static final String USER2 = USER + 2;
public static final String PARENT_QUEUE = "c";
public static final Set<String> accessibleNodeLabelsOnC = new HashSet<>();
public static final String NODEL_LABEL_GPU = "GPU";
public static final String NODEL_LABEL_SSD = "SSD";
protected MockRM mockRM = null;
protected CapacityScheduler cs;
private final TestCapacityScheduler tcs = new TestCapacityScheduler();
protected SpyDispatcher dispatcher;
private static EventHandler<Event> rmAppEventEventHandler;
public static class SpyDispatcher extends AsyncDispatcher {
public static BlockingQueue<Event> eventQueue = new LinkedBlockingQueue<>();
public static class SpyRMAppEventHandler implements EventHandler<Event> {
public void handle(Event event) {
eventQueue.add(event);
}
}
@Override
protected void dispatch(Event event) {
eventQueue.add(event);
}
@Override
public EventHandler<Event> getEventHandler() {
return rmAppEventEventHandler;
}
void spyOnNextEvent(Event expectedEvent, long timeout)
throws InterruptedException {
Event event = eventQueue.poll(timeout, TimeUnit.MILLISECONDS);
assertEquals(expectedEvent.getType(), event.getType());
assertEquals(expectedEvent.getClass(), event.getClass());
}
}
@Before
public void setUp() throws Exception {
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
setupQueueConfiguration(conf);
conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
ResourceScheduler.class);
setupQueueMappings(conf);
mockRM = new MockRM(conf);
cs = (CapacityScheduler) mockRM.getResourceScheduler();
dispatcher = new SpyDispatcher();
rmAppEventEventHandler = new SpyDispatcher.SpyRMAppEventHandler();
dispatcher.register(RMAppEventType.class, rmAppEventEventHandler);
cs.updatePlacementRules();
mockRM.start();
cs.start();
}
public static CapacitySchedulerConfiguration setupQueueMappings(
CapacitySchedulerConfiguration conf) {
List<String> queuePlacementRules = new ArrayList<>();
queuePlacementRules.add(YarnConfiguration.USER_GROUP_PLACEMENT_RULE);
conf.setQueuePlacementRules(queuePlacementRules);
//set queue mapping
List<UserGroupMappingPlacementRule.QueueMapping> queueMappings =
new ArrayList<>();
for (int i = 0; i <= 3; i++) {
//Set C as parent queue name for auto queue creation
UserGroupMappingPlacementRule.QueueMapping userQueueMapping =
new UserGroupMappingPlacementRule.QueueMapping(
UserGroupMappingPlacementRule.QueueMapping.MappingType.USER,
USER + i, getQueueMapping(PARENT_QUEUE, USER + i));
queueMappings.add(userQueueMapping);
}
conf.setQueueMappings(queueMappings);
//override with queue mappings
conf.setOverrideWithQueueMappings(true);
return conf;
}
/**
* @param conf, to be modified
* @return, CS configuration which has C as an auto creation enabled parent
* queue
* <p>
* root / \ \ \ a b c d / \ / | \ a1 a2 b1
* b2 b3
*/
public static CapacitySchedulerConfiguration setupQueueConfiguration(
CapacitySchedulerConfiguration conf) {
//setup new queues with one of them auto enabled
// Define top-level queues
// Set childQueue for root
conf.setQueues(CapacitySchedulerConfiguration.ROOT,
new String[] { "a", "b", "c", "d" });
conf.setCapacity(A, A_CAPACITY);
conf.setCapacity(B, B_CAPACITY);
conf.setCapacity(C, C_CAPACITY);
conf.setCapacity(D, D_CAPACITY);
// Define 2nd-level queues
conf.setQueues(A, new String[] { "a1", "a2" });
conf.setCapacity(A1, A1_CAPACITY);
conf.setUserLimitFactor(A1, 100.0f);
conf.setCapacity(A2, A2_CAPACITY);
conf.setUserLimitFactor(A2, 100.0f);
conf.setQueues(B, new String[] { "b1", "b2", "b3" });
conf.setCapacity(B1, B1_CAPACITY);
conf.setUserLimitFactor(B1, 100.0f);
conf.setCapacity(B2, B2_CAPACITY);
conf.setUserLimitFactor(B2, 100.0f);
conf.setCapacity(B3, B3_CAPACITY);
conf.setUserLimitFactor(B3, 100.0f);
conf.setUserLimitFactor(C, 1.0f);
conf.setAutoCreateChildQueueEnabled(C, true);
//Setup leaf queue template configs
conf.setAutoCreatedLeafQueueConfigCapacity(C, 50.0f);
conf.setAutoCreatedLeafQueueConfigMaxCapacity(C, 100.0f);
conf.setAutoCreatedLeafQueueConfigUserLimit(C, 100);
conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f);
LOG.info("Setup " + C + " as an auto leaf creation enabled parent queue");
conf.setUserLimitFactor(D, 1.0f);
conf.setAutoCreateChildQueueEnabled(D, true);
conf.setUserLimit(D, 100);
conf.setUserLimitFactor(D, 3.0f);
//Setup leaf queue template configs
conf.setAutoCreatedLeafQueueConfigCapacity(D, 10.0f);
conf.setAutoCreatedLeafQueueConfigMaxCapacity(D, 100.0f);
conf.setAutoCreatedLeafQueueConfigUserLimit(D, 3);
conf.setAutoCreatedLeafQueueConfigUserLimitFactor(D, 100);
conf.set(CapacitySchedulerConfiguration.PREFIX + C + DOT
+ CapacitySchedulerConfiguration
.AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX
+ DOT + CapacitySchedulerConfiguration.ORDERING_POLICY,
FAIR_APP_ORDERING_POLICY);
accessibleNodeLabelsOnC.add(NODEL_LABEL_GPU);
accessibleNodeLabelsOnC.add(NODEL_LABEL_SSD);
accessibleNodeLabelsOnC.add(NO_LABEL);
conf.setAccessibleNodeLabels(C, accessibleNodeLabelsOnC);
conf.setCapacityByLabel(C, NODEL_LABEL_GPU, 50);
conf.setCapacityByLabel(C, NODEL_LABEL_SSD, 50);
LOG.info("Setup " + D + " as an auto leaf creation enabled parent queue");
return conf;
}
@After
public void tearDown() throws Exception {
if (mockRM != null) {
mockRM.stop();
}
}
protected void validateCapacities(AutoCreatedLeafQueue autoCreatedLeafQueue,
float capacity, float absCapacity, float maxCapacity,
float absMaxCapacity) {
assertEquals(capacity, autoCreatedLeafQueue.getCapacity(), EPSILON);
assertEquals(absCapacity, autoCreatedLeafQueue.getAbsoluteCapacity(),
EPSILON);
assertEquals(maxCapacity, autoCreatedLeafQueue.getMaximumCapacity(),
EPSILON);
assertEquals(absMaxCapacity,
autoCreatedLeafQueue.getAbsoluteMaximumCapacity(), EPSILON);
}
protected void cleanupQueue(String queueName) throws YarnException {
AutoCreatedLeafQueue queue = (AutoCreatedLeafQueue) cs.getQueue(queueName);
if (queue != null) {
setEntitlement(queue, new QueueEntitlement(0.0f, 0.0f));
((ManagedParentQueue) queue.getParent()).removeChildQueue(
queue.getQueueName());
cs.getCapacitySchedulerQueueManager().removeQueue(queue.getQueueName());
}
}
protected ApplicationId submitApp(MockRM rm, CSQueue parentQueue,
String leafQueueName, String user, int expectedNumAppsInParentQueue,
int expectedNumAppsInLeafQueue) throws Exception {
CapacityScheduler capacityScheduler =
(CapacityScheduler) rm.getResourceScheduler();
// submit an app
RMApp rmApp = rm.submitApp(GB, "test-auto-queue-activation", user, null,
leafQueueName);
// check preconditions
List<ApplicationAttemptId> appsInParentQueue =
capacityScheduler.getAppsInQueue(parentQueue.getQueueName());
assertEquals(expectedNumAppsInParentQueue, appsInParentQueue.size());
List<ApplicationAttemptId> appsInLeafQueue =
capacityScheduler.getAppsInQueue(leafQueueName);
assertEquals(expectedNumAppsInLeafQueue, appsInLeafQueue.size());
return rmApp.getApplicationId();
}
protected List<UserGroupMappingPlacementRule.QueueMapping> setupQueueMapping(
CapacityScheduler newCS, String user, String parentQueue, String queue) {
List<UserGroupMappingPlacementRule.QueueMapping> queueMappings =
new ArrayList<>();
queueMappings.add(new UserGroupMappingPlacementRule.QueueMapping(
UserGroupMappingPlacementRule.QueueMapping.MappingType.USER, user,
getQueueMapping(parentQueue, queue)));
newCS.getConfiguration().setQueueMappings(queueMappings);
return queueMappings;
}
protected MockRM setupSchedulerInstance() {
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
setupQueueConfiguration(conf);
conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
ResourceScheduler.class);
List<String> queuePlacementRules = new ArrayList<String>();
queuePlacementRules.add(YarnConfiguration.USER_GROUP_PLACEMENT_RULE);
conf.setQueuePlacementRules(queuePlacementRules);
setupQueueMappings(conf);
MockRM newMockRM = new MockRM(conf);
newMockRM.start();
((CapacityScheduler) newMockRM.getResourceScheduler()).start();
return newMockRM;
}
protected void checkQueueCapacities(CapacityScheduler newCS, float capacityC,
float capacityD) {
CSQueue rootQueue = newCS.getRootQueue();
CSQueue queueC = tcs.findQueue(rootQueue, C);
CSQueue queueD = tcs.findQueue(rootQueue, D);
CSQueue queueC1 = tcs.findQueue(queueC, C1);
CSQueue queueC2 = tcs.findQueue(queueC, C2);
CSQueue queueC3 = tcs.findQueue(queueC, C3);
float capC = capacityC / 100.0f;
float capD = capacityD / 100.0f;
tcs.checkQueueCapacity(queueC, capC, capC, 1.0f, 1.0f);
tcs.checkQueueCapacity(queueD, capD, capD, 1.0f, 1.0f);
tcs.checkQueueCapacity(queueC1, C1_CAPACITY / 100.0f,
(C1_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
tcs.checkQueueCapacity(queueC2, C2_CAPACITY / 100.0f,
(C2_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
if (queueC3 != null) {
ManagedParentQueue parentQueue = (ManagedParentQueue) queueC;
QueueCapacities cap =
parentQueue.getLeafQueueTemplate().getQueueCapacities();
tcs.checkQueueCapacity(queueC3, cap.getCapacity(),
(cap.getCapacity()) * capC, 1.0f, 1.0f);
}
}
static String getQueueMapping(String parentQueue, String leafQueue) {
return parentQueue + DOT + leafQueue;
}
protected ApplicationAttemptId submitApp(CapacityScheduler newCS, String user,
String queue, String parentQueue) {
ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
SchedulerEvent addAppEvent = new AppAddedSchedulerEvent(appId, queue, user,
new ApplicationPlacementContext(queue, parentQueue));
ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
appId, 1);
SchedulerEvent addAttemptEvent = new AppAttemptAddedSchedulerEvent(
appAttemptId, false);
newCS.handle(addAppEvent);
newCS.handle(addAttemptEvent);
return appAttemptId;
}
protected RMApp submitApp(String user, String queue, String nodeLabel)
throws Exception {
RMApp app = mockRM.submitApp(GB,
"test-auto-queue-creation" + RandomUtils.nextInt(100), user, null,
queue, nodeLabel);
Assert.assertEquals(app.getAmNodeLabelExpression(), nodeLabel);
// check preconditions
List<ApplicationAttemptId> appsInC = cs.getAppsInQueue(PARENT_QUEUE);
assertEquals(1, appsInC.size());
assertNotNull(cs.getQueue(queue));
return app;
}
void setEntitlement(AutoCreatedLeafQueue queue,
QueueEntitlement entitlement) {
queue.setCapacity(entitlement.getCapacity());
queue.setAbsoluteCapacity(
queue.getParent().getAbsoluteCapacity() * entitlement.getCapacity());
// note: we currently set maxCapacity to capacity
// this might be revised later
queue.setMaxCapacity(entitlement.getMaxCapacity());
}
protected void validateUserAndAppLimits(
AutoCreatedLeafQueue autoCreatedLeafQueue, int maxApps,
int maxAppsPerUser) {
assertEquals(maxApps, autoCreatedLeafQueue.getMaxApplications());
assertEquals(maxAppsPerUser,
autoCreatedLeafQueue.getMaxApplicationsPerUser());
}
protected void validateInitialQueueEntitlement(CSQueue parentQueue,
String leafQueueName, float expectedTotalChildQueueAbsCapacity)
throws SchedulerDynamicEditException {
validateInitialQueueEntitlement(cs, parentQueue, leafQueueName,
expectedTotalChildQueueAbsCapacity);
}
protected void validateInitialQueueEntitlement(
CapacityScheduler capacityScheduler, CSQueue parentQueue,
String leafQueueName, float expectedTotalChildQueueAbsCapacity)
throws SchedulerDynamicEditException {
ManagedParentQueue autoCreateEnabledParentQueue =
(ManagedParentQueue) parentQueue;
GuaranteedOrZeroCapacityOverTimePolicy policy =
(GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue
.getAutoCreatedQueueManagementPolicy();
assertEquals(expectedTotalChildQueueAbsCapacity,
policy.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
AutoCreatedLeafQueue leafQueue =
(AutoCreatedLeafQueue) capacityScheduler.getQueue(leafQueueName);
for (String label : accessibleNodeLabelsOnC) {
validateCapacitiesByLabel(autoCreateEnabledParentQueue, leafQueue, label);
}
assertEquals(true, policy.isActive(leafQueue));
}
protected void validateCapacitiesByLabel(
ManagedParentQueue autoCreateEnabledParentQueue,
AutoCreatedLeafQueue leafQueue, String label) {
assertEquals(
autoCreateEnabledParentQueue.getLeafQueueTemplate().getQueueCapacities()
.getCapacity(), leafQueue.getQueueCapacities().getCapacity(label),
EPSILON);
assertEquals(
autoCreateEnabledParentQueue.getLeafQueueTemplate().getQueueCapacities()
.getMaximumCapacity(),
leafQueue.getQueueCapacities().getMaximumCapacity(label), EPSILON);
}
protected void validateActivatedQueueEntitlement(CSQueue parentQueue,
String leafQueueName, float expectedTotalChildQueueAbsCapacity,
List<QueueManagementChange> queueManagementChanges)
throws SchedulerDynamicEditException {
ManagedParentQueue autoCreateEnabledParentQueue =
(ManagedParentQueue) parentQueue;
GuaranteedOrZeroCapacityOverTimePolicy policy =
(GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue
.getAutoCreatedQueueManagementPolicy();
QueueCapacities cap = autoCreateEnabledParentQueue.getLeafQueueTemplate()
.getQueueCapacities();
QueueEntitlement expectedEntitlement = new QueueEntitlement(
cap.getCapacity(), cap.getMaximumCapacity());
//validate capacity
validateQueueEntitlements(leafQueueName, expectedEntitlement,
queueManagementChanges);
//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,
String leafQueueName, float expectedTotalChildQueueAbsCapacity,
List<QueueManagementChange> queueManagementChanges)
throws SchedulerDynamicEditException {
QueueEntitlement expectedEntitlement = new QueueEntitlement(0.0f, 1.0f);
ManagedParentQueue autoCreateEnabledParentQueue =
(ManagedParentQueue) parentQueue;
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
leafQueueName);
GuaranteedOrZeroCapacityOverTimePolicy policy =
(GuaranteedOrZeroCapacityOverTimePolicy) autoCreateEnabledParentQueue
.getAutoCreatedQueueManagementPolicy();
//validate parent queue state
assertEquals(expectedTotalChildQueueAbsCapacity,
policy.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
//validate leaf queue state
assertEquals(false, policy.isActive(leafQueue));
//validate capacity
validateQueueEntitlements(leafQueueName, expectedEntitlement,
queueManagementChanges);
}
private void validateQueueEntitlements(String leafQueueName,
QueueEntitlement expectedEntitlement,
List<QueueManagementChange> queueEntitlementChanges) {
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
leafQueueName);
validateQueueEntitlementChangesForLeafQueue(leafQueue, expectedEntitlement,
queueEntitlementChanges);
}
private void validateQueueEntitlementChangesForLeafQueue(CSQueue leafQueue,
QueueEntitlement expectedQueueEntitlement,
final List<QueueManagementChange> queueEntitlementChanges) {
boolean found = false;
for (QueueManagementChange entitlementChange : queueEntitlementChanges) {
if (leafQueue.getQueueName().equals(
entitlementChange.getQueue().getQueueName())) {
AutoCreatedLeafQueueConfig updatedQueueTemplate =
entitlementChange.getUpdatedQueueTemplate();
for (String label : accessibleNodeLabelsOnC) {
QueueEntitlement newEntitlement = new QueueEntitlement(
updatedQueueTemplate.getQueueCapacities().getCapacity(label),
updatedQueueTemplate.getQueueCapacities()
.getMaximumCapacity(label));
assertEquals(expectedQueueEntitlement, newEntitlement);
}
found = true;
break;
}
}
if (!found) {
fail("Could not find the specified leaf queue in entitlement changes : "
+ leafQueue.getQueueName());
}
}
}

View File

@ -26,51 +26,54 @@ import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement
.ApplicationPlacementContext;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt
.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common
.QueueEntitlement;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy
.FairOrderingPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.security
.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security
.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security
.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.yarn.server.resourcemanager.placement.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.CapacitySchedulerConfiguration.DOT;
import static org.apache.hadoop.yarn.server.resourcemanager.placement
.UserGroupMappingPlacementRule.CURRENT_USER_MAPPING;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CSQueueUtils.EPSILON;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
@ -79,198 +82,14 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
/**
* Tests for creation and reinitilization of auto created leaf queues
* Tests for creation and reinitialization of auto created leaf queues
* under a ManagedParentQueue.
*/
public class TestCapacitySchedulerAutoQueueCreation {
public class TestCapacitySchedulerAutoQueueCreation
extends TestCapacitySchedulerAutoCreatedQueueBase {
private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
private final int GB = 1024;
private final static ContainerUpdates NULL_UPDATE_REQUESTS =
new ContainerUpdates();
private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
private static final String C = CapacitySchedulerConfiguration.ROOT + ".c";
private static final String D = CapacitySchedulerConfiguration.ROOT + ".d";
private static final String A1 = A + ".a1";
private static final String A2 = A + ".a2";
private static final String B1 = B + ".b1";
private static final String B2 = B + ".b2";
private static final String B3 = B + ".b3";
private static final String C1 = C + ".c1";
private static final String C2 = C + ".c2";
private static final String C3 = C + ".c3";
private static float A_CAPACITY = 20f;
private static float B_CAPACITY = 40f;
private static float C_CAPACITY = 20f;
private static float D_CAPACITY = 20f;
private static float A1_CAPACITY = 30;
private static float A2_CAPACITY = 70;
private static float B1_CAPACITY = 60f;
private static float B2_CAPACITY = 20f;
private static float B3_CAPACITY = 20f;
private static float C1_CAPACITY = 20f;
private static float C2_CAPACITY = 20f;
private static String USER = "user_";
private static String USER0 = USER + 0;
private static String USER2 = USER + 2;
private static String PARENT_QUEUE = "c";
private MockRM mockRM = null;
private CapacityScheduler cs;
private final TestCapacityScheduler tcs = new TestCapacityScheduler();
private static SpyDispatcher dispatcher;
private static EventHandler<Event> rmAppEventEventHandler;
private static class SpyDispatcher extends AsyncDispatcher {
private static BlockingQueue<Event> eventQueue =
new LinkedBlockingQueue<>();
private static class SpyRMAppEventHandler implements EventHandler<Event> {
public void handle(Event event) {
eventQueue.add(event);
}
}
@Override
protected void dispatch(Event event) {
eventQueue.add(event);
}
@Override
public EventHandler<Event> getEventHandler() {
return rmAppEventEventHandler;
}
void spyOnNextEvent(Event expectedEvent, long timeout)
throws InterruptedException {
Event event = eventQueue.poll(timeout, TimeUnit.MILLISECONDS);
assertEquals(expectedEvent.getType(), event.getType());
assertEquals(expectedEvent.getClass(), event.getClass());
}
}
@Before
public void setUp() throws Exception {
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
setupQueueConfiguration(conf);
conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
ResourceScheduler.class);
List<String> queuePlacementRules = new ArrayList<>();
queuePlacementRules.add(YarnConfiguration.USER_GROUP_PLACEMENT_RULE);
conf.setQueuePlacementRules(queuePlacementRules);
setupQueueMappings(conf);
mockRM = new MockRM(conf);
cs = (CapacityScheduler) mockRM.getResourceScheduler();
dispatcher = new SpyDispatcher();
rmAppEventEventHandler = new SpyDispatcher.SpyRMAppEventHandler();
dispatcher.register(RMAppEventType.class, rmAppEventEventHandler);
cs.updatePlacementRules();
mockRM.start();
cs.start();
}
private CapacitySchedulerConfiguration setupQueueMappings(
CapacitySchedulerConfiguration conf) {
//set queue mapping
List<UserGroupMappingPlacementRule.QueueMapping> queueMappings =
new ArrayList<>();
for (int i = 0; i <= 3; i++) {
//Set C as parent queue name for auto queue creation
UserGroupMappingPlacementRule.QueueMapping userQueueMapping =
new UserGroupMappingPlacementRule.QueueMapping(
UserGroupMappingPlacementRule.QueueMapping.MappingType.USER,
USER + i, getQueueMapping(PARENT_QUEUE, USER + i));
queueMappings.add(userQueueMapping);
}
conf.setQueueMappings(queueMappings);
//override with queue mappings
conf.setOverrideWithQueueMappings(true);
return conf;
}
/**
* @param conf, to be modified
* @return, CS configuration which has C
* as an auto creation enabled parent queue
* <p>
* root
* / \ \ \
* a b c d
* / \ / | \
* a1 a2 b1 b2 b3
*/
private CapacitySchedulerConfiguration setupQueueConfiguration(
CapacitySchedulerConfiguration conf) {
//setup new queues with one of them auto enabled
// Define top-level queues
// Set childQueue for root
conf.setQueues(CapacitySchedulerConfiguration.ROOT,
new String[] { "a", "b", "c", "d" });
conf.setCapacity(A, A_CAPACITY);
conf.setCapacity(B, B_CAPACITY);
conf.setCapacity(C, C_CAPACITY);
conf.setCapacity(D, D_CAPACITY);
// Define 2nd-level queues
conf.setQueues(A, new String[] { "a1", "a2" });
conf.setCapacity(A1, A1_CAPACITY);
conf.setUserLimitFactor(A1, 100.0f);
conf.setCapacity(A2, A2_CAPACITY);
conf.setUserLimitFactor(A2, 100.0f);
conf.setQueues(B, new String[] { "b1", "b2", "b3" });
conf.setCapacity(B1, B1_CAPACITY);
conf.setUserLimitFactor(B1, 100.0f);
conf.setCapacity(B2, B2_CAPACITY);
conf.setUserLimitFactor(B2, 100.0f);
conf.setCapacity(B3, B3_CAPACITY);
conf.setUserLimitFactor(B3, 100.0f);
conf.setUserLimitFactor(C, 1.0f);
conf.setAutoCreateChildQueueEnabled(C, true);
//Setup leaf queue template configs
conf.setAutoCreatedLeafQueueTemplateCapacity(C, 50.0f);
conf.setAutoCreatedLeafQueueTemplateMaxCapacity(C, 100.0f);
LOG.info("Setup " + C + " as an auto leaf creation enabled parent queue");
conf.setUserLimitFactor(D, 1.0f);
conf.setAutoCreateChildQueueEnabled(D, true);
//Setup leaf queue template configs
conf.setAutoCreatedLeafQueueTemplateCapacity(D, 10.0f);
conf.setAutoCreatedLeafQueueTemplateMaxCapacity(D, 100.0f);
LOG.info("Setup " + D + " as an auto leaf creation enabled parent queue");
return conf;
}
@After
public void tearDown() throws Exception {
if (mockRM != null) {
mockRM.stop();
}
}
private static final Log LOG = LogFactory.getLog(
TestCapacitySchedulerAutoQueueCreation.class);
@Test(timeout = 10000)
public void testAutoCreateLeafQueueCreation() throws Exception {
@ -289,7 +108,11 @@ public class TestCapacitySchedulerAutoQueueCreation {
ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue(
PARENT_QUEUE);
assertEquals(parentQueue, autoCreatedLeafQueue.getParent());
validateCapacities(autoCreatedLeafQueue);
validateInitialQueueEntitlement(parentQueue, USER0, 0.1f);
validateUserAndAppLimits(autoCreatedLeafQueue, 1000, 1000);
assertTrue(autoCreatedLeafQueue
.getOrderingPolicy() instanceof FairOrderingPolicy);
} finally {
cleanupQueue(USER0);
}
@ -297,7 +120,6 @@ public class TestCapacitySchedulerAutoQueueCreation {
@Test
public void testReinitializeStoppedAutoCreatedLeafQueue() throws Exception {
try {
String host = "127.0.0.1";
RMNode node = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1,
@ -306,20 +128,28 @@ public class TestCapacitySchedulerAutoQueueCreation {
// submit an app
RMApp app = mockRM.submitApp(GB, "test-auto-queue-creation-1", USER0,
RMApp app1 = mockRM.submitApp(GB, "test-auto-queue-creation-1", USER0,
null, USER0);
RMApp app2 = mockRM.submitApp(GB, "test-auto-queue-creation-2", USER1,
null, USER1);
// check preconditions
List<ApplicationAttemptId> appsInC = cs.getAppsInQueue(PARENT_QUEUE);
assertEquals(1, appsInC.size());
assertEquals(2, appsInC.size());
assertNotNull(cs.getQueue(USER0));
assertNotNull(cs.getQueue(USER1));
AutoCreatedLeafQueue autoCreatedLeafQueue =
(AutoCreatedLeafQueue) cs.getQueue(USER0);
AutoCreatedLeafQueue user0Queue = (AutoCreatedLeafQueue) cs.getQueue(
USER0);
AutoCreatedLeafQueue user1Queue = (AutoCreatedLeafQueue) cs.getQueue(
USER0);
ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue(
PARENT_QUEUE);
assertEquals(parentQueue, autoCreatedLeafQueue.getParent());
validateCapacities(autoCreatedLeafQueue);
assertEquals(parentQueue, user0Queue.getParent());
assertEquals(parentQueue, user1Queue.getParent());
validateInitialQueueEntitlement(parentQueue, USER0, 0.2f);
validateInitialQueueEntitlement(parentQueue, USER1, 0.2f);
ApplicationAttemptId appAttemptId = appsInC.get(0);
@ -337,7 +167,7 @@ public class TestCapacitySchedulerAutoQueueCreation {
CapacityScheduler.schedule(cs);
//change state to draining
autoCreatedLeafQueue.stopQueue();
user0Queue.stopQueue();
cs.killAllAppsInQueue(USER0);
@ -346,83 +176,27 @@ public class TestCapacitySchedulerAutoQueueCreation {
mockRM.waitForState(appAttemptId.getApplicationId(), RMAppState.KILLED);
//change state to stopped
autoCreatedLeafQueue.stopQueue();
user0Queue.stopQueue();
assertEquals(QueueState.STOPPED,
autoCreatedLeafQueue.getQueueInfo().getQueueState());
user0Queue.getQueueInfo().getQueueState());
cs.reinitialize(cs.getConf(), mockRM.getRMContext());
AutoCreatedLeafQueue user0QueueReinited =
(AutoCreatedLeafQueue) cs.getQueue(USER0);
validateCapacities(user0QueueReinited, 0.0f, 0.0f, 1.0f, 1.0f);
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) cs.getQueue(
USER0);
validateCapacities(leafQueue);
USER1);
validateInitialQueueEntitlement(parentQueue, leafQueue.getQueueName(),
0.1f);
} finally {
cleanupQueue(USER0);
}
}
@Test
public void testRefreshQueuesWithAutoCreatedLeafQueues() throws Exception {
MockRM newMockRM = setupSchedulerInstance();
try {
CapacityScheduler newCS =
(CapacityScheduler) newMockRM.getResourceScheduler();
CapacitySchedulerConfiguration conf = newCS.getConfiguration();
// Test add one auto created queue dynamically and manually modify
// capacity
ManagedParentQueue parentQueue = (ManagedParentQueue) newCS.getQueue("c");
AutoCreatedLeafQueue c1 = new AutoCreatedLeafQueue(newCS, "c1",
parentQueue);
newCS.addQueue(c1);
c1.setEntitlement(new QueueEntitlement(C1_CAPACITY / 100, 1f));
// Test add another auto created queue and use setEntitlement to modify
// capacity
AutoCreatedLeafQueue c2 = new AutoCreatedLeafQueue(newCS, "c2",
(ManagedParentQueue) newCS.getQueue("c"));
newCS.addQueue(c2);
newCS.setEntitlement("c2", new QueueEntitlement(C2_CAPACITY / 100, 1f));
// Verify all allocations match
checkQueueCapacities(newCS, C_CAPACITY, D_CAPACITY);
// Reinitialize and verify all dynamic queued survived
conf.setCapacity(A, 20f);
conf.setCapacity(B, 20f);
conf.setCapacity(C, 40f);
conf.setCapacity(D, 20f);
newCS.reinitialize(conf, newMockRM.getRMContext());
checkQueueCapacities(newCS, 40f, 20f);
//chnage parent template configs and reinitialize
conf.setAutoCreatedLeafQueueTemplateCapacity(C, 30.0f);
conf.setAutoCreatedLeafQueueTemplateMaxCapacity(C, 100.0f);
newCS.reinitialize(conf, newMockRM.getRMContext());
ManagedParentQueue c = (ManagedParentQueue) newCS.getQueue("c");
AutoCreatedLeafQueue c3 = new AutoCreatedLeafQueue(newCS, "c3", c);
newCS.addQueue(c3);
AbstractManagedParentQueue.AutoCreatedLeafQueueTemplate
leafQueueTemplate = parentQueue.getLeafQueueTemplate();
QueueCapacities cap = leafQueueTemplate.getQueueCapacities();
c3.setEntitlement(
new QueueEntitlement(cap.getCapacity(), cap.getMaximumCapacity()));
newCS.reinitialize(conf, newMockRM.getRMContext());
checkQueueCapacities(newCS, 40f, 20f);
} finally {
if (newMockRM != null) {
((CapacityScheduler) newMockRM.getResourceScheduler()).stop();
newMockRM.stop();
}
}
}
@Test
public void testConvertAutoCreateDisabledOnManagedParentQueueFails()
throws Exception {
@ -460,7 +234,7 @@ public class TestCapacitySchedulerAutoQueueCreation {
CapacitySchedulerConfiguration newConf =
new CapacitySchedulerConfiguration();
setupQueueConfiguration(newConf);
newConf.setAutoCreatedLeafQueueTemplateCapacity(A1, A1_CAPACITY / 10);
newConf.setAutoCreatedLeafQueueConfigCapacity(A1, A1_CAPACITY / 10);
newConf.setAutoCreateChildQueueEnabled(A1, true);
newCS.setConf(new YarnConfiguration());
@ -490,7 +264,7 @@ public class TestCapacitySchedulerAutoQueueCreation {
CapacitySchedulerConfiguration newConf =
new CapacitySchedulerConfiguration();
setupQueueConfiguration(newConf);
newConf.setAutoCreatedLeafQueueTemplateCapacity(A, A_CAPACITY / 10);
newConf.setAutoCreatedLeafQueueConfigCapacity(A, A_CAPACITY / 10);
newConf.setAutoCreateChildQueueEnabled(A, true);
newCS.setConf(new YarnConfiguration());
@ -531,39 +305,6 @@ public class TestCapacitySchedulerAutoQueueCreation {
assertEquals(RMAppState.FAILED, app.getState());
}
private void validateCapacities(AutoCreatedLeafQueue autoCreatedLeafQueue) {
assertEquals(autoCreatedLeafQueue.getCapacity(), 0.0f, EPSILON);
assertEquals(autoCreatedLeafQueue.getAbsoluteCapacity(), 0.0f, EPSILON);
assertEquals(autoCreatedLeafQueue.getMaximumCapacity(), 0.0f, EPSILON);
assertEquals(autoCreatedLeafQueue.getAbsoluteMaximumCapacity(), 0.0f,
EPSILON);
int maxAppsForAutoCreatedQueues = (int) (
CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS
* autoCreatedLeafQueue.getParent().getAbsoluteCapacity());
assertEquals(autoCreatedLeafQueue.getMaxApplicationsPerUser(),
maxAppsForAutoCreatedQueues);
assertEquals(autoCreatedLeafQueue.getMaxApplicationsPerUser(),
(int) (maxAppsForAutoCreatedQueues * (cs.getConfiguration()
.getUserLimitFactor(
autoCreatedLeafQueue.getParent().getQueuePath()))));
}
private void cleanupQueue(String queueName) throws YarnException {
AutoCreatedLeafQueue queue = (AutoCreatedLeafQueue) cs.getQueue(queueName);
if (queue != null) {
queue.setEntitlement(new QueueEntitlement(0.0f, 0.0f));
((ManagedParentQueue) queue.getParent()).removeChildQueue(
queue.getQueueName());
cs.getCapacitySchedulerQueueManager().removeQueue(queue.getQueueName());
} else{
throw new YarnException("Queue does not exist " + queueName);
}
}
String getQueueMapping(String parentQueue, String leafQueue) {
return parentQueue + DOT + leafQueue;
}
@Test(timeout = 10000)
public void testQueueMappingValidationFailsWithInvalidParentQueueInMapping()
throws Exception {
@ -586,8 +327,7 @@ public class TestCapacitySchedulerAutoQueueCreation {
//expected exception
assertTrue(e.getMessage().contains(
"invalid parent queue which does not have auto creation of leaf "
+ "queues enabled ["
+ "a" + "]"));
+ "queues enabled [" + "a" + "]"));
}
//"a" is not auto create enabled and app_user does not exist as a leaf
@ -650,9 +390,6 @@ public class TestCapacitySchedulerAutoQueueCreation {
(CapacityScheduler) newMockRM.getResourceScheduler();
try {
newMockRM.start();
newCS.start();
submitApp(newCS, USER0, USER0, PARENT_QUEUE);
assertNotNull(newCS.getQueue(USER0));
@ -700,12 +437,16 @@ public class TestCapacitySchedulerAutoQueueCreation {
AutoCreatedLeafQueue c1 = new AutoCreatedLeafQueue(newCS, "c1",
parentQueue);
newCS.addQueue(c1);
c1.setEntitlement(new QueueEntitlement(0.5f, 1f));
c1.setCapacity(0.5f);
c1.setAbsoluteCapacity(c1.getParent().getAbsoluteCapacity() * 1f);
c1.setMaxCapacity(1f);
setEntitlement(c1, new QueueEntitlement(0.5f, 1f));
AutoCreatedLeafQueue c2 = new AutoCreatedLeafQueue(newCS, "c2",
parentQueue);
newCS.addQueue(c2);
c2.setEntitlement(new QueueEntitlement(0.5f, 1f));
setEntitlement(c2, new QueueEntitlement(0.5f, 1f));
try {
AutoCreatedLeafQueue c3 = new AutoCreatedLeafQueue(newCS, "c3",
@ -723,72 +464,160 @@ public class TestCapacitySchedulerAutoQueueCreation {
}
}
private List<UserGroupMappingPlacementRule.QueueMapping> setupQueueMapping(
CapacityScheduler newCS, String user, String parentQueue, String queue) {
List<UserGroupMappingPlacementRule.QueueMapping> queueMappings =
new ArrayList<>();
queueMappings.add(new UserGroupMappingPlacementRule.QueueMapping(
UserGroupMappingPlacementRule.QueueMapping.MappingType.USER, user,
getQueueMapping(parentQueue, queue)));
newCS.getConfiguration().setQueueMappings(queueMappings);
return queueMappings;
}
@Test
public void testAutoCreatedQueueActivationDeactivation() throws Exception {
private MockRM setupSchedulerInstance() {
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
setupQueueConfiguration(conf);
conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
ResourceScheduler.class);
try {
String host = "127.0.0.1";
RMNode node = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1,
host);
cs.handle(new NodeAddedSchedulerEvent(node));
List<String> queuePlacementRules = new ArrayList<String>();
queuePlacementRules.add(YarnConfiguration.USER_GROUP_PLACEMENT_RULE);
conf.setQueuePlacementRules(queuePlacementRules);
CSQueue parentQueue = cs.getQueue(PARENT_QUEUE);
setupQueueMappings(conf);
//submit app1 as USER1
submitApp(mockRM, parentQueue, USER1, USER1, 1, 1);
validateInitialQueueEntitlement(parentQueue, USER1, 0.1f);
MockRM newMockRM = new MockRM(conf);
return newMockRM;
}
//submit another app2 as USER2
ApplicationId user2AppId = submitApp(mockRM, parentQueue, USER2, USER2, 2,
1);
validateInitialQueueEntitlement(parentQueue, USER2, 0.2f);
void checkQueueCapacities(CapacityScheduler newCS, float capacityC,
float capacityD) {
CSQueue rootQueue = newCS.getRootQueue();
CSQueue queueC = tcs.findQueue(rootQueue, C);
CSQueue queueD = tcs.findQueue(rootQueue, D);
CSQueue queueC1 = tcs.findQueue(queueC, C1);
CSQueue queueC2 = tcs.findQueue(queueC, C2);
CSQueue queueC3 = tcs.findQueue(queueC, C3);
//submit another app3 as USER1
submitApp(mockRM, parentQueue, USER1, USER1, 3, 2);
float capC = capacityC / 100.0f;
float capD = capacityD / 100.0f;
//validate total activated abs capacity remains the same
GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy =
(GuaranteedOrZeroCapacityOverTimePolicy) ((ManagedParentQueue)
parentQueue)
.getAutoCreatedQueueManagementPolicy();
assertEquals(autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON);
tcs.checkQueueCapacity(queueC, capC, capC, 1.0f, 1.0f);
tcs.checkQueueCapacity(queueD, capD, capD, 1.0f, 1.0f);
tcs.checkQueueCapacity(queueC1, C1_CAPACITY / 100.0f,
(C1_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
tcs.checkQueueCapacity(queueC2, C2_CAPACITY / 100.0f,
(C2_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
//submit user_3 app. This cant be scheduled since there is no capacity
submitApp(mockRM, parentQueue, USER3, USER3, 4, 1);
final CSQueue user3LeafQueue = cs.getQueue(USER3);
validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.0f, 0.0f,
1.0f, 1.0f);
if (queueC3 != null) {
ManagedParentQueue parentQueue = (ManagedParentQueue) queueC;
QueueCapacities cap =
parentQueue.getLeafQueueTemplate().getQueueCapacities();
tcs.checkQueueCapacity(queueC3, cap.getCapacity(),
(cap.getCapacity()) * capC, 1.0f, 1.0f);
assertEquals(autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON);
//deactivate USER2 queue
cs.killAllAppsInQueue(USER2);
mockRM.waitForState(user2AppId, RMAppState.KILLED);
//Verify if USER_2 can be deactivated since it has no pending appsA
List<QueueManagementChange> queueManagementChanges =
autoCreatedQueueManagementPolicy.computeQueueManagementChanges();
ManagedParentQueue managedParentQueue = (ManagedParentQueue) parentQueue;
managedParentQueue.validateAndApplyQueueManagementChanges(
queueManagementChanges);
validateDeactivatedQueueEntitlement(parentQueue, USER2, 0.2f,
queueManagementChanges);
//USER_3 should now get activated
validateActivatedQueueEntitlement(parentQueue, USER3, 0.2f,
queueManagementChanges);
} finally {
cleanupQueue(USER1);
cleanupQueue(USER2);
cleanupQueue(USER3);
}
}
ApplicationAttemptId submitApp(CapacityScheduler newCS, String user,
String queue, String parentQueue) {
ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
SchedulerEvent addAppEvent = new AppAddedSchedulerEvent(appId, queue, user,
new ApplicationPlacementContext(queue, parentQueue));
ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
appId, 1);
SchedulerEvent addAttemptEvent = new AppAttemptAddedSchedulerEvent(
appAttemptId, false);
newCS.handle(addAppEvent);
newCS.handle(addAttemptEvent);
return appAttemptId;
@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);
} finally {
cleanupQueue(USER1);
}
}
@Test
public void testReinitializeQueuesWithAutoCreatedLeafQueues()
throws Exception {
MockRM newMockRM = setupSchedulerInstance();
try {
CapacityScheduler newCS =
(CapacityScheduler) newMockRM.getResourceScheduler();
CapacitySchedulerConfiguration conf = newCS.getConfiguration();
String host = "127.0.0.1";
RMNode node = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1,
host);
newCS.handle(new NodeAddedSchedulerEvent(node));
CSQueue parentQueue = newCS.getQueue(PARENT_QUEUE);
//submit app1 as USER1
submitApp(newMockRM, parentQueue, USER1, USER1, 1, 1);
validateInitialQueueEntitlement(newCS, parentQueue, USER1, 0.1f);
//submit another app2 as USER2
ApplicationId user2AppId = submitApp(newMockRM, parentQueue, USER2, USER2,
2, 1);
validateInitialQueueEntitlement(newCS, parentQueue, USER2, 0.2f);
//update parent queue capacity
conf.setCapacity(C, 30f);
conf.setCapacity(D, 10f);
conf.setMaximumCapacity(C, 50f);
newCS.reinitialize(conf, newMockRM.getRMContext());
// validate that leaf queues abs capacity is now changed
AutoCreatedLeafQueue user0Queue = (AutoCreatedLeafQueue) newCS.getQueue(
USER1);
validateCapacities(user0Queue, 0.5f, 0.15f, 1.0f, 0.5f);
validateUserAndAppLimits(user0Queue, 1500, 1500);
//update leaf queue template capacities
conf.setAutoCreatedLeafQueueConfigCapacity(C, 30f);
conf.setAutoCreatedLeafQueueConfigMaxCapacity(C, 40f);
newCS.reinitialize(conf, newMockRM.getRMContext());
validateCapacities(user0Queue, 0.3f, 0.09f, 0.4f, 0.2f);
validateUserAndAppLimits(user0Queue, 900, 900);
//submit app1 as USER3
submitApp(newMockRM, parentQueue, USER3, USER3, 3, 1);
validateInitialQueueEntitlement(newCS, parentQueue, USER3, 0.27f);
AutoCreatedLeafQueue user3Queue = (AutoCreatedLeafQueue) newCS.getQueue(
USER1);
validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f, 0.2f);
validateUserAndAppLimits(user3Queue, 900, 900);
//submit app1 as USER1 - is already activated. there should be no diff
// in capacities
submitApp(newMockRM, parentQueue, USER3, USER3, 4, 2);
validateInitialQueueEntitlement(newCS, parentQueue, USER3, 0.27f);
validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f, 0.2f);
validateUserAndAppLimits(user3Queue, 900, 900);
} finally {
cleanupQueue(USER1);
cleanupQueue(USER2);
if (newMockRM != null) {
((CapacityScheduler) newMockRM.getResourceScheduler()).stop();
newMockRM.stop();
}
}
}
}

View File

@ -77,21 +77,21 @@ public class TestCapacitySchedulerDynamicBehavior {
CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
//set default queue capacity to zero
((AutoCreatedLeafQueue) cs
((ReservationQueue) cs
.getQueue("a" + ReservationConstants.DEFAULT_QUEUE_SUFFIX))
.setEntitlement(
new QueueEntitlement(0f, 1f));
// Test add one reservation dynamically and manually modify capacity
AutoCreatedLeafQueue a1 =
new AutoCreatedLeafQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
ReservationQueue a1 =
new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
cs.addQueue(a1);
a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
// Test add another reservation queue and use setEntitlement to modify
// capacity
AutoCreatedLeafQueue a2 =
new AutoCreatedLeafQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
ReservationQueue a2 =
new ReservationQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
cs.addQueue(a2);
cs.setEntitlement("a2", new QueueEntitlement(A2_CAPACITY / 100, 1.0f));
@ -113,8 +113,8 @@ public class TestCapacitySchedulerDynamicBehavior {
try {
// Test invalid addition (adding non-zero size queue)
AutoCreatedLeafQueue a1 =
new AutoCreatedLeafQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
ReservationQueue a1 =
new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
cs.addQueue(a1);
fail();
@ -123,11 +123,11 @@ public class TestCapacitySchedulerDynamicBehavior {
}
// Test add one reservation dynamically and manually modify capacity
AutoCreatedLeafQueue a1 =
new AutoCreatedLeafQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
ReservationQueue a1 =
new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
cs.addQueue(a1);
//set default queue capacity to zero
((AutoCreatedLeafQueue) cs
((ReservationQueue) cs
.getQueue("a" + ReservationConstants.DEFAULT_QUEUE_SUFFIX))
.setEntitlement(
new QueueEntitlement(0f, 1f));
@ -135,8 +135,8 @@ public class TestCapacitySchedulerDynamicBehavior {
// Test add another reservation queue and use setEntitlement to modify
// capacity
AutoCreatedLeafQueue a2 =
new AutoCreatedLeafQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
ReservationQueue a2 =
new ReservationQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
cs.addQueue(a2);
@ -162,8 +162,8 @@ public class TestCapacitySchedulerDynamicBehavior {
CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
// Test add one reservation dynamically and manually modify capacity
AutoCreatedLeafQueue a1 =
new AutoCreatedLeafQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
ReservationQueue a1 =
new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
cs.addQueue(a1);
a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
@ -230,8 +230,8 @@ public class TestCapacitySchedulerDynamicBehavior {
// create the default reservation queue
String defQName = "a" + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
AutoCreatedLeafQueue defQ =
new AutoCreatedLeafQueue(scheduler, defQName,
ReservationQueue defQ =
new ReservationQueue(scheduler, defQName,
(PlanQueue) scheduler.getQueue("a"));
scheduler.addQueue(defQ);
defQ.setEntitlement(new QueueEntitlement(1f, 1f));

View File

@ -0,0 +1,40 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
public class TestGuaranteedOrZeroCapacityOverTimePolicy {
@Test
public void testGetMaxLeavesToBeActivated()
throws SchedulerDynamicEditException {
GuaranteedOrZeroCapacityOverTimePolicy policy =
new GuaranteedOrZeroCapacityOverTimePolicy();
assertEquals(1, policy.getMaxLeavesToBeActivated(0.17f, 0.03f, 1));
assertEquals(5, policy.getMaxLeavesToBeActivated(0.17f, 0.03f, 7));
assertEquals(0, policy.getMaxLeavesToBeActivated(0, 0.03f, 10));
}
}

View File

@ -18,6 +18,14 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
.NO_LABEL;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CSQueueUtils.EPSILON;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CapacitySchedulerConfiguration.DOT;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CapacitySchedulerConfiguration.ROOT;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@ -44,6 +52,7 @@ import java.util.concurrent.CyclicBarrier;
import com.google.common.collect.ImmutableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -189,7 +198,7 @@ public class TestLeafQueue {
root =
CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
CapacitySchedulerConfiguration.ROOT,
ROOT,
queues, queues,
TestUtils.spyHook);
root.updateClusterResource(Resources.createResource(100 * 16 * GB, 100 * 32),
@ -222,12 +231,12 @@ public class TestLeafQueue {
final String newRoot) {
// Define top-level queues
conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {newRoot});
conf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT, 100);
conf.setAcl(CapacitySchedulerConfiguration.ROOT,
conf.setQueues(ROOT, new String[] {newRoot});
conf.setMaximumCapacity(ROOT, 100);
conf.setAcl(ROOT,
QueueACL.SUBMIT_APPLICATIONS, " ");
final String Q_newRoot = CapacitySchedulerConfiguration.ROOT + "." + newRoot;
final String Q_newRoot = ROOT + "." + newRoot;
conf.setQueues(Q_newRoot, new String[] {A, B, C, D, E});
conf.setCapacity(Q_newRoot, 100);
conf.setMaximumCapacity(Q_newRoot, 100);
@ -410,7 +419,7 @@ public class TestLeafQueue {
CapacitySchedulerConfiguration testConf =
new CapacitySchedulerConfiguration();
String tproot = CapacitySchedulerConfiguration.ROOT + "." +
String tproot = ROOT + "." +
"testPolicyRoot" + System.currentTimeMillis();
OrderingPolicy<FiCaSchedulerApp> comPol =
@ -485,7 +494,7 @@ public class TestLeafQueue {
CapacitySchedulerConfiguration testConf =
new CapacitySchedulerConfiguration();
String tproot = CapacitySchedulerConfiguration.ROOT + "." +
String tproot = ROOT + "." +
"testPolicyRoot" + System.currentTimeMillis();
OrderingPolicy<FiCaSchedulerApp> schedOrder =
@ -722,12 +731,12 @@ public class TestLeafQueue {
Priority priority = TestUtils.createMockPriority(1);
app0.updateResourceRequests(Collections.singletonList(TestUtils
.createResourceRequest(ResourceRequest.ANY, 20 * GB, 29, 1, true,
priority, recordFactory, RMNodeLabelsManager.NO_LABEL)));
priority, recordFactory, NO_LABEL)));
assign = b.assignContainers(clusterResource, node0, new ResourceLimits(
clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
app0.updateResourceRequests(Collections.singletonList(TestUtils
.createResourceRequest(ResourceRequest.ANY, 10 * GB, 29, 2, true,
priority, recordFactory, RMNodeLabelsManager.NO_LABEL)));
priority, recordFactory, NO_LABEL)));
assign = b.assignContainers(clusterResource, node0, new ResourceLimits(
clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assertTrue("Still within limits, should assign",
@ -796,11 +805,11 @@ public class TestLeafQueue {
Priority priority = TestUtils.createMockPriority(1);
app0.updateResourceRequests(Collections.singletonList(TestUtils
.createResourceRequest(ResourceRequest.ANY, 1 * GB, 40, 10, true,
priority, recordFactory, RMNodeLabelsManager.NO_LABEL)));
priority, recordFactory, NO_LABEL)));
app2.updateResourceRequests(Collections.singletonList(TestUtils
.createResourceRequest(ResourceRequest.ANY, 2 * GB, 10, 10, true,
priority, recordFactory, RMNodeLabelsManager.NO_LABEL)));
priority, recordFactory, NO_LABEL)));
/**
* Start testing...
@ -2277,7 +2286,7 @@ public class TestLeafQueue {
CapacitySchedulerConfiguration.RACK_LOCALITY_ADDITIONAL_DELAY, 1);
Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext,
csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues,
csConf, null, ROOT, newQueues, queues,
TestUtils.spyHook);
root.reinitialize(newRoot, cs.getClusterResource());
@ -2712,9 +2721,12 @@ public class TestLeafQueue {
CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT
* 2);
Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext,
csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues,
TestUtils.spyHook);
CSQueue newRoot =
CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
ROOT,
newQueues, queues,
TestUtils.spyHook);
queues = newQueues;
root.reinitialize(newRoot, csContext.getClusterResource());
// after reinitialization
@ -2738,7 +2750,7 @@ public class TestLeafQueue {
Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
CSQueue newRoot =
CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
CapacitySchedulerConfiguration.ROOT,
ROOT,
newQueues, queues,
TestUtils.spyHook);
root.reinitialize(newRoot, cs.getClusterResource());
@ -3621,7 +3633,7 @@ public class TestLeafQueue {
assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
assertEquals(0 * GB,
e.getTotalPendingResourcesConsideringUserLimit(clusterResource,
RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
NO_LABEL, false).getMemorySize());
// Assign 2nd container of 1GB
applyCSAssignment(clusterResource,
@ -3635,7 +3647,7 @@ public class TestLeafQueue {
assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// Can't allocate 3rd container due to user-limit. Headroom still 0.
applyCSAssignment(clusterResource,
@ -3645,7 +3657,7 @@ public class TestLeafQueue {
assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// Increase user-limit-factor from 1GB to 10GB (1% * 10 * 100GB = 10GB).
// Pending for both app_0 and app_1 are still 3GB, so user-limit-factor
@ -3653,7 +3665,7 @@ public class TestLeafQueue {
// getTotalPendingResourcesConsideringUserLimit()
e.setUserLimitFactor(10.0f);
assertEquals(3*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
applyCSAssignment(clusterResource,
e.assignContainers(clusterResource, node_0,
@ -3663,7 +3675,7 @@ public class TestLeafQueue {
assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
assertEquals(2*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// Get the last 2 containers for app_1, no more pending requests.
applyCSAssignment(clusterResource,
@ -3677,7 +3689,7 @@ public class TestLeafQueue {
assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize());
assertEquals(2*GB, app_1.getCurrentConsumption().getMemorySize());
assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// Release each container from app_0
for (RMContainer rmContainer : app_0.getLiveContainers()) {
@ -3788,7 +3800,7 @@ public class TestLeafQueue {
// With queue capacity set at 1% of 100GB and user-limit-factor set to 1.0,
// queue 'e' should be able to consume 1GB per user.
assertEquals(2*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// None of the apps have assigned resources
// user_0's apps:
assertEquals(0*GB, app_0.getCurrentConsumption().getMemorySize());
@ -3805,7 +3817,7 @@ public class TestLeafQueue {
// The first container was assigned to user_0's app_0. Queues total headroom
// has 1GB left for user_1.
assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// user_0's apps:
assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
@ -3823,7 +3835,7 @@ public class TestLeafQueue {
// this container went to user_0's app_1. so, headroom for queue 'e'e is
// still 1GB for user_1
assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// user_0's apps:
assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
@ -3839,7 +3851,7 @@ public class TestLeafQueue {
// Container was allocated to user_1's app_2 since user_1, Now, no headroom
// is left.
assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// user_0's apps:
assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
@ -3855,7 +3867,7 @@ public class TestLeafQueue {
// Allocated to user_1's app_2 since scheduler allocates 1 container
// above user resource limit. Available headroom still 0.
assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// user_0's apps:
long app_0_consumption = app_0.getCurrentConsumption().getMemorySize();
assertEquals(1*GB, app_0_consumption);
@ -3875,7 +3887,7 @@ public class TestLeafQueue {
// Cannot allocate 5th container because both users are above their allowed
// user resource limit. Values should be the same as previously.
assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// user_0's apps:
assertEquals(app_0_consumption, app_0.getCurrentConsumption().getMemorySize());
assertEquals(app_1_consumption, app_1.getCurrentConsumption().getMemorySize());
@ -3894,7 +3906,7 @@ public class TestLeafQueue {
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps);
// Next container goes to user_0's app_1, since it still wanted 1GB.
assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
// user_0's apps:
assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
assertEquals(2*GB, app_1.getCurrentConsumption().getMemorySize());
@ -3909,7 +3921,7 @@ public class TestLeafQueue {
// Last container goes to user_1's app_3, since it still wanted 1GB.
// user_0's apps:
assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
clusterResource, RMNodeLabelsManager.NO_LABEL, false).getMemorySize());
clusterResource, NO_LABEL, false).getMemorySize());
assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize());
assertEquals(2*GB, app_1.getCurrentConsumption().getMemorySize());
// user_1's apps:
@ -4027,6 +4039,59 @@ public class TestLeafQueue {
app.getResourceUsageReport().getClusterUsagePercentage(), 0.01f);
}
@Test
public void testSetupQueueConfigsWithSpecifiedConfiguration()
throws IOException {
try {
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(
new Configuration(false), false);
final String leafQueueName =
"testSetupQueueConfigsWithSpecifiedConfiguration";
assertEquals(0, conf.size());
conf.setNodeLocalityDelay(60);
conf.setCapacity(ROOT + DOT + leafQueueName, 10);
conf.setMaximumCapacity(ROOT + DOT + leafQueueName, 100);
conf.setUserLimitFactor(ROOT + DOT +leafQueueName, 0.1f);
csConf.setNodeLocalityDelay(30);
csConf.setGlobalMaximumApplicationsPerQueue(20);
LeafQueue leafQueue = new LeafQueue(csContext, conf,
leafQueueName, cs.getRootQueue(),
null);
assertEquals(30, leafQueue.getNodeLocalityDelay());
assertEquals(20, leafQueue.getMaxApplications());
assertEquals(2, leafQueue.getMaxApplicationsPerUser());
//check queue configs
conf.setMaximumAMResourcePercentPerPartition(leafQueue.getQueueName(),
NO_LABEL, 10);
conf.setMaximumCapacity(leafQueue.getQueueName(), 10);
assertEquals(0.1, leafQueue.getMaxAMResourcePerQueuePercent(),
EPSILON);
assertEquals(1, leafQueue.getMaximumCapacity(),
EPSILON);
assertEquals(0.1, leafQueue.getCapacity(),
EPSILON);
assertEquals(0.1, leafQueue.getAbsoluteCapacity(),
EPSILON);
assertEquals(1.0, leafQueue.getAbsoluteMaximumCapacity(),
EPSILON);
} finally {
//revert config changes
csConf.setNodeLocalityDelay(
CapacitySchedulerConfiguration.DEFAULT_NODE_LOCALITY_DELAY);
csConf.setGlobalMaximumApplicationsPerQueue(
(int) CapacitySchedulerConfiguration.UNDEFINED);
}
}
private ApplicationAttemptId createAppAttemptId(int appId, int attemptId) {
ApplicationId appIdImpl = ApplicationId.newInstance(0, appId);
ApplicationAttemptId attId =

View File

@ -0,0 +1,121 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
import org.junit.Before;
import org.junit.Test;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
.capacity.CSQueueUtils.EPSILON;
import static org.junit.Assert.assertEquals;
public class TestQueueManagementDynamicEditPolicy extends
TestCapacitySchedulerAutoCreatedQueueBase {
private QueueManagementDynamicEditPolicy policy = new
QueueManagementDynamicEditPolicy();
@Before
public void setUp() throws Exception {
super.setUp();
policy.init(cs.getConfiguration(), cs.getRMContext(), cs);
}
@Test
public void testEditSchedule() throws Exception {
try {
policy.editSchedule();
assertEquals(2, policy.getManagedParentQueues().size());
CSQueue parentQueue = cs.getQueue(PARENT_QUEUE);
GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy =
(GuaranteedOrZeroCapacityOverTimePolicy) ((ManagedParentQueue)
parentQueue)
.getAutoCreatedQueueManagementPolicy();
assertEquals(0f, autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
//submit app1 as USER1
ApplicationId user1AppId = submitApp(mockRM, parentQueue, USER1, USER1, 1,
1);
validateInitialQueueEntitlement(parentQueue, USER1, 0.1f);
//submit another app2 as USER2
ApplicationId user2AppId = submitApp(mockRM, parentQueue, USER2, USER2, 2,
1);
validateInitialQueueEntitlement(parentQueue, USER2, 0.2f);
//validate total activated abs capacity
assertEquals(0.2f, autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), EPSILON);
//submit user_3 app. This cant be scheduled since there is no capacity
submitApp(mockRM, parentQueue, USER3, USER3, 3, 1);
final CSQueue user3LeafQueue = cs.getQueue(USER3);
validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.0f, 0.0f,
1.0f, 1.0f);
assertEquals(autoCreatedQueueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity(), 0.2f, EPSILON);
//deactivate USER2 queue
cs.killAllAppsInQueue(USER2);
mockRM.waitForState(user2AppId, RMAppState.KILLED);
//deactivate USER1 queue
cs.killAllAppsInQueue(USER1);
mockRM.waitForState(user1AppId, RMAppState.KILLED);
policy.editSchedule();
waitForPolicyState(0.1f, autoCreatedQueueManagementPolicy, 1000);
validateCapacities((AutoCreatedLeafQueue) user3LeafQueue, 0.5f, 0.1f,
1.0f, 1.0f);
validateCapacitiesByLabel((ManagedParentQueue) parentQueue, (AutoCreatedLeafQueue) user3LeafQueue,
NODEL_LABEL_GPU);
} finally {
cleanupQueue(USER1);
cleanupQueue(USER2);
cleanupQueue(USER3);
}
}
private void waitForPolicyState(float expectedVal,
GuaranteedOrZeroCapacityOverTimePolicy queueManagementPolicy, int
timesec) throws
InterruptedException {
long start = System.currentTimeMillis();
while (System.currentTimeMillis() - start < timesec * 1000) {
if (Float.compare(expectedVal, queueManagementPolicy
.getAbsoluteActivatedChildQueueCapacity()) != 0) {
Thread.sleep(100);
} else {
break;
}
}
}
}

View File

@ -38,9 +38,9 @@ import org.junit.Test;
/**
* Test class for dynamic auto created leaf queues.
* @see AutoCreatedLeafQueue
* @see ReservationQueue
*/
public class TestAutoCreatedLeafQueue {
public class TestReservationQueue {
private CapacitySchedulerConfiguration csConf;
private CapacitySchedulerContext csContext;
@ -48,12 +48,13 @@ public class TestAutoCreatedLeafQueue {
final static int GB = 1024;
private final ResourceCalculator resourceCalculator =
new DefaultResourceCalculator();
private AutoCreatedLeafQueue autoCreatedLeafQueue;
private ReservationQueue autoCreatedLeafQueue;
@Before
public void setup() throws IOException {
// setup a context / conf
csConf = new CapacitySchedulerConfiguration();
YarnConfiguration conf = new YarnConfiguration();
csContext = mock(CapacitySchedulerContext.class);
when(csContext.getConfiguration()).thenReturn(csConf);
@ -70,7 +71,7 @@ public class TestAutoCreatedLeafQueue {
// create a queue
PlanQueue pq = new PlanQueue(csContext, "root", null, null);
autoCreatedLeafQueue = new AutoCreatedLeafQueue(csContext, "a", pq);
autoCreatedLeafQueue = new ReservationQueue(csContext, "a", pq);
}
private void validateAutoCreatedLeafQueue(double capacity) {