YARN-5264. Store all queue-specific information in FSQueue. (Yufei Gu via kasha)
(cherry picked from commit f6ea9be547
)
This commit is contained in:
parent
4ee1729cb0
commit
a5748eec7b
|
@ -22,6 +22,8 @@ import java.util.HashSet;
|
|||
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 org.apache.hadoop.yarn.util.resource.Resources;
|
|||
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 class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
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 class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
* @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 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
* @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 class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
* @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 class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
|
||||
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 class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -264,29 +264,27 @@ public class FSLeafQueue extends FSQueue {
|
|||
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 class FSLeafQueue extends FSQueue {
|
|||
* @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 class FSLeafQueue extends FSQueue {
|
|||
* @param weight queue weight
|
||||
*/
|
||||
public void setWeights(float weight) {
|
||||
scheduler.getAllocationConfiguration().setQueueWeight(getName(),
|
||||
new ResourceWeights(weight));
|
||||
this.weights = new ResourceWeights(weight);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -152,8 +152,6 @@ public class FSParentQueue extends FSQueue {
|
|||
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 class FSParentQueue extends FSQueue {
|
|||
" 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 class FSParentQueue extends FSQueue {
|
|||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("The updated demand for " + getName() + " is " + demand +
|
||||
"; the max is " + maxRes);
|
||||
"; the max is " + maxShare);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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 abstract class FSQueue implements Queue, Schedulable {
|
|||
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 @@ public abstract class FSQueue implements Queue, Schedulable {
|
|||
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 class FSQueue implements Queue, Schedulable {
|
|||
* @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;
|
||||
}
|
||||
|
|
|
@ -1599,11 +1599,10 @@ public class FairScheduler extends
|
|||
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 @@ public class FairScheduler extends
|
|||
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());
|
||||
|
|
|
@ -66,8 +66,7 @@ public class MaxRunningAppsEnforcer {
|
|||
}
|
||||
// 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 class MaxRunningAppsEnforcer {
|
|||
// 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 class MaxRunningAppsEnforcer {
|
|||
*/
|
||||
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());
|
||||
|
|
|
@ -73,8 +73,9 @@ public class QueueManager {
|
|||
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 @@ public class QueueManager {
|
|||
}
|
||||
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 @@ public class QueueManager {
|
|||
// 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 @@ public class QueueManager {
|
|||
!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 class QueueManager {
|
|||
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
|
||||
|
|
|
@ -101,7 +101,7 @@ public class FairSchedulerQueueInfo {
|
|||
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();
|
||||
|
|
|
@ -247,8 +247,8 @@ public class TestAppRunnability extends FairSchedulerTestBase {
|
|||
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 class TestAppRunnability extends FairSchedulerTestBase {
|
|||
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);
|
||||
|
|
|
@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
|
|||
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 class TestFSLeafQueue extends FairSchedulerTestBase {
|
|||
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());
|
||||
|
|
|
@ -4348,9 +4348,9 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
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 class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
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);
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.junit.Test;
|
|||
|
||||
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 class TestMaxRunningAppsEnforcer {
|
|||
|
||||
queueManager = new QueueManager(scheduler);
|
||||
queueManager.initialize(conf);
|
||||
queueMaxApps = allocConf.queueMaxApps;
|
||||
userMaxApps = allocConf.userMaxApps;
|
||||
maxAppsEnforcer = new MaxRunningAppsEnforcer(scheduler);
|
||||
appNum = 0;
|
||||
|
@ -90,11 +88,12 @@ public class TestMaxRunningAppsEnforcer {
|
|||
|
||||
@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 class TestMaxRunningAppsEnforcer {
|
|||
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 class TestMaxRunningAppsEnforcer {
|
|||
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 class TestMaxRunningAppsEnforcer {
|
|||
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 class TestMaxRunningAppsEnforcer {
|
|||
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");
|
||||
|
|
|
@ -45,8 +45,6 @@ public class TestQueueManager {
|
|||
|
||||
// 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 class TestQueueManager {
|
|||
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 class TestQueueManager {
|
|||
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 class TestQueueManager {
|
|||
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 class TestQueueManager {
|
|||
// 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());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue