YARN-5264. Store all queue-specific information in FSQueue. (Yufei Gu via kasha)

(cherry picked from commit f6ea9be547)
This commit is contained in:
Karthik Kambatla 2016-09-02 14:56:29 -07:00
parent 4ee1729cb0
commit a5748eec7b
13 changed files with 161 additions and 149 deletions

View File

@ -22,6 +22,8 @@
import java.util.Map;
import java.util.Set;
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.security.authorize.AccessControlList;
@ -37,6 +39,7 @@
import com.google.common.annotations.VisibleForTesting;
public class AllocationConfiguration extends ReservationSchedulerConfiguration {
private static final Log LOG = LogFactory.getLog(FSQueue.class.getName());
private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
private static final ResourceCalculator RESOURCE_CALCULATOR =
@ -242,26 +245,24 @@ public boolean isPreemptable(String queueName) {
return !nonPreemptableQueues.contains(queueName);
}
public ResourceWeights getQueueWeight(String queue) {
private ResourceWeights getQueueWeight(String queue) {
ResourceWeights weight = queueWeights.get(queue);
return (weight == null) ? ResourceWeights.NEUTRAL : weight;
}
public void setQueueWeight(String queue, ResourceWeights weight) {
queueWeights.put(queue, weight);
}
public int getUserMaxApps(String user) {
Integer maxApps = userMaxApps.get(user);
return (maxApps == null) ? userMaxAppsDefault : maxApps;
}
public int getQueueMaxApps(String queue) {
@VisibleForTesting
int getQueueMaxApps(String queue) {
Integer maxApps = queueMaxApps.get(queue);
return (maxApps == null) ? queueMaxAppsDefault : maxApps;
}
public float getQueueMaxAMShare(String queue) {
@VisibleForTesting
float getQueueMaxAMShare(String queue) {
Float maxAMShare = queueMaxAMShares.get(queue);
return (maxAMShare == null) ? queueMaxAMShareDefault : maxAMShare;
}
@ -273,21 +274,12 @@ public float getQueueMaxAMShare(String queue) {
* @return the min allocation on this queue or {@link Resources#none}
* if not set
*/
public Resource getMinResources(String queue) {
@VisibleForTesting
Resource getMinResources(String queue) {
Resource minQueueResource = minQueueResources.get(queue);
return (minQueueResource == null) ? Resources.none() : minQueueResource;
}
/**
* Set the maximum resource allocation for the given queue.
*
* @param queue the target queue
* @param maxResource the maximum resource allocation
*/
void setMaxResources(String queue, Resource maxResource) {
maxQueueResources.put(queue, maxResource);
}
/**
* Get the maximum resource allocation for the given queue. If the max in not
* set, return the larger of the min and the default max.
@ -295,7 +287,8 @@ void setMaxResources(String queue, Resource maxResource) {
* @param queue the target queue's name
* @return the max allocation on this queue
*/
public Resource getMaxResources(String queue) {
@VisibleForTesting
Resource getMaxResources(String queue) {
Resource maxQueueResource = maxQueueResources.get(queue);
if (maxQueueResource == null) {
Resource minQueueResource = minQueueResources.get(queue);
@ -317,21 +310,11 @@ public Resource getMaxResources(String queue) {
* @param queue the target queue's name
* @return the max allocation on this queue or null if not set
*/
public Resource getMaxChildResources(String queue) {
@VisibleForTesting
Resource getMaxChildResources(String queue) {
return maxChildQueueResources.get(queue);
}
/**
* Set the maximum resource allocation for the children of the given queue.
* Use of this method is primarily intended for testing purposes.
*
* @param queue the target queue
* @param maxResource the maximum resource allocation
*/
void setMaxChildResources(String queue, Resource maxResource) {
maxChildQueueResources.put(queue, maxResource);
}
public boolean hasAccess(String queueName, QueueACL acl,
UserGroupInformation user) {
int lastPeriodIndex = queueName.length();
@ -346,8 +329,9 @@ public boolean hasAccess(String queueName, QueueACL acl,
return false;
}
public SchedulingPolicy getSchedulingPolicy(String queueName) {
@VisibleForTesting
SchedulingPolicy getSchedulingPolicy(String queueName) {
SchedulingPolicy policy = schedulingPolicies.get(queueName);
return (policy == null) ? defaultSchedulingPolicy : policy;
}
@ -423,4 +407,35 @@ public void setReservationWindow(long window) {
public void setAverageCapacity(int avgCapacity) {
globalReservationQueueConfig.setAverageCapacity(avgCapacity);
}
/**
* Initialize a {@link FSQueue} with queue-specific properties and its
* metrics.
* @param queue the FSQueue needed to be initialized
* @param scheduler the scheduler which the queue belonged to
*/
public void initFSQueue(FSQueue queue, FairScheduler scheduler){
// Set queue-specific properties.
String name = queue.getName();
queue.setWeights(getQueueWeight(name));
queue.setMinShare(getMinResources(name));
queue.setMaxShare(getMaxResources(name));
queue.setMaxRunningApps(getQueueMaxApps(name));
queue.setMaxAMShare(getQueueMaxAMShare(name));
queue.setMaxChildQueueResource(getMaxChildResources(name));
try {
SchedulingPolicy policy = getSchedulingPolicy(name);
policy.initialize(scheduler.getClusterResource());
queue.setPolicy(policy);
} catch (AllocationConfigurationException ex) {
LOG.warn("Failed to set the scheduling policy "
+ getDefaultSchedulingPolicy(), ex);
}
// Set queue metrics.
queue.getMetrics().setMinShare(getMinResources(name));
queue.getMetrics().setMaxShare(getMaxResources(name));
queue.getMetrics().setMaxApps(getQueueMaxApps(name));
queue.getMetrics().setSchedulingPolicy(getSchedulingPolicy(name).getName());
}
}

View File

@ -264,29 +264,27 @@ public Resource getAmResourceUsage() {
public void updateDemand() {
// Compute demand by iterating through apps in the queue
// Limit demand to maxResources
Resource maxRes = scheduler.getAllocationConfiguration()
.getMaxResources(getName());
demand = Resources.createResource(0);
readLock.lock();
try {
for (FSAppAttempt sched : runnableApps) {
if (Resources.equals(demand, maxRes)) {
if (Resources.equals(demand, maxShare)) {
break;
}
updateDemandForApp(sched, maxRes);
updateDemandForApp(sched, maxShare);
}
for (FSAppAttempt sched : nonRunnableApps) {
if (Resources.equals(demand, maxRes)) {
if (Resources.equals(demand, maxShare)) {
break;
}
updateDemandForApp(sched, maxRes);
updateDemandForApp(sched, maxShare);
}
} finally {
readLock.unlock();
}
if (LOG.isDebugEnabled()) {
LOG.debug("The updated demand for " + getName() + " is " + demand
+ "; the max is " + maxRes);
+ "; the max is " + maxShare);
LOG.debug("The updated fairshare for " + getName() + " is "
+ getFairShare());
}
@ -486,8 +484,6 @@ public ActiveUsersManager getActiveUsersManager() {
* @return true if this queue can run
*/
public boolean canRunAppAM(Resource amResource) {
float maxAMShare =
scheduler.getAllocationConfiguration().getQueueMaxAMShare(getName());
if (Math.abs(maxAMShare - -1.0f) < 0.0001) {
return true;
}
@ -543,8 +539,7 @@ public void updateStarvationStats() {
* @param weight queue weight
*/
public void setWeights(float weight) {
scheduler.getAllocationConfiguration().setQueueWeight(getName(),
new ResourceWeights(weight));
this.weights = new ResourceWeights(weight);
}
/**

View File

@ -152,8 +152,6 @@ public Resource getResourceUsage() {
public void updateDemand() {
// Compute demand by iterating through apps in the queue
// Limit demand to maxResources
Resource maxRes = scheduler.getAllocationConfiguration()
.getMaxResources(getName());
writeLock.lock();
try {
demand = Resources.createResource(0);
@ -166,8 +164,8 @@ public void updateDemand() {
" now " + demand);
}
demand = Resources.add(demand, toAdd);
demand = Resources.componentwiseMin(demand, maxRes);
if (Resources.equals(demand, maxRes)) {
demand = Resources.componentwiseMin(demand, maxShare);
if (Resources.equals(demand, maxShare)) {
break;
}
}
@ -176,7 +174,7 @@ public void updateDemand() {
}
if (LOG.isDebugEnabled()) {
LOG.debug("The updated demand for " + getName() + " is " + demand +
"; the max is " + maxRes);
"; the max is " + maxShare);
}
}

View File

@ -59,6 +59,15 @@ public abstract class FSQueue implements Queue, Schedulable {
protected SchedulingPolicy policy = SchedulingPolicy.DEFAULT_POLICY;
protected ResourceWeights weights;
protected Resource minShare;
protected Resource maxShare;
protected int maxRunningApps;
protected Resource maxChildQueueResource;
// maxAMShare is a value between 0 and 1.
protected float maxAMShare;
private long fairSharePreemptionTimeout = Long.MAX_VALUE;
private long minSharePreemptionTimeout = Long.MAX_VALUE;
private float fairSharePreemptionThreshold = 0.5f;
@ -68,15 +77,20 @@ public FSQueue(String name, FairScheduler scheduler, FSParentQueue parent) {
this.name = name;
this.scheduler = scheduler;
this.metrics = FSQueueMetrics.forQueue(getName(), parent, true, scheduler.getConf());
metrics.setMinShare(getMinShare());
metrics.setMaxShare(getMaxShare());
AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
metrics.setMaxApps(allocConf.getQueueMaxApps(name));
metrics.setSchedulingPolicy(allocConf.getSchedulingPolicy(name).getName());
this.parent = parent;
}
/**
* Initialize a queue by setting its queue-specific properties and its
* metrics.
* This function is invoked when a new queue is created or reloading the
* allocation configuration.
*/
public void init() {
AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
allocConf.initFSQueue(this, scheduler);
}
public String getName() {
return name;
}
@ -103,19 +117,51 @@ protected void throwPolicyDoesnotApplyException(SchedulingPolicy policy)
public abstract void setPolicy(SchedulingPolicy policy)
throws AllocationConfigurationException;
public void setWeights(ResourceWeights weights){
this.weights = weights;
}
@Override
public ResourceWeights getWeights() {
return scheduler.getAllocationConfiguration().getQueueWeight(getName());
return weights;
}
public void setMinShare(Resource minShare){
this.minShare = minShare;
}
@Override
public Resource getMinShare() {
return scheduler.getAllocationConfiguration().getMinResources(getName());
return minShare;
}
public void setMaxShare(Resource maxShare){
this.maxShare = maxShare;
}
@Override
public Resource getMaxShare() {
return scheduler.getAllocationConfiguration().getMaxResources(getName());
return maxShare;
}
public void setMaxChildQueueResource(Resource maxChildShare){
this.maxChildQueueResource = maxChildShare;
}
public Resource getMaxChildQueueResource() {
return maxChildQueueResource;
}
public void setMaxRunningApps(int maxRunningApps){
this.maxRunningApps = maxRunningApps;
}
public int getMaxRunningApps() {
return maxRunningApps;
}
public void setMaxAMShare(float maxAMShare){
this.maxAMShare = maxAMShare;
}
@Override
@ -302,8 +348,7 @@ public abstract void collectSchedulerApplications(
* @return true if check passes (can assign) or false otherwise
*/
protected boolean assignContainerPreCheck(FSSchedulerNode node) {
if (!Resources.fitsIn(getResourceUsage(),
scheduler.getAllocationConfiguration().getMaxResources(getName()))
if (!Resources.fitsIn(getResourceUsage(), maxShare)
|| node.getReservedContainer() != null) {
return false;
}

View File

@ -1599,11 +1599,10 @@ private void applyChildDefaults() {
if ((queue.getParent() != null) &&
!configuredLeafQueues.contains(queue.getName()) &&
!configuredParentQueues.contains(queue.getName())) {
Resource max =
allocConf.getMaxChildResources(queue.getParent().getName());
Resource max = queue.getParent().getMaxChildQueueResource();
if (max != null) {
allocConf.setMaxResources(queue.getName(), max);
queue.setMaxShare(max);
}
}
}
@ -1665,7 +1664,7 @@ private void verifyMoveDoesNotViolateConstraints(FSAppAttempt app,
FSQueue cur = targetQueue;
while (cur != lowestCommonAncestor) {
// maxRunningApps
if (cur.getNumRunnableApps() == allocConf.getQueueMaxApps(cur.getQueueName())) {
if (cur.getNumRunnableApps() == cur.getMaxRunningApps()) {
throw new YarnException("Moving app attempt " + appAttId + " to queue "
+ queueName + " would violate queue maxRunningApps constraints on"
+ " queue " + cur.getQueueName());

View File

@ -66,8 +66,7 @@ public boolean canAppBeRunnable(FSQueue queue, String user) {
}
// Check queue and all parent queues
while (queue != null) {
int queueMaxApps = allocConf.getQueueMaxApps(queue.getName());
if (queue.getNumRunnableApps() >= queueMaxApps) {
if (queue.getNumRunnableApps() >= queue.getMaxRunningApps()) {
return false;
}
queue = queue.getParent();
@ -143,11 +142,10 @@ public void updateRunnabilityOnAppRemoval(FSAppAttempt app, FSLeafQueue queue) {
// Thus we find the ancestor queue highest in the tree for which the app
// that was at its maxRunningApps before the removal.
FSQueue highestQueueWithAppsNowRunnable = (queue.getNumRunnableApps() ==
allocConf.getQueueMaxApps(queue.getName()) - 1) ? queue : null;
queue.getMaxRunningApps() - 1) ? queue : null;
FSParentQueue parent = queue.getParent();
while (parent != null) {
if (parent.getNumRunnableApps() == allocConf.getQueueMaxApps(parent
.getName()) - 1) {
if (parent.getNumRunnableApps() == parent.getMaxRunningApps() - 1) {
highestQueueWithAppsNowRunnable = parent;
}
parent = parent.getParent();
@ -265,8 +263,7 @@ public void untrackNonRunnableApp(FSAppAttempt app) {
*/
private void gatherPossiblyRunnableAppLists(FSQueue queue,
List<List<FSAppAttempt>> appLists) {
if (queue.getNumRunnableApps() < scheduler.getAllocationConfiguration()
.getQueueMaxApps(queue.getName())) {
if (queue.getNumRunnableApps() < queue.getMaxRunningApps()) {
if (queue instanceof FSLeafQueue) {
appLists.add(
((FSLeafQueue)queue).getCopyOfNonRunnableAppSchedulables());

View File

@ -73,8 +73,9 @@ public FSParentQueue getRootQueue() {
public void initialize(Configuration conf) throws IOException,
SAXException, AllocationConfigurationException, ParserConfigurationException {
rootQueue = new FSParentQueue("root", scheduler, null);
rootQueue.init();
queues.put(rootQueue.getName(), rootQueue);
// Create the default queue
getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, true);
}
@ -174,7 +175,7 @@ private FSQueue getQueue(
}
return queue;
}
/**
* Create a leaf or parent queue based on what is specified in
* {@code queueType} and place it in the tree. Create any parents that don't
@ -273,31 +274,14 @@ private FSQueue createNewQueues(FSQueueType queueType,
// Only create a leaf queue at the very end
if (!i.hasNext() && (queueType != FSQueueType.PARENT)) {
FSLeafQueue leafQueue = new FSLeafQueue(queueName, scheduler, parent);
try {
leafQueue.setPolicy(queueConf.getDefaultSchedulingPolicy());
} catch (AllocationConfigurationException ex) {
LOG.warn("Failed to set default scheduling policy "
+ queueConf.getDefaultSchedulingPolicy()
+ " on new leaf queue.", ex);
}
leafQueues.add(leafQueue);
queue = leafQueue;
} else {
newParent = new FSParentQueue(queueName, scheduler, parent);
try {
newParent.setPolicy(queueConf.getDefaultSchedulingPolicy());
} catch (AllocationConfigurationException ex) {
LOG.warn("Failed to set default scheduling policy "
+ queueConf.getDefaultSchedulingPolicy()
+ " on new parent queue.", ex);
}
queue = newParent;
}
queue.init();
parent.addChildQueue(queue);
setChildResourceLimits(parent, queue, queueConf);
queues.put(queue.getName(), queue);
@ -331,10 +315,10 @@ void setChildResourceLimits(FSParentQueue parent, FSQueue child,
!configuredQueues.get(FSQueueType.PARENT).contains(child.getName())) {
// For ad hoc queues, set their max reource allocations based on
// their parents' default child settings.
Resource maxChild = queueConf.getMaxChildResources(parent.getName());
Resource maxChild = parent.getMaxChildQueueResource();
if (maxChild != null) {
queueConf.setMaxResources(child.getName(), maxChild);
child.setMaxShare(maxChild);
}
}
}
@ -515,22 +499,7 @@ public void updateAllocationConfiguration(AllocationConfiguration queueConf) {
rootQueue.recomputeSteadyShares();
for (FSQueue queue : queues.values()) {
// Update queue metrics
FSQueueMetrics queueMetrics = queue.getMetrics();
queueMetrics.setMinShare(queue.getMinShare());
queueMetrics.setMaxShare(queue.getMaxShare());
// Set scheduling policies and update queue metrics
try {
SchedulingPolicy policy = queueConf.getSchedulingPolicy(queue.getName());
policy.initialize(scheduler.getClusterResource());
queue.setPolicy(policy);
queueMetrics.setMaxApps(queueConf.getQueueMaxApps(queue.getName()));
queueMetrics.setSchedulingPolicy(policy.getName());
} catch (AllocationConfigurationException ex) {
LOG.warn("Cannot apply configured scheduling policy to queue "
+ queue.getName(), ex);
}
queue.init();
}
// Update steady fair shares for all queues

View File

@ -101,7 +101,7 @@ public FairSchedulerQueueInfo(FSQueue queue, FairScheduler scheduler) {
fractionMemMinShare = (float)minResources.getMemorySize() / clusterResources.getMemorySize();
fractionMemMaxShare = (float)maxResources.getMemorySize() / clusterResources.getMemorySize();
maxApps = allocConf.getQueueMaxApps(queueName);
maxApps = queue.getMaxRunningApps();
pendingContainers = queue.getMetrics().getPendingContainers();
allocatedContainers = queue.getMetrics().getAllocatedContainers();

View File

@ -247,8 +247,8 @@ public void testMoveNonRunnableApp() throws Exception {
QueueManager queueMgr = scheduler.getQueueManager();
FSLeafQueue oldQueue = queueMgr.getLeafQueue("queue1", true);
FSLeafQueue targetQueue = queueMgr.getLeafQueue("queue2", true);
scheduler.getAllocationConfiguration().queueMaxApps.put("root.queue1", 0);
scheduler.getAllocationConfiguration().queueMaxApps.put("root.queue2", 0);
oldQueue.setMaxRunningApps(0);
targetQueue.setMaxRunningApps(0);
ApplicationAttemptId appAttId =
createSchedulingRequest(1024, 1, "queue1", "user1", 3);
@ -265,7 +265,7 @@ public void testMoveMakesAppRunnable() throws Exception {
QueueManager queueMgr = scheduler.getQueueManager();
FSLeafQueue oldQueue = queueMgr.getLeafQueue("queue1", true);
FSLeafQueue targetQueue = queueMgr.getLeafQueue("queue2", true);
scheduler.getAllocationConfiguration().queueMaxApps.put("root.queue1", 0);
oldQueue.setMaxRunningApps(0);
ApplicationAttemptId appAttId =
createSchedulingRequest(1024, 1, "queue1", "user1", 3);

View File

@ -22,7 +22,6 @@
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.File;
import java.io.FileWriter;
@ -80,16 +79,11 @@ public void testUpdateDemand() {
resourceManager = new MockRM(conf);
resourceManager.start();
scheduler = (FairScheduler) resourceManager.getResourceScheduler();
scheduler.allocConf = mock(AllocationConfiguration.class);
String queueName = "root.queue1";
when(scheduler.allocConf.getMaxResources(queueName)).thenReturn(maxResource);
when(scheduler.allocConf.getMinResources(queueName)).thenReturn(Resources.none());
when(scheduler.allocConf.getQueueMaxApps(queueName)).
thenReturn(Integer.MAX_VALUE);
when(scheduler.allocConf.getSchedulingPolicy(queueName))
.thenReturn(SchedulingPolicy.DEFAULT_POLICY);
FSLeafQueue schedulable = new FSLeafQueue(queueName, scheduler, null);
schedulable.init();
schedulable.setMaxShare(maxResource);
assertEquals(schedulable.getMetrics().getMaxApps(), Integer.MAX_VALUE);
assertEquals(schedulable.getMetrics().getSchedulingPolicy(),
SchedulingPolicy.DEFAULT_POLICY.getName());

View File

@ -4348,9 +4348,9 @@ public void testMoveWouldViolateMaxAppsConstraints() throws Exception {
scheduler.reinitialize(conf, resourceManager.getRMContext());
QueueManager queueMgr = scheduler.getQueueManager();
queueMgr.getLeafQueue("queue2", true);
scheduler.getAllocationConfiguration().queueMaxApps.put("root.queue2", 0);
FSQueue queue2 = queueMgr.getLeafQueue("queue2", true);
queue2.setMaxRunningApps(0);
ApplicationAttemptId appAttId =
createSchedulingRequest(1024, 1, "queue1", "user1", 3);
@ -4365,9 +4365,8 @@ public void testMoveWouldViolateMaxResourcesConstraints() throws Exception {
QueueManager queueMgr = scheduler.getQueueManager();
FSLeafQueue oldQueue = queueMgr.getLeafQueue("queue1", true);
queueMgr.getLeafQueue("queue2", true);
scheduler.getAllocationConfiguration().maxQueueResources.put("root.queue2",
Resource.newInstance(1024, 1));
FSQueue queue2 = queueMgr.getLeafQueue("queue2", true);
queue2.setMaxShare(Resource.newInstance(1024, 1));
ApplicationAttemptId appAttId =
createSchedulingRequest(1024, 1, "queue1", "user1", 3);

View File

@ -37,7 +37,6 @@
public class TestMaxRunningAppsEnforcer {
private QueueManager queueManager;
private Map<String, Integer> queueMaxApps;
private Map<String, Integer> userMaxApps;
private MaxRunningAppsEnforcer maxAppsEnforcer;
private int appNum;
@ -59,7 +58,6 @@ public void setup() throws Exception {
queueManager = new QueueManager(scheduler);
queueManager.initialize(conf);
queueMaxApps = allocConf.queueMaxApps;
userMaxApps = allocConf.userMaxApps;
maxAppsEnforcer = new MaxRunningAppsEnforcer(scheduler);
appNum = 0;
@ -90,11 +88,12 @@ private void removeApp(FSAppAttempt app) {
@Test
public void testRemoveDoesNotEnableAnyApp() {
FSParentQueue root = queueManager.getRootQueue();
FSLeafQueue leaf1 = queueManager.getLeafQueue("root.queue1", true);
FSLeafQueue leaf2 = queueManager.getLeafQueue("root.queue2", true);
queueMaxApps.put("root", 2);
queueMaxApps.put("root.queue1", 1);
queueMaxApps.put("root.queue2", 1);
root.setMaxRunningApps(2);
leaf1.setMaxRunningApps(1);
leaf2.setMaxRunningApps(1);
FSAppAttempt app1 = addApp(leaf1, "user");
addApp(leaf2, "user");
addApp(leaf2, "user");
@ -111,7 +110,8 @@ public void testRemoveDoesNotEnableAnyApp() {
public void testRemoveEnablesAppOnCousinQueue() {
FSLeafQueue leaf1 = queueManager.getLeafQueue("root.queue1.subqueue1.leaf1", true);
FSLeafQueue leaf2 = queueManager.getLeafQueue("root.queue1.subqueue2.leaf2", true);
queueMaxApps.put("root.queue1", 2);
FSParentQueue queue1 = queueManager.getParentQueue("root.queue1", true);
queue1.setMaxRunningApps(2);
FSAppAttempt app1 = addApp(leaf1, "user");
addApp(leaf2, "user");
addApp(leaf2, "user");
@ -128,7 +128,7 @@ public void testRemoveEnablesAppOnCousinQueue() {
public void testRemoveEnablesOneByQueueOneByUser() {
FSLeafQueue leaf1 = queueManager.getLeafQueue("root.queue1.leaf1", true);
FSLeafQueue leaf2 = queueManager.getLeafQueue("root.queue1.leaf2", true);
queueMaxApps.put("root.queue1.leaf1", 2);
leaf1.setMaxRunningApps(2);
userMaxApps.put("user1", 1);
FSAppAttempt app1 = addApp(leaf1, "user1");
addApp(leaf1, "user2");
@ -148,7 +148,8 @@ public void testRemoveEnablesOneByQueueOneByUser() {
public void testRemoveEnablingOrderedByStartTime() {
FSLeafQueue leaf1 = queueManager.getLeafQueue("root.queue1.subqueue1.leaf1", true);
FSLeafQueue leaf2 = queueManager.getLeafQueue("root.queue1.subqueue2.leaf2", true);
queueMaxApps.put("root.queue1", 2);
FSParentQueue queue1 = queueManager.getParentQueue("root.queue1", true);
queue1.setMaxRunningApps(2);
FSAppAttempt app1 = addApp(leaf1, "user");
addApp(leaf2, "user");
addApp(leaf2, "user");
@ -168,7 +169,8 @@ public void testRemoveEnablingOrderedByStartTime() {
public void testMultipleAppsWaitingOnCousinQueue() {
FSLeafQueue leaf1 = queueManager.getLeafQueue("root.queue1.subqueue1.leaf1", true);
FSLeafQueue leaf2 = queueManager.getLeafQueue("root.queue1.subqueue2.leaf2", true);
queueMaxApps.put("root.queue1", 2);
FSParentQueue queue1 = queueManager.getParentQueue("root.queue1", true);
queue1.setMaxRunningApps(2);
FSAppAttempt app1 = addApp(leaf1, "user");
addApp(leaf2, "user");
addApp(leaf2, "user");

View File

@ -45,8 +45,6 @@ public void setUp() throws Exception {
// Set up some queues to test default child max resource inheritance
allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.test");
allocConf.setMaxChildResources("root.test",
Resources.createResource(8192, 256));
allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.test.childA");
allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.test.childB");
@ -207,16 +205,17 @@ public void testCreateQueueWithChildDefaults() {
AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
queueManager.updateAllocationConfiguration(allocConf);
queueManager.getQueue("root.test").setMaxChildQueueResource(
Resources.createResource(8192, 256));
FSQueue q1 = queueManager.createQueue("root.test.childC", FSQueueType.LEAF);
assertNotNull("Leaf queue root.test.childC was not created",
queueManager.getLeafQueue("root.test.childC", false));
assertEquals("createQueue() returned wrong queue",
"root.test.childC", q1.getName());
assertEquals("Max resources for root.queue1 were not inherited from "
+ "parent's max child resources", Resources.createResource(8192, 256),
allocConf.getMaxResources("root.test.childC"));
q1.getMaxShare());
FSQueue q2 = queueManager.createQueue("root.test.childD",
FSQueueType.PARENT);
@ -228,7 +227,7 @@ public void testCreateQueueWithChildDefaults() {
assertEquals("Max resources for root.test.childD were not inherited "
+ "from parent's max child resources",
Resources.createResource(8192, 256),
allocConf.getMaxResources("root.test.childD"));
q2.getMaxShare());
// Check that the childA and childB queues weren't impacted
// by the child defaults
@ -236,12 +235,12 @@ public void testCreateQueueWithChildDefaults() {
queueManager.getLeafQueue("root.test.childA", false));
assertEquals("Max resources for root.test.childA were inherited from "
+ "parent's max child resources", Resources.unbounded(),
allocConf.getMaxResources("root.test.childA"));
queueManager.getLeafQueue("root.test.childA", false).getMaxShare());
assertNotNull("Leaf queue root.test.childB was not created during setup",
queueManager.getParentQueue("root.test.childB", false));
assertEquals("Max resources for root.test.childB were inherited from "
+ "parent's max child resources", Resources.unbounded(),
allocConf.getMaxResources("root.test.childB"));
queueManager.getParentQueue("root.test.childB", false).getMaxShare());
}
/**
@ -260,11 +259,11 @@ public void testCreateLeafQueueWithDefaults() {
// Min default is 0,0
assertEquals("Min resources were not set to default",
Resources.createResource(0, 0),
allocConf.getMinResources("root.queue1"));
q1.getMinShare());
// Max default is unbounded
assertEquals("Max resources were not set to default", Resources.unbounded(),
allocConf.getMaxResources("root.queue1"));
q1.getMaxShare());
}
/**