YARN-11079. Make an AbstractParentQueue to store common ParentQueue and ManagedParentQueue functionality. Contributed by Susheel Gupta
This commit is contained in:
parent
ceb8878d4f
commit
c7699d3dcd
|
@ -21,6 +21,7 @@ import org.apache.commons.collections.CollectionUtils;
|
|||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractParentQueue;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
|
@ -41,7 +42,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
|
|||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
||||
.ManagedParentQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptableQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
|
||||
|
@ -630,9 +630,9 @@ public class ProportionalCapacityPreemptionPolicy
|
|||
partitionToLookAt, killable, absCap, absMaxCap, partitionResource,
|
||||
reserved, curQueue, effMinRes, effMaxRes);
|
||||
|
||||
if (curQueue instanceof ParentQueue) {
|
||||
if (curQueue instanceof AbstractParentQueue) {
|
||||
String configuredOrderingPolicy =
|
||||
((ParentQueue) curQueue).getQueueOrderingPolicy().getConfigName();
|
||||
((AbstractParentQueue) curQueue).getQueueOrderingPolicy().getConfigName();
|
||||
|
||||
// Recursively add children
|
||||
for (CSQueue c : curQueue.getChildQueues()) {
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.LinkedHashMap;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractParentQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
|
||||
|
@ -57,7 +58,7 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
|
|||
final ArrayList<TempQueuePerPartition> children;
|
||||
private Collection<TempAppPerPartition> apps;
|
||||
AbstractLeafQueue leafQueue;
|
||||
ParentQueue parentQueue;
|
||||
AbstractParentQueue parentQueue;
|
||||
boolean preemptionDisabled;
|
||||
|
||||
protected Resource pendingDeductReserved;
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.placement.csmappingrule;
|
||||
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractParentQueue;
|
||||
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.ManagedParentQueue;
|
||||
|
@ -123,8 +124,8 @@ public final class MappingRuleValidationHelper {
|
|||
//if the grandparent allows new dynamic creation, the dynamic parent and
|
||||
//the dynamic leaf queue can be created as well
|
||||
CSQueue grandParentQueue = queueManager.getQueue(grandParentPath);
|
||||
if (grandParentQueue != null && grandParentQueue instanceof ParentQueue &&
|
||||
((ParentQueue)grandParentQueue).isEligibleForAutoQueueCreation()) {
|
||||
if (grandParentQueue != null && grandParentQueue instanceof AbstractParentQueue &&
|
||||
((AbstractParentQueue)grandParentQueue).isEligibleForAutoQueueCreation()) {
|
||||
//Grandparent is a new dynamic parent queue, which allows deep queue
|
||||
//creation
|
||||
return ValidationResult.CREATABLE;
|
||||
|
|
|
@ -410,8 +410,8 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|||
*/
|
||||
protected void setDynamicQueueProperties() {
|
||||
// Set properties from parent template
|
||||
if (parent instanceof ParentQueue) {
|
||||
((ParentQueue) parent).getAutoCreatedQueueTemplate()
|
||||
if (parent instanceof AbstractParentQueue) {
|
||||
((AbstractParentQueue) parent).getAutoCreatedQueueTemplate()
|
||||
.setTemplateEntriesForChild(queueContext.getConfiguration(), getQueuePath());
|
||||
|
||||
String parentTemplate = String.format("%s.%s", parent.getQueuePath(),
|
||||
|
@ -1262,7 +1262,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|||
CapacityConfigType.ABSOLUTE_RESOURCE)) {
|
||||
newEffectiveMinResource = createNormalizedMinResource(
|
||||
usageTracker.getQueueResourceQuotas().getConfiguredMinResource(label),
|
||||
((ParentQueue) parent).getEffectiveMinRatio(label));
|
||||
((AbstractParentQueue) parent).getEffectiveMinRatio(label));
|
||||
|
||||
// Max resource of a queue should be the minimum of {parent's maxResources,
|
||||
// this queue's maxResources}. Both parent's maxResources and this queue's
|
||||
|
|
|
@ -35,7 +35,7 @@ import java.util.Map;
|
|||
* From the user perspective this is equivalent to a LeafQueue,
|
||||
* but functionality wise is a sub-class of ParentQueue
|
||||
*/
|
||||
public abstract class AbstractManagedParentQueue extends ParentQueue {
|
||||
public abstract class AbstractManagedParentQueue extends AbstractParentQueue {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
AbstractManagedParentQueue.class);
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -30,14 +30,14 @@ public interface AutoCreatedQueueManagementPolicy {
|
|||
* @param parentQueue parent queue
|
||||
* @throws IOException an I/O exception has occurred.
|
||||
*/
|
||||
void init(ParentQueue parentQueue) throws IOException;
|
||||
void init(AbstractParentQueue parentQueue) throws IOException;
|
||||
|
||||
/**
|
||||
* Reinitialize policy state ( if required ).
|
||||
* @param parentQueue parent queue
|
||||
* @throws IOException an I/O exception has occurred.
|
||||
*/
|
||||
void reinitialize(ParentQueue parentQueue) throws IOException;
|
||||
void reinitialize(AbstractParentQueue parentQueue) throws IOException;
|
||||
|
||||
/**
|
||||
* Get initial template for the specified leaf queue.
|
||||
|
|
|
@ -125,10 +125,10 @@ public class CSMaxRunningAppsEnforcer {
|
|||
String user = app.getUser();
|
||||
AbstractCSQueue queue = (AbstractCSQueue) app.getQueue();
|
||||
// Increment running counts for all parent queues
|
||||
ParentQueue parent = (ParentQueue) queue.getParent();
|
||||
AbstractParentQueue parent = (AbstractParentQueue) queue.getParent();
|
||||
while (parent != null) {
|
||||
parent.incrementRunnableApps();
|
||||
parent = (ParentQueue) parent.getParent();
|
||||
parent = (AbstractParentQueue) parent.getParent();
|
||||
}
|
||||
|
||||
Integer userNumRunnable = usersNumRunnableApps.get(user);
|
||||
|
@ -189,12 +189,12 @@ public class CSMaxRunningAppsEnforcer {
|
|||
(queue.getNumRunnableApps() == queue.getMaxParallelApps() - 1)
|
||||
? queue : null;
|
||||
|
||||
ParentQueue parent = (ParentQueue) queue.getParent();
|
||||
AbstractParentQueue parent = (AbstractParentQueue) queue.getParent();
|
||||
while (parent != null) {
|
||||
if (parent.getNumRunnableApps() == parent.getMaxParallelApps() - 1) {
|
||||
highestQueueWithAppsNowRunnable = parent;
|
||||
}
|
||||
parent = (ParentQueue) parent.getParent();
|
||||
parent = (AbstractParentQueue) parent.getParent();
|
||||
}
|
||||
|
||||
List<List<FiCaSchedulerApp>> appsNowMaybeRunnable =
|
||||
|
@ -303,10 +303,10 @@ public class CSMaxRunningAppsEnforcer {
|
|||
|
||||
// Update runnable app bookkeeping for queues
|
||||
AbstractCSQueue queue = (AbstractCSQueue) app.getQueue();
|
||||
ParentQueue parent = (ParentQueue) queue.getParent();
|
||||
AbstractParentQueue parent = (AbstractParentQueue) queue.getParent();
|
||||
while (parent != null) {
|
||||
parent.decrementRunnableApps();
|
||||
parent = (ParentQueue) parent.getParent();
|
||||
parent = (AbstractParentQueue) parent.getParent();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -2097,7 +2097,7 @@ public class CapacityScheduler extends
|
|||
{
|
||||
QueueManagementChangeEvent queueManagementChangeEvent =
|
||||
(QueueManagementChangeEvent) event;
|
||||
ParentQueue parentQueue = queueManagementChangeEvent.getParentQueue();
|
||||
AbstractParentQueue parentQueue = queueManagementChangeEvent.getParentQueue();
|
||||
try {
|
||||
final List<QueueManagementChange> queueManagementChanges =
|
||||
queueManagementChangeEvent.getQueueManagementChanges();
|
||||
|
|
|
@ -172,7 +172,7 @@ public final class CapacitySchedulerConfigValidator {
|
|||
|
||||
private static void validateParentQueueConversion(CSQueue oldQueue,
|
||||
CSQueue newQueue) throws IOException {
|
||||
if (oldQueue instanceof ParentQueue) {
|
||||
if (oldQueue instanceof AbstractParentQueue) {
|
||||
if (!(oldQueue instanceof ManagedParentQueue) && newQueue instanceof ManagedParentQueue) {
|
||||
throw new IOException(
|
||||
"Can not convert parent queue: " + oldQueue.getQueuePath()
|
||||
|
@ -199,7 +199,7 @@ public final class CapacitySchedulerConfigValidator {
|
|||
|
||||
private static void validateLeafQueueConversion(CSQueue oldQueue,
|
||||
CSQueue newQueue) throws IOException {
|
||||
if (oldQueue instanceof AbstractLeafQueue && newQueue instanceof ParentQueue) {
|
||||
if (oldQueue instanceof AbstractLeafQueue && newQueue instanceof AbstractParentQueue) {
|
||||
if (isEitherQueueStopped(oldQueue.getState(), newQueue.getState())) {
|
||||
LOG.info("Converting the leaf queue: {} to parent queue.", oldQueue.getQueuePath());
|
||||
} else {
|
||||
|
|
|
@ -249,7 +249,8 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|||
boolean isAutoCreateEnabled = conf.isAutoCreateChildQueueEnabled(fullQueueName);
|
||||
// if a queue is eligible for auto queue creation v2 it must be a ParentQueue
|
||||
// (even if it is empty)
|
||||
final boolean isDynamicParent = oldQueue instanceof ParentQueue && oldQueue.isDynamicQueue();
|
||||
final boolean isDynamicParent = oldQueue instanceof AbstractParentQueue &&
|
||||
oldQueue.isDynamicQueue();
|
||||
boolean isAutoQueueCreationEnabledParent = isDynamicParent || conf.isAutoQueueCreationV2Enabled(
|
||||
fullQueueName) || isAutoCreateEnabled;
|
||||
|
||||
|
@ -270,7 +271,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|||
throw new IllegalStateException("Only Leaf Queues can be reservable for " + fullQueueName);
|
||||
}
|
||||
|
||||
ParentQueue parentQueue;
|
||||
AbstractParentQueue parentQueue;
|
||||
if (isAutoCreateEnabled) {
|
||||
parentQueue = new ManagedParentQueue(queueContext, queueName, parent, oldQueues.get(
|
||||
fullQueueName));
|
||||
|
@ -591,7 +592,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|||
"above the limit.");
|
||||
}
|
||||
|
||||
if (!(firstExistingParent instanceof ParentQueue)) {
|
||||
if (!(firstExistingParent instanceof AbstractParentQueue)) {
|
||||
throw new SchedulerDynamicEditException(
|
||||
"Could not auto create hierarchy of "
|
||||
+ queue.getFullPath() + ". Queue " + queue.getParent() +
|
||||
|
@ -599,7 +600,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|||
);
|
||||
}
|
||||
|
||||
ParentQueue existingParentQueue = (ParentQueue) firstExistingParent;
|
||||
AbstractParentQueue existingParentQueue = (AbstractParentQueue) firstExistingParent;
|
||||
|
||||
if (!existingParentQueue.isEligibleForAutoQueueCreation()) {
|
||||
throw new SchedulerDynamicEditException("Auto creation of queue " +
|
||||
|
|
|
@ -56,7 +56,7 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
|
|||
final String queueName, final CSQueue parent, final CSQueue old)
|
||||
throws IOException {
|
||||
super(queueContext, queueName, parent, old);
|
||||
|
||||
super.setupQueueConfigs(queueContext.getClusterResource());
|
||||
shouldFailAutoCreationWhenGuaranteedCapacityExceeded =
|
||||
queueContext.getConfiguration()
|
||||
.getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -47,6 +47,7 @@ public class PlanQueue extends AbstractManagedParentQueue {
|
|||
public PlanQueue(CapacitySchedulerQueueContext queueContext, String queueName,
|
||||
CSQueue parent, CSQueue old) throws IOException {
|
||||
super(queueContext, queueName, parent, old);
|
||||
super.setupQueueConfigs(queueContext.getClusterResource());
|
||||
updateAbsoluteCapacities();
|
||||
|
||||
// Set the reservation queue attributes for the Plan
|
||||
|
|
|
@ -20,6 +20,7 @@ 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.AbstractParentQueue;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
|
||||
|
@ -31,7 +32,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCrea
|
|||
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.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;
|
||||
|
@ -251,7 +251,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|||
}
|
||||
|
||||
@Override
|
||||
public void init(final ParentQueue parentQueue) throws IOException {
|
||||
public void init(final AbstractParentQueue parentQueue) throws IOException {
|
||||
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||
readLock = lock.readLock();
|
||||
writeLock = lock.writeLock();
|
||||
|
@ -533,7 +533,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|||
* queues
|
||||
*/
|
||||
private Map<String, QueueCapacities> deactivateLeafQueuesIfInActive(
|
||||
ParentQueue parentQueue, String nodeLabel,
|
||||
AbstractParentQueue parentQueue, String nodeLabel,
|
||||
LeafQueueEntitlements leafQueueEntitlements)
|
||||
throws SchedulerDynamicEditException {
|
||||
Map<String, QueueCapacities> deactivatedQueues = new HashMap<>();
|
||||
|
@ -658,7 +658,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|||
}
|
||||
|
||||
@Override
|
||||
public void reinitialize(final ParentQueue parentQueue) throws IOException {
|
||||
public void reinitialize(final AbstractParentQueue parentQueue) throws IOException {
|
||||
if (!(parentQueue instanceof ManagedParentQueue)) {
|
||||
throw new IllegalStateException(
|
||||
"Expected instance of type " + ManagedParentQueue.class + " found "
|
||||
|
|
|
@ -17,8 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
||||
.ParentQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractParentQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
||||
.QueueManagementChange;
|
||||
|
||||
|
@ -29,17 +28,17 @@ import java.util.List;
|
|||
*/
|
||||
public class QueueManagementChangeEvent extends SchedulerEvent {
|
||||
|
||||
private ParentQueue parentQueue;
|
||||
private AbstractParentQueue parentQueue;
|
||||
private List<QueueManagementChange> queueManagementChanges;
|
||||
|
||||
public QueueManagementChangeEvent(ParentQueue parentQueue,
|
||||
public QueueManagementChangeEvent(AbstractParentQueue parentQueue,
|
||||
List<QueueManagementChange> queueManagementChanges) {
|
||||
super(SchedulerEventType.MANAGE_QUEUE);
|
||||
this.parentQueue = parentQueue;
|
||||
this.queueManagementChanges = queueManagementChanges;
|
||||
}
|
||||
|
||||
public ParentQueue getParentQueue() {
|
||||
public AbstractParentQueue getParentQueue() {
|
||||
return parentQueue;
|
||||
}
|
||||
|
||||
|
|
|
@ -26,10 +26,10 @@ 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.AbstractParentQueue;
|
||||
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.ParentQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.helper.CapacitySchedulerInfoHelper;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -100,8 +100,8 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
|
|||
queueAcls.addAll(getSortedQueueAclInfoList(parent, queueName, conf));
|
||||
|
||||
queuePriority = parent.getPriority().getPriority();
|
||||
if (parent instanceof ParentQueue) {
|
||||
ParentQueue queue = (ParentQueue) parent;
|
||||
if (parent instanceof AbstractParentQueue) {
|
||||
AbstractParentQueue queue = (AbstractParentQueue) parent;
|
||||
orderingPolicyInfo = queue.getQueueOrderingPolicy()
|
||||
.getConfigName();
|
||||
autoQueueTemplateProperties = CapacitySchedulerInfoHelper
|
||||
|
|
|
@ -36,10 +36,10 @@ import org.apache.hadoop.yarn.security.AccessType;
|
|||
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.AbstractCSQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractParentQueue;
|
||||
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.CSQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.helper.CapacitySchedulerInfoHelper;
|
||||
|
@ -160,8 +160,8 @@ public class CapacitySchedulerQueueInfo {
|
|||
queueAcls.addAll(getSortedQueueAclInfoList(q, queuePath, conf));
|
||||
|
||||
queuePriority = q.getPriority().getPriority();
|
||||
if (q instanceof ParentQueue) {
|
||||
ParentQueue queue = (ParentQueue) q;
|
||||
if (q instanceof AbstractParentQueue) {
|
||||
AbstractParentQueue queue = (AbstractParentQueue) q;
|
||||
orderingPolicyInfo = queue.getQueueOrderingPolicy()
|
||||
.getConfigName();
|
||||
autoQueueTemplateProperties = CapacitySchedulerInfoHelper
|
||||
|
|
|
@ -19,6 +19,7 @@ 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.AbstractParentQueue;
|
||||
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.ManagedParentQueue;
|
||||
|
@ -84,7 +85,7 @@ public class CapacitySchedulerInfoHelper {
|
|||
public static String getQueueType(CSQueue queue) {
|
||||
if (queue instanceof AbstractLeafQueue) {
|
||||
return LEAF_QUEUE;
|
||||
} else if (queue instanceof ParentQueue) {
|
||||
} else if (queue instanceof AbstractParentQueue) {
|
||||
return PARENT_QUEUE;
|
||||
}
|
||||
return UNKNOWN_QUEUE;
|
||||
|
|
|
@ -517,12 +517,12 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|||
|
||||
empty = cs.getQueue("root.empty-auto-parent");
|
||||
Assert.assertTrue("empty-auto-parent is not a ParentQueue",
|
||||
empty instanceof ParentQueue);
|
||||
empty instanceof AbstractParentQueue);
|
||||
Assert.assertEquals("empty-auto-parent has children",
|
||||
0, empty.getChildQueues().size());
|
||||
Assert.assertTrue("empty-auto-parent is not eligible " +
|
||||
"for auto queue creation",
|
||||
((ParentQueue)empty).isEligibleForAutoQueueCreation());
|
||||
((AbstractParentQueue)empty).isEligibleForAutoQueueCreation());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -712,7 +712,7 @@ public class TestCapacitySchedulerQueues {
|
|||
+ " to convert a leaf queue WITHOUT running apps");
|
||||
}
|
||||
b1 = cs.getQueue(targetQueue);
|
||||
Assert.assertTrue(b1 instanceof ParentQueue);
|
||||
Assert.assertTrue(b1 instanceof AbstractParentQueue);
|
||||
Assert.assertEquals(QueueState.RUNNING, b1.getState());
|
||||
Assert.assertTrue(!b1.getChildQueues().isEmpty());
|
||||
}
|
||||
|
|
|
@ -137,8 +137,8 @@ public class TestChildQueueOrder {
|
|||
" alloc=" + allocation + " node=" + node.getNodeName());
|
||||
}
|
||||
final Resource allocatedResource = Resources.createResource(allocation);
|
||||
if (queue instanceof ParentQueue) {
|
||||
((ParentQueue)queue).allocateResource(clusterResource,
|
||||
if (queue instanceof AbstractParentQueue) {
|
||||
((AbstractParentQueue)queue).allocateResource(clusterResource,
|
||||
allocatedResource, RMNodeLabelsManager.NO_LABEL);
|
||||
} else {
|
||||
FiCaSchedulerApp app1 = getMockApplication(0, "");
|
||||
|
|
|
@ -199,8 +199,8 @@ public class TestParentQueue {
|
|||
" alloc=" + allocation + " node=" + node.getNodeName());
|
||||
}
|
||||
final Resource allocatedResource = Resources.createResource(allocation);
|
||||
if (queue instanceof ParentQueue) {
|
||||
((ParentQueue)queue).allocateResource(clusterResource,
|
||||
if (queue instanceof AbstractParentQueue) {
|
||||
((AbstractParentQueue)queue).allocateResource(clusterResource,
|
||||
allocatedResource, RMNodeLabelsManager.NO_LABEL);
|
||||
} else {
|
||||
FiCaSchedulerApp app1 = getMockApplication(0, "");
|
||||
|
|
Loading…
Reference in New Issue