YARN-11024. Create an AbstractLeafQueue to store the common LeafQueue + AutoCreatedLeafQueue functionality. Contributed by Benjamin Teke
This commit is contained in:
parent
40464a5e6c
commit
898055e204
|
@ -24,7 +24,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
|
||||
|
@ -86,7 +86,7 @@ public class FifoCandidatesSelector
|
|||
}
|
||||
|
||||
// compute resToObtainByPartition considered inter-queue preemption
|
||||
LeafQueue leafQueue = preemptionContext.getQueueByPartition(queueName,
|
||||
AbstractLeafQueue leafQueue = preemptionContext.getQueueByPartition(queueName,
|
||||
RMNodeLabelsManager.NO_LABEL).leafQueue;
|
||||
|
||||
Map<String, Resource> resToObtainByPartition =
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.IntraQueue
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.IntraQueuePreemptionOrderPolicy;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderingPolicy;
|
||||
|
@ -577,7 +577,7 @@ public class FifoIntraQueuePreemptionPlugin
|
|||
}
|
||||
|
||||
private Resource calculateUsedAMResourcesPerQueue(String partition,
|
||||
LeafQueue leafQueue, Map<String, Resource> perUserAMUsed) {
|
||||
AbstractLeafQueue leafQueue, Map<String, Resource> perUserAMUsed) {
|
||||
Collection<FiCaSchedulerApp> runningApps = leafQueue.getApplications();
|
||||
Resource amUsed = Resources.createResource(0, 0);
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.IntraQueuePreemptionOrderPolicy;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.AbstractComparatorOrderingPolicy;
|
||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||
|
@ -146,7 +146,7 @@ public class IntraQueueCandidatesSelector extends PreemptionCandidatesSelector {
|
|||
|
||||
// 4. Iterate from most under-served queue in order.
|
||||
for (String queueName : queueNames) {
|
||||
LeafQueue leafQueue = preemptionContext.getQueueByPartition(queueName,
|
||||
AbstractLeafQueue leafQueue = preemptionContext.getQueueByPartition(queueName,
|
||||
RMNodeLabelsManager.NO_LABEL).leafQueue;
|
||||
|
||||
// skip if not a leafqueue
|
||||
|
@ -181,7 +181,7 @@ public class IntraQueueCandidatesSelector extends PreemptionCandidatesSelector {
|
|||
leafQueue.getReadLock().lock();
|
||||
try {
|
||||
for (FiCaSchedulerApp app : apps) {
|
||||
preemptFromLeastStarvedApp(leafQueue, app, selectedCandidates,
|
||||
preemptFromLeastStarvedApp(app, selectedCandidates,
|
||||
curCandidates, clusterResource, totalPreemptedResourceAllowed,
|
||||
resToObtainByPartition, rollingResourceUsagePerUser);
|
||||
}
|
||||
|
@ -195,7 +195,7 @@ public class IntraQueueCandidatesSelector extends PreemptionCandidatesSelector {
|
|||
}
|
||||
|
||||
private void initializeUsageAndUserLimitForCompute(Resource clusterResource,
|
||||
String partition, LeafQueue leafQueue,
|
||||
String partition, AbstractLeafQueue leafQueue,
|
||||
Map<String, Resource> rollingResourceUsagePerUser) {
|
||||
for (String user : leafQueue.getAllUsers()) {
|
||||
// Initialize used resource of a given user for rolling computation.
|
||||
|
@ -206,8 +206,7 @@ public class IntraQueueCandidatesSelector extends PreemptionCandidatesSelector {
|
|||
}
|
||||
}
|
||||
|
||||
private void preemptFromLeastStarvedApp(LeafQueue leafQueue,
|
||||
FiCaSchedulerApp app,
|
||||
private void preemptFromLeastStarvedApp(FiCaSchedulerApp app,
|
||||
Map<ApplicationAttemptId, Set<RMContainer>> selectedCandidates,
|
||||
Map<ApplicationAttemptId, Set<RMContainer>> curCandidates,
|
||||
Resource clusterResource, Resource totalPreemptedResourceAllowed,
|
||||
|
@ -293,7 +292,7 @@ public class IntraQueueCandidatesSelector extends PreemptionCandidatesSelector {
|
|||
for (String queueName : queueNames) {
|
||||
TempQueuePerPartition tq = context.getQueueByPartition(queueName,
|
||||
partition);
|
||||
LeafQueue leafQueue = tq.leafQueue;
|
||||
AbstractLeafQueue leafQueue = tq.leafQueue;
|
||||
|
||||
// skip if its parent queue
|
||||
if (null == leafQueue) {
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.util.Map;
|
|||
|
||||
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.AbstractLeafQueue;
|
||||
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;
|
||||
|
@ -56,7 +56,7 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
|
|||
|
||||
final ArrayList<TempQueuePerPartition> children;
|
||||
private Collection<TempAppPerPartition> apps;
|
||||
LeafQueue leafQueue;
|
||||
AbstractLeafQueue leafQueue;
|
||||
ParentQueue parentQueue;
|
||||
boolean preemptionDisabled;
|
||||
|
||||
|
@ -81,8 +81,8 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
|
|||
super(queueName, current, Resource.newInstance(0, 0), reserved,
|
||||
Resource.newInstance(0, 0));
|
||||
|
||||
if (queue instanceof LeafQueue) {
|
||||
LeafQueue l = (LeafQueue) queue;
|
||||
if (queue instanceof AbstractLeafQueue) {
|
||||
AbstractLeafQueue l = (AbstractLeafQueue) queue;
|
||||
pending = l.getTotalPendingResourcesConsideringUserLimit(
|
||||
totalPartitionResource, partition, false);
|
||||
pendingDeductReserved = l.getTotalPendingResourcesConsideringUserLimit(
|
||||
|
@ -113,7 +113,7 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
|
|||
this.effMaxRes = effMaxRes;
|
||||
}
|
||||
|
||||
public void setLeafQueue(LeafQueue l) {
|
||||
public void setLeafQueue(AbstractLeafQueue l) {
|
||||
assert children.size() == 0;
|
||||
this.leafQueue = l;
|
||||
}
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -277,7 +277,7 @@ public class CSMappingPlacementRule extends PlacementRule {
|
|||
}
|
||||
|
||||
CSQueue queue = queueManager.getQueueByFullName(normalizedName);
|
||||
if (queue != null && !(queue instanceof LeafQueue)) {
|
||||
if (queue != null && !(queue instanceof AbstractLeafQueue)) {
|
||||
throw new YarnException("Mapping rule returned a non-leaf queue '" +
|
||||
normalizedName + "', cannot place application in it.");
|
||||
}
|
||||
|
|
|
@ -21,13 +21,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.placement;
|
|||
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.CapacitySchedulerQueueManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
|
||||
|
||||
/**
|
||||
* Utility class for Capacity Scheduler queue PlacementRules.
|
||||
*/
|
||||
|
@ -83,7 +81,7 @@ public final class QueuePlacementRuleUtils {
|
|||
public static QueueMapping validateAndGetQueueMapping(
|
||||
CapacitySchedulerQueueManager queueManager, CSQueue queue,
|
||||
QueueMapping mapping) throws IOException {
|
||||
if (!(queue instanceof LeafQueue)) {
|
||||
if (!(queue instanceof AbstractLeafQueue)) {
|
||||
throw new IOException(
|
||||
"mapping contains invalid or non-leaf queue : " +
|
||||
mapping.getFullPath());
|
||||
|
|
|
@ -22,7 +22,7 @@ import org.apache.hadoop.util.Sets;
|
|||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
|
||||
|
@ -91,7 +91,7 @@ public class MappingRuleValidationContextImpl
|
|||
"' under it.");
|
||||
case QUEUE_EXISTS:
|
||||
CSQueue queue = queueManager.getQueue(normalizedPath);
|
||||
if (!(queue instanceof LeafQueue)) {
|
||||
if (!(queue instanceof AbstractLeafQueue)) {
|
||||
throw new YarnException("Target queue '" + path.getFullPath() +
|
||||
"' but it's not a leaf queue.");
|
||||
}
|
||||
|
@ -157,7 +157,7 @@ public class MappingRuleValidationContextImpl
|
|||
//if the static part of our queue exists, and it's not a leaf queue,
|
||||
//we cannot do any deeper validation
|
||||
if (queue != null) {
|
||||
if (queue instanceof LeafQueue) {
|
||||
if (queue instanceof AbstractLeafQueue) {
|
||||
throw new YarnException("Queue path '" + path +"' is invalid " +
|
||||
"because '" + normalizedStaticPart + "' is a leaf queue, " +
|
||||
"which can have no other queues under it.");
|
||||
|
|
|
@ -34,7 +34,7 @@ import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
|
|||
* Abstract class for dynamic auto created queues managed by an implementation
|
||||
* of AbstractManagedParentQueue
|
||||
*/
|
||||
public class AbstractAutoCreatedLeafQueue extends LeafQueue {
|
||||
public class AbstractAutoCreatedLeafQueue extends AbstractLeafQueue {
|
||||
|
||||
protected AbstractManagedParentQueue parent;
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -36,15 +36,19 @@ import java.util.Set;
|
|||
* ManagedParentQueue for auto created dynamic queues
|
||||
*/
|
||||
public class AutoCreatedLeafQueue extends AbstractAutoCreatedLeafQueue {
|
||||
|
||||
private static final Logger LOG = LoggerFactory
|
||||
.getLogger(AutoCreatedLeafQueue.class);
|
||||
|
||||
public AutoCreatedLeafQueue(CapacitySchedulerContext cs, String queueName,
|
||||
ManagedParentQueue parent) throws IOException {
|
||||
// TODO once YARN-10907 is merged the duplicated collection of
|
||||
// leafQueueConfigs won't be necessary
|
||||
super(cs, parent.getLeafQueueConfigs(queueName),
|
||||
queueName,
|
||||
parent, null);
|
||||
super.setupQueueConfigs(cs.getClusterResource(), parent.getLeafQueueConfigs(queueName));
|
||||
|
||||
LOG.debug("Initialized AutoCreatedLeafQueue: name={}, fullname={}", queueName, getQueuePath());
|
||||
updateCapacitiesToZero();
|
||||
}
|
||||
|
||||
|
|
|
@ -182,7 +182,7 @@ public class CSMaxRunningAppsEnforcer {
|
|||
// the queue was already at its max before the removal.
|
||||
// Thus we find the ancestor queue highest in the tree for which the app
|
||||
// that was at its maxRunningApps before the removal.
|
||||
LeafQueue queue = app.getCSLeafQueue();
|
||||
AbstractLeafQueue queue = app.getCSLeafQueue();
|
||||
AbstractCSQueue highestQueueWithAppsNowRunnable =
|
||||
(queue.getNumRunnableApps() == queue.getMaxParallelApps() - 1)
|
||||
? queue : null;
|
||||
|
@ -243,7 +243,7 @@ public class CSMaxRunningAppsEnforcer {
|
|||
}
|
||||
|
||||
if (checkRunnabilityWithUpdate(next)) {
|
||||
LeafQueue nextQueue = next.getCSLeafQueue();
|
||||
AbstractLeafQueue nextQueue = next.getCSLeafQueue();
|
||||
LOG.info("{} is now runnable in {}",
|
||||
next.getApplicationAttemptId(), nextQueue);
|
||||
trackRunnableApp(next);
|
||||
|
@ -322,9 +322,9 @@ public class CSMaxRunningAppsEnforcer {
|
|||
private void gatherPossiblyRunnableAppLists(AbstractCSQueue queue,
|
||||
List<List<FiCaSchedulerApp>> appLists) {
|
||||
if (queue.getNumRunnableApps() < queue.getMaxParallelApps()) {
|
||||
if (queue instanceof LeafQueue) {
|
||||
if (queue instanceof AbstractLeafQueue) {
|
||||
appLists.add(
|
||||
((LeafQueue)queue).getCopyOfNonRunnableAppSchedulables());
|
||||
((AbstractLeafQueue)queue).getCopyOfNonRunnableAppSchedulables());
|
||||
} else {
|
||||
for (CSQueue child : queue.getChildQueues()) {
|
||||
gatherPossiblyRunnableAppLists((AbstractCSQueue) child, appLists);
|
||||
|
|
|
@ -27,13 +27,13 @@ import org.apache.hadoop.yarn.util.resource.Resources;
|
|||
public class CapacityHeadroomProvider {
|
||||
|
||||
UsersManager.User user;
|
||||
LeafQueue queue;
|
||||
AbstractLeafQueue queue;
|
||||
FiCaSchedulerApp application;
|
||||
LeafQueue.QueueResourceLimitsInfo queueResourceLimitsInfo;
|
||||
AbstractLeafQueue.QueueResourceLimitsInfo queueResourceLimitsInfo;
|
||||
|
||||
public CapacityHeadroomProvider(UsersManager.User user, LeafQueue queue,
|
||||
public CapacityHeadroomProvider(UsersManager.User user, AbstractLeafQueue queue,
|
||||
FiCaSchedulerApp application,
|
||||
LeafQueue.QueueResourceLimitsInfo queueResourceLimitsInfo) {
|
||||
AbstractLeafQueue.QueueResourceLimitsInfo queueResourceLimitsInfo) {
|
||||
|
||||
this.user = user;
|
||||
this.queue = queue;
|
||||
|
|
|
@ -917,7 +917,7 @@ public class CapacityScheduler extends
|
|||
throw new QueueInvalidException(queueErrorMsg);
|
||||
}
|
||||
}
|
||||
if (!(queue instanceof LeafQueue)) {
|
||||
if (!(queue instanceof AbstractLeafQueue)) {
|
||||
// During RM restart, this means leaf queue was converted to a parent
|
||||
// queue, which is not supported for running apps.
|
||||
if (!appShouldFailFast) {
|
||||
|
@ -942,7 +942,7 @@ public class CapacityScheduler extends
|
|||
// that means its previous state was DRAINING. So we auto transit
|
||||
// the state to DRAINING for recovery.
|
||||
if (queue.getState() == QueueState.STOPPED) {
|
||||
((LeafQueue) queue).recoverDrainingState();
|
||||
((AbstractLeafQueue) queue).recoverDrainingState();
|
||||
}
|
||||
// Submit to the queue
|
||||
try {
|
||||
|
@ -1090,7 +1090,7 @@ public class CapacityScheduler extends
|
|||
return;
|
||||
}
|
||||
|
||||
if (!(queue instanceof LeafQueue)) {
|
||||
if (!(queue instanceof AbstractLeafQueue)) {
|
||||
String message =
|
||||
"Application " + applicationId + " submitted by user : " + user
|
||||
+ " to non-leaf queue : " + queueName;
|
||||
|
@ -1242,7 +1242,7 @@ public class CapacityScheduler extends
|
|||
return;
|
||||
}
|
||||
CSQueue queue = (CSQueue) application.getQueue();
|
||||
if (!(queue instanceof LeafQueue)) {
|
||||
if (!(queue instanceof AbstractLeafQueue)) {
|
||||
LOG.error("Cannot finish application " + "from non-leaf queue: " + queue
|
||||
.getQueuePath());
|
||||
} else{
|
||||
|
@ -1301,7 +1301,7 @@ public class CapacityScheduler extends
|
|||
// Inform the queue
|
||||
Queue queue = attempt.getQueue();
|
||||
CSQueue csQueue = (CSQueue) queue;
|
||||
if (!(csQueue instanceof LeafQueue)) {
|
||||
if (!(csQueue instanceof AbstractLeafQueue)) {
|
||||
LOG.error(
|
||||
"Cannot finish application " + "from non-leaf queue: "
|
||||
+ csQueue.getQueuePath());
|
||||
|
@ -1367,7 +1367,7 @@ public class CapacityScheduler extends
|
|||
// Release containers
|
||||
releaseContainers(release, application);
|
||||
|
||||
LeafQueue updateDemandForQueue = null;
|
||||
AbstractLeafQueue updateDemandForQueue = null;
|
||||
|
||||
// Sanity check for new allocation requests
|
||||
normalizeResourceRequests(ask);
|
||||
|
@ -1398,7 +1398,7 @@ public class CapacityScheduler extends
|
|||
// Update application requests
|
||||
if (application.updateResourceRequests(ask) || application
|
||||
.updateSchedulingRequests(schedulingRequests)) {
|
||||
updateDemandForQueue = (LeafQueue) application.getQueue();
|
||||
updateDemandForQueue = (AbstractLeafQueue) application.getQueue();
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -1783,7 +1783,7 @@ public class CapacityScheduler extends
|
|||
LOG.debug("Trying to fulfill reservation for application {} on node: {}",
|
||||
reservedApplication.getApplicationId(), node.getNodeID());
|
||||
|
||||
LeafQueue queue = ((LeafQueue) reservedApplication.getQueue());
|
||||
AbstractLeafQueue queue = ((AbstractLeafQueue) reservedApplication.getQueue());
|
||||
CSAssignment assignment = queue.assignContainers(getClusterResource(),
|
||||
new SimpleCandidateNodeSet<>(node),
|
||||
// TODO, now we only consider limits for parent for non-labeled
|
||||
|
@ -2386,7 +2386,7 @@ public class CapacityScheduler extends
|
|||
}
|
||||
|
||||
// Inform the queue
|
||||
LeafQueue queue = (LeafQueue) application.getQueue();
|
||||
AbstractLeafQueue queue = (AbstractLeafQueue) application.getQueue();
|
||||
queue.completedContainer(getClusterResource(), application, node,
|
||||
rmContainer, containerStatus, event, null, true);
|
||||
}
|
||||
|
@ -2673,7 +2673,7 @@ public class CapacityScheduler extends
|
|||
throws YarnException {
|
||||
writeLock.lock();
|
||||
try {
|
||||
LeafQueue queue = this.queueManager.getAndCheckLeafQueue(inQueue);
|
||||
AbstractLeafQueue queue = this.queueManager.getAndCheckLeafQueue(inQueue);
|
||||
AbstractManagedParentQueue parent =
|
||||
(AbstractManagedParentQueue) queue.getParent();
|
||||
|
||||
|
@ -2716,10 +2716,10 @@ public class CapacityScheduler extends
|
|||
throw new YarnException("App to be moved " + appId + " not found.");
|
||||
}
|
||||
String sourceQueueName = application.getQueue().getQueueName();
|
||||
LeafQueue source =
|
||||
AbstractLeafQueue source =
|
||||
this.queueManager.getAndCheckLeafQueue(sourceQueueName);
|
||||
String destQueueName = handleMoveToPlanQueue(targetQueueName);
|
||||
LeafQueue dest = this.queueManager.getAndCheckLeafQueue(destQueueName);
|
||||
AbstractLeafQueue dest = this.queueManager.getAndCheckLeafQueue(destQueueName);
|
||||
|
||||
String user = application.getUser();
|
||||
try {
|
||||
|
@ -2777,7 +2777,7 @@ public class CapacityScheduler extends
|
|||
((CSQueue) queue).getQueuePath() : queue.getQueueName();
|
||||
this.queueManager.getAndCheckLeafQueue(sourceQueueName);
|
||||
String destQueueName = handleMoveToPlanQueue(newQueue);
|
||||
LeafQueue dest = this.queueManager.getAndCheckLeafQueue(destQueueName);
|
||||
AbstractLeafQueue dest = this.queueManager.getAndCheckLeafQueue(destQueueName);
|
||||
// Validation check - ACLs, submission limits for user & queue
|
||||
String user = application.getUser();
|
||||
// Check active partition only when attempt is available
|
||||
|
@ -2804,7 +2804,7 @@ public class CapacityScheduler extends
|
|||
* @param dest
|
||||
* @throws YarnException
|
||||
*/
|
||||
private void checkQueuePartition(FiCaSchedulerApp app, LeafQueue dest)
|
||||
private void checkQueuePartition(FiCaSchedulerApp app, AbstractLeafQueue dest)
|
||||
throws YarnException {
|
||||
if (!YarnConfiguration.areNodeLabelsEnabled(conf)) {
|
||||
return;
|
||||
|
@ -2854,7 +2854,7 @@ public class CapacityScheduler extends
|
|||
}
|
||||
return getMaximumResourceCapability();
|
||||
}
|
||||
if (!(queue instanceof LeafQueue)) {
|
||||
if (!(queue instanceof AbstractLeafQueue)) {
|
||||
LOG.error("queue " + queueName + " is not an leaf queue");
|
||||
return getMaximumResourceCapability();
|
||||
}
|
||||
|
@ -2863,7 +2863,7 @@ public class CapacityScheduler extends
|
|||
// getMaximumResourceCapability() returns maximum allocation considers
|
||||
// per-node maximum resources. So return (component-wise) min of the two.
|
||||
|
||||
Resource queueMaxAllocation = ((LeafQueue)queue).getMaximumAllocation();
|
||||
Resource queueMaxAllocation = queue.getMaximumAllocation();
|
||||
Resource clusterMaxAllocationConsiderNodeMax =
|
||||
getMaximumResourceCapability();
|
||||
|
||||
|
@ -2989,7 +2989,7 @@ public class CapacityScheduler extends
|
|||
|
||||
// As we use iterator over a TreeSet for OrderingPolicy, once we change
|
||||
// priority then reinsert back to make order correct.
|
||||
LeafQueue queue = (LeafQueue) getQueue(rmApp.getQueue());
|
||||
AbstractLeafQueue queue = (AbstractLeafQueue) getQueue(rmApp.getQueue());
|
||||
queue.updateApplicationPriority(application, appPriority);
|
||||
|
||||
LOG.info("Priority '" + appPriority + "' is updated in queue :"
|
||||
|
@ -3404,14 +3404,14 @@ public class CapacityScheduler extends
|
|||
readLock.lock();
|
||||
try {
|
||||
CSQueue queue = getQueue(queueName);
|
||||
if (queue == null || !(queue instanceof LeafQueue)) {
|
||||
if (!(queue instanceof AbstractLeafQueue)) {
|
||||
return lifetimeRequestedByApp;
|
||||
}
|
||||
|
||||
long defaultApplicationLifetime =
|
||||
((LeafQueue) queue).getDefaultApplicationLifetime();
|
||||
queue.getDefaultApplicationLifetime();
|
||||
long maximumApplicationLifetime =
|
||||
((LeafQueue) queue).getMaximumApplicationLifetime();
|
||||
queue.getMaximumApplicationLifetime();
|
||||
|
||||
// check only for maximum, that's enough because default can't
|
||||
// exceed maximum
|
||||
|
@ -3434,7 +3434,7 @@ public class CapacityScheduler extends
|
|||
@Override
|
||||
public long getMaximumApplicationLifetime(String queueName) {
|
||||
CSQueue queue = getQueue(queueName);
|
||||
if (queue == null || !(queue instanceof LeafQueue)) {
|
||||
if (!(queue instanceof AbstractLeafQueue)) {
|
||||
if (isAmbiguous(queueName)) {
|
||||
LOG.error("Ambiguous queue reference: " + queueName
|
||||
+ " please use full queue path instead.");
|
||||
|
@ -3444,7 +3444,7 @@ public class CapacityScheduler extends
|
|||
return -1;
|
||||
}
|
||||
// In seconds
|
||||
return ((LeafQueue) queue).getMaximumApplicationLifetime();
|
||||
return queue.getMaximumApplicationLifetime();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -186,7 +186,7 @@ public final class CapacitySchedulerConfigValidator {
|
|||
+ " is set to true");
|
||||
}
|
||||
|
||||
if (newQueue instanceof LeafQueue) {
|
||||
if (newQueue instanceof AbstractLeafQueue) {
|
||||
LOG.info("Converting the parent queue: {} to leaf queue.", oldQueue.getQueuePath());
|
||||
}
|
||||
}
|
||||
|
@ -194,7 +194,7 @@ public final class CapacitySchedulerConfigValidator {
|
|||
|
||||
private static void validateLeafQueueConversion(CSQueue oldQueue,
|
||||
CSQueue newQueue) throws IOException {
|
||||
if (oldQueue instanceof LeafQueue && newQueue instanceof ParentQueue) {
|
||||
if (oldQueue instanceof AbstractLeafQueue && newQueue instanceof ParentQueue) {
|
||||
if (isEitherQueueStopped(oldQueue.getState(), newQueue.getState())) {
|
||||
LOG.info("Converting the leaf queue: {} to parent queue.", oldQueue.getQueuePath());
|
||||
} else {
|
||||
|
|
|
@ -375,8 +375,8 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|||
permissions.add(
|
||||
new Permission(csQueue.getPrivilegedEntity(), csQueue.getACLs()));
|
||||
|
||||
if (queue instanceof LeafQueue) {
|
||||
LeafQueue lQueue = (LeafQueue) queue;
|
||||
if (queue instanceof AbstractLeafQueue) {
|
||||
AbstractLeafQueue lQueue = (AbstractLeafQueue) queue;
|
||||
|
||||
// Clear Priority ACLs first since reinitialize also call same.
|
||||
appPriorityACLManager.clearPriorityACLs(lQueue.getQueuePath());
|
||||
|
@ -397,17 +397,17 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|||
* @throws YarnException if the queue does not exist or the queue
|
||||
* is not the type of LeafQueue.
|
||||
*/
|
||||
public LeafQueue getAndCheckLeafQueue(String queue) throws YarnException {
|
||||
public AbstractLeafQueue getAndCheckLeafQueue(String queue) throws YarnException {
|
||||
CSQueue ret = this.getQueue(queue);
|
||||
if (ret == null) {
|
||||
throw new YarnException("The specified Queue: " + queue
|
||||
+ " doesn't exist");
|
||||
}
|
||||
if (!(ret instanceof LeafQueue)) {
|
||||
if (!(ret instanceof AbstractLeafQueue)) {
|
||||
throw new YarnException("The specified Queue: " + queue
|
||||
+ " is not a Leaf Queue.");
|
||||
}
|
||||
return (LeafQueue) ret;
|
||||
return (AbstractLeafQueue) ret;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -527,7 +527,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|||
* @throws YarnException if the given path is not eligible to be auto created
|
||||
* @throws IOException if the given path can not be added to the parent
|
||||
*/
|
||||
public LeafQueue createQueue(QueuePath queue)
|
||||
public AbstractLeafQueue createQueue(QueuePath queue)
|
||||
throws YarnException, IOException {
|
||||
String leafQueueName = queue.getLeafName();
|
||||
String parentQueueName = queue.getParent();
|
||||
|
@ -668,7 +668,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|||
return leafQueue;
|
||||
}
|
||||
|
||||
private LeafQueue createLegacyAutoQueue(QueuePath queue)
|
||||
private AbstractLeafQueue createLegacyAutoQueue(QueuePath queue)
|
||||
throws IOException, SchedulerDynamicEditException {
|
||||
CSQueue parentQueue = getQueue(queue.getParent());
|
||||
// Case 1: Handle ManagedParentQueue
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -334,7 +334,7 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
|
|||
try {
|
||||
List<FiCaSchedulerApp> apps = new ArrayList<>();
|
||||
for (CSQueue childQueue : getChildQueues()) {
|
||||
apps.addAll(((LeafQueue) childQueue).getApplications());
|
||||
apps.addAll(((AbstractLeafQueue) childQueue).getApplications());
|
||||
}
|
||||
return Collections.unmodifiableList(apps);
|
||||
} finally {
|
||||
|
@ -347,7 +347,7 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
|
|||
try {
|
||||
List<FiCaSchedulerApp> apps = new ArrayList<>();
|
||||
for (CSQueue childQueue : getChildQueues()) {
|
||||
apps.addAll(((LeafQueue) childQueue).getPendingApplications());
|
||||
apps.addAll(((AbstractLeafQueue) childQueue).getPendingApplications());
|
||||
}
|
||||
return Collections.unmodifiableList(apps);
|
||||
} finally {
|
||||
|
@ -360,7 +360,7 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
|
|||
try {
|
||||
List<FiCaSchedulerApp> apps = new ArrayList<>();
|
||||
for (CSQueue childQueue : getChildQueues()) {
|
||||
apps.addAll(((LeafQueue) childQueue).getAllApplications());
|
||||
apps.addAll(((AbstractLeafQueue) childQueue).getAllApplications());
|
||||
}
|
||||
return Collections.unmodifiableList(apps);
|
||||
} finally {
|
||||
|
|
|
@ -675,10 +675,10 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
// parent Queue has been converted to child queue. The CS has already
|
||||
// checked to ensure that this child-queue is in STOPPED state if
|
||||
// Child queue has been converted to ParentQueue.
|
||||
if ((childQueue instanceof LeafQueue
|
||||
if ((childQueue instanceof AbstractLeafQueue
|
||||
&& newChildQueue instanceof ParentQueue)
|
||||
|| (childQueue instanceof ParentQueue
|
||||
&& newChildQueue instanceof LeafQueue)) {
|
||||
&& newChildQueue instanceof AbstractLeafQueue)) {
|
||||
// We would convert this LeafQueue to ParentQueue, or vice versa.
|
||||
// consider this as the combination of DELETE then ADD.
|
||||
newChildQueue.setParent(this);
|
||||
|
@ -1134,7 +1134,7 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
assignment = childAssignment;
|
||||
}
|
||||
Resource blockedHeadroom = null;
|
||||
if (childQueue instanceof LeafQueue) {
|
||||
if (childQueue instanceof AbstractLeafQueue) {
|
||||
blockedHeadroom = childLimits.getHeadroom();
|
||||
} else {
|
||||
blockedHeadroom = childLimits.getBlockedHeadroom();
|
||||
|
@ -1548,7 +1548,7 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
FiCaSchedulerNode node = csContext.getNode(
|
||||
toKillContainer.getAllocatedNode());
|
||||
if (null != attempt && null != node) {
|
||||
LeafQueue lq = attempt.getCSLeafQueue();
|
||||
AbstractLeafQueue lq = attempt.getCSLeafQueue();
|
||||
lq.completedContainer(clusterResource, attempt, node, toKillContainer,
|
||||
SchedulerUtils.createPreemptedContainerStatus(
|
||||
toKillContainer.getContainerId(),
|
||||
|
|
|
@ -40,6 +40,11 @@ public class ReservationQueue extends AbstractAutoCreatedLeafQueue {
|
|||
public ReservationQueue(CapacitySchedulerContext cs, String queueName,
|
||||
PlanQueue parent) throws IOException {
|
||||
super(cs, queueName, parent, null);
|
||||
super.setupQueueConfigs(cs.getClusterResource(),
|
||||
cs.getConfiguration());
|
||||
|
||||
LOG.debug("Initialized ReservationQueue: name={}, fullname={}",
|
||||
queueName, getQueuePath());
|
||||
// the following parameters are common to all reservation in the plan
|
||||
updateQuotas(parent.getUserLimitForReservation(),
|
||||
parent.getUserLimitFactor(),
|
||||
|
|
|
@ -55,7 +55,7 @@ public class UsersManager implements AbstractUsersManager {
|
|||
/*
|
||||
* Member declaration for UsersManager class.
|
||||
*/
|
||||
private final LeafQueue lQueue;
|
||||
private final AbstractLeafQueue lQueue;
|
||||
private final RMNodeLabelsManager labelManager;
|
||||
private final ResourceCalculator resourceCalculator;
|
||||
private final CapacitySchedulerContext scheduler;
|
||||
|
@ -301,7 +301,7 @@ public class UsersManager implements AbstractUsersManager {
|
|||
* @param resourceCalculator
|
||||
* rc
|
||||
*/
|
||||
public UsersManager(QueueMetrics metrics, LeafQueue lQueue,
|
||||
public UsersManager(QueueMetrics metrics, AbstractLeafQueue lQueue,
|
||||
RMNodeLabelsManager labelManager, CapacitySchedulerContext scheduler,
|
||||
ResourceCalculator resourceCalculator) {
|
||||
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||
|
|
|
@ -20,35 +20,23 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
|||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
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.SchedulerDynamicEditException;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
|
||||
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.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 org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
@ -136,7 +124,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|||
return false;
|
||||
}
|
||||
|
||||
public boolean createLeafQueueStateIfNotExists(LeafQueue leafQueue,
|
||||
public boolean createLeafQueueStateIfNotExists(AbstractLeafQueue leafQueue,
|
||||
String partition) {
|
||||
return addLeafQueueStateIfNotExists(leafQueue.getQueuePath(), partition,
|
||||
new LeafQueueStatePerPartition());
|
||||
|
@ -482,9 +470,9 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|||
Set<String> newQueues = new HashSet<>();
|
||||
|
||||
for (CSQueue newQueue : managedParentQueue.getChildQueues()) {
|
||||
if (newQueue instanceof LeafQueue) {
|
||||
if (newQueue instanceof AbstractLeafQueue) {
|
||||
for (String nodeLabel : leafQueueTemplateNodeLabels) {
|
||||
leafQueueState.createLeafQueueStateIfNotExists((LeafQueue) newQueue,
|
||||
leafQueueState.createLeafQueueStateIfNotExists((AbstractLeafQueue) newQueue,
|
||||
nodeLabel);
|
||||
newPartitions.add(nodeLabel);
|
||||
}
|
||||
|
@ -590,7 +578,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|||
if (leafQueue != null) {
|
||||
if (isActive(leafQueue, nodeLabel) && !hasPendingApps(leafQueue)) {
|
||||
QueueCapacities capacities = leafQueueEntitlements.getCapacityOfQueue(leafQueue);
|
||||
updateToZeroCapacity(capacities, nodeLabel, (LeafQueue)childQueue);
|
||||
updateToZeroCapacity(capacities, nodeLabel, (AbstractLeafQueue) childQueue);
|
||||
deactivatedQueues.put(leafQueue.getQueuePath(), leafQueueTemplateCapacities);
|
||||
}
|
||||
} else {
|
||||
|
@ -780,7 +768,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|||
}
|
||||
|
||||
private void updateToZeroCapacity(QueueCapacities capacities,
|
||||
String nodeLabel, LeafQueue leafQueue) {
|
||||
String nodeLabel, AbstractLeafQueue leafQueue) {
|
||||
capacities.setCapacity(nodeLabel, 0.0f);
|
||||
capacities.setMaximumCapacity(nodeLabel,
|
||||
leafQueueTemplateCapacities.getMaximumCapacity(nodeLabel));
|
||||
|
@ -801,7 +789,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|||
}
|
||||
|
||||
@VisibleForTesting
|
||||
LeafQueueStatePerPartition getLeafQueueState(LeafQueue queue,
|
||||
LeafQueueStatePerPartition getLeafQueueState(AbstractLeafQueue queue,
|
||||
String partition) throws SchedulerDynamicEditException {
|
||||
readLock.lock();
|
||||
try {
|
||||
|
|
|
@ -66,11 +66,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerCha
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMContainerLaunchDiagnosticsConstants;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AbstractContainerAllocator;
|
||||
|
@ -958,8 +958,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
}
|
||||
}
|
||||
|
||||
public LeafQueue getCSLeafQueue() {
|
||||
return (LeafQueue)queue;
|
||||
public AbstractLeafQueue getCSLeafQueue() {
|
||||
return (AbstractLeafQueue)queue;
|
||||
}
|
||||
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
|
@ -996,7 +996,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
|
||||
protected void getPendingAppDiagnosticMessage(
|
||||
StringBuilder diagnosticMessage) {
|
||||
LeafQueue queue = getCSLeafQueue();
|
||||
AbstractLeafQueue queue = getCSLeafQueue();
|
||||
diagnosticMessage.append(" Details : AM Partition = ")
|
||||
.append(appAMNodePartitionName.isEmpty()
|
||||
? NodeLabel.DEFAULT_NODE_LABEL_PARTITION : appAMNodePartitionName)
|
||||
|
@ -1019,7 +1019,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
|
||||
protected void getActivedAppDiagnosticMessage(
|
||||
StringBuilder diagnosticMessage) {
|
||||
LeafQueue queue = getCSLeafQueue();
|
||||
AbstractLeafQueue queue = getCSLeafQueue();
|
||||
QueueCapacities queueCapacities = queue.getQueueCapacities();
|
||||
QueueResourceQuotas queueResourceQuotas = queue.getQueueResourceQuotas();
|
||||
diagnosticMessage.append(" Details : AM Partition = ")
|
||||
|
|
|
@ -25,10 +25,10 @@ import javax.xml.bind.annotation.XmlTransient;
|
|||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
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.LeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.helper.CapacitySchedulerInfoHelper;
|
||||
|
||||
|
@ -179,7 +179,7 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
|
|||
List<CSQueue> childLeafQueues = new ArrayList<>();
|
||||
List<CSQueue> childNonLeafQueues = new ArrayList<>();
|
||||
for (CSQueue queue : parent.getChildQueues()) {
|
||||
if (queue instanceof LeafQueue) {
|
||||
if (queue instanceof AbstractLeafQueue) {
|
||||
childLeafQueues.add(queue);
|
||||
} else {
|
||||
childNonLeafQueues.add(queue);
|
||||
|
@ -190,8 +190,8 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
|
|||
|
||||
for (CSQueue queue : childQueues) {
|
||||
CapacitySchedulerQueueInfo info;
|
||||
if (queue instanceof LeafQueue) {
|
||||
info = new CapacitySchedulerLeafQueueInfo(cs, (LeafQueue) queue);
|
||||
if (queue instanceof AbstractLeafQueue) {
|
||||
info = new CapacitySchedulerLeafQueueInfo(cs, (AbstractLeafQueue) queue);
|
||||
} else {
|
||||
info = new CapacitySchedulerQueueInfo(cs, queue);
|
||||
info.queues = getQueues(cs, queue);
|
||||
|
|
|
@ -27,10 +27,10 @@ import javax.xml.bind.annotation.XmlTransient;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
||||
.AutoCreatedLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UserInfo;
|
||||
|
||||
|
@ -63,7 +63,7 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
|
|||
CapacitySchedulerLeafQueueInfo() {
|
||||
};
|
||||
|
||||
CapacitySchedulerLeafQueueInfo(CapacityScheduler cs, LeafQueue q) {
|
||||
CapacitySchedulerLeafQueueInfo(CapacityScheduler cs, AbstractLeafQueue q) {
|
||||
super(cs, q);
|
||||
numActiveApplications = q.getNumActiveApplications();
|
||||
numPendingApplications = q.getNumPendingApplications();
|
||||
|
|
|
@ -18,9 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.helper;
|
|||
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
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.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.webapp.dao.AutoQueueTemplatePropertiesInfo;
|
||||
|
@ -82,7 +82,7 @@ public class CapacitySchedulerInfoHelper {
|
|||
}
|
||||
|
||||
public static String getQueueType(CSQueue queue) {
|
||||
if (queue instanceof LeafQueue) {
|
||||
if (queue instanceof AbstractLeafQueue) {
|
||||
return LEAF_QUEUE;
|
||||
} else if (queue instanceof ParentQueue) {
|
||||
return PARENT_QUEUE;
|
||||
|
|
|
@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicat
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
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.LeafQueue;
|
||||
|
@ -445,7 +446,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
|
|||
checkCSLeafQueue(rm, app, clusterResource, queueResource, usedResource,
|
||||
numContainers);
|
||||
|
||||
LeafQueue queue = (LeafQueue) app.getQueue();
|
||||
AbstractLeafQueue queue = (AbstractLeafQueue) app.getQueue();
|
||||
Resource availableResources =
|
||||
Resources.subtract(queueResource, usedResource);
|
||||
// ************ check app headroom ****************
|
||||
|
@ -470,7 +471,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
|
|||
SchedulerApplication<SchedulerApplicationAttempt> app,
|
||||
Resource clusterResource, Resource queueResource, Resource usedResource,
|
||||
int numContainers) {
|
||||
LeafQueue leafQueue = (LeafQueue) app.getQueue();
|
||||
AbstractLeafQueue leafQueue = (AbstractLeafQueue) app.getQueue();
|
||||
// assert queue used resources.
|
||||
assertEquals(usedResource, leafQueue.getUsedResources());
|
||||
assertEquals(numContainers, leafQueue.getNumContainers());
|
||||
|
|
|
@ -139,7 +139,7 @@ public class TestCSMaxRunningAppsEnforcer {
|
|||
}
|
||||
|
||||
private void removeApp(FiCaSchedulerApp attempt) {
|
||||
LeafQueue queue = attempt.getCSLeafQueue();
|
||||
AbstractLeafQueue queue = attempt.getCSLeafQueue();
|
||||
queue.finishApplicationAttempt(attempt, queue.getQueuePath());
|
||||
maxAppsEnforcer.untrackApp(attempt);
|
||||
maxAppsEnforcer.updateRunnabilityOnAppRemoval(attempt);
|
||||
|
|
|
@ -685,7 +685,7 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|||
"root.a.*") + "capacity", "6w");
|
||||
cs.reinitialize(csConf, mockRM.getRMContext());
|
||||
|
||||
LeafQueue a2 = createQueue("root.a.a-auto.a2");
|
||||
AbstractLeafQueue a2 = createQueue("root.a.a-auto.a2");
|
||||
Assert.assertEquals("weight is not set by template", 6f,
|
||||
a2.getQueueCapacities().getWeight(), 1e-6);
|
||||
Assert.assertEquals("user limit factor should be disabled with dynamic queues",
|
||||
|
@ -719,7 +719,7 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|||
"root.a") + CapacitySchedulerConfiguration
|
||||
.AUTO_CREATE_CHILD_QUEUE_AUTO_REMOVAL_ENABLE, false);
|
||||
cs.reinitialize(csConf, mockRM.getRMContext());
|
||||
LeafQueue a3 = createQueue("root.a.a3");
|
||||
AbstractLeafQueue a3 = createQueue("root.a.a3");
|
||||
Assert.assertFalse("auto queue deletion should be turned off on a3",
|
||||
a3.isEligibleForAutoDeletion());
|
||||
|
||||
|
@ -729,27 +729,26 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|||
csConf.setQueues("root", new String[]{"a", "b", "c"});
|
||||
csConf.setAutoQueueCreationV2Enabled("root.c", true);
|
||||
cs.reinitialize(csConf, mockRM.getRMContext());
|
||||
LeafQueue c1 = createQueue("root.c.c1");
|
||||
AbstractLeafQueue c1 = createQueue("root.c.c1");
|
||||
Assert.assertEquals("weight is not set for label TEST", 6f,
|
||||
c1.getQueueCapacities().getWeight("TEST"), 1e-6);
|
||||
cs.reinitialize(csConf, mockRM.getRMContext());
|
||||
c1 = (LeafQueue) cs.getQueue("root.c.c1");
|
||||
c1 = (AbstractLeafQueue) cs.getQueue("root.c.c1");
|
||||
Assert.assertEquals("weight is not set for label TEST", 6f,
|
||||
c1.getQueueCapacities().getWeight("TEST"), 1e-6);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoCreatedQueueConfigChange() throws Exception {
|
||||
startScheduler();
|
||||
LeafQueue a2 = createQueue("root.a.a-auto.a2");
|
||||
AbstractLeafQueue a2 = createQueue("root.a.a-auto.a2");
|
||||
csConf.setNonLabeledQueueWeight("root.a.a-auto.a2", 4f);
|
||||
cs.reinitialize(csConf, mockRM.getRMContext());
|
||||
|
||||
Assert.assertEquals("weight is not explicitly set", 4f,
|
||||
a2.getQueueCapacities().getWeight(), 1e-6);
|
||||
|
||||
a2 = (LeafQueue) cs.getQueue("root.a.a-auto.a2");
|
||||
a2 = (AbstractLeafQueue) cs.getQueue("root.a.a-auto.a2");
|
||||
csConf.setState("root.a.a-auto.a2", QueueState.STOPPED);
|
||||
cs.reinitialize(csConf, mockRM.getRMContext());
|
||||
Assert.assertEquals("root.a.a-auto.a2 has not been stopped",
|
||||
|
@ -1223,7 +1222,7 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|||
Assert.assertNull("root.e.e1-auto should have been removed", eAuto);
|
||||
}
|
||||
|
||||
protected LeafQueue createQueue(String queuePath) throws YarnException,
|
||||
protected AbstractLeafQueue createQueue(String queuePath) throws YarnException,
|
||||
IOException {
|
||||
return autoQueueHandler.createQueue(new QueuePath(queuePath));
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue