Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
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;
Expand All @@ -41,7 +42,6 @@

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;
Expand Down Expand Up @@ -630,9 +630,9 @@ private TempQueuePerPartition cloneQueues(CSQueue curQueue,
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()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
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;
Expand Down Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -123,8 +124,8 @@ public static ValidationResult validateQueuePathAutoCreation(
//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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -410,8 +410,8 @@ protected void setupQueueConfigs(Resource clusterResource) throws
*/
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(),
Expand Down Expand Up @@ -1262,7 +1262,7 @@ void updateEffectiveResources(Resource clusterResource) {
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
* 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);
Expand Down
Loading