|
@@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
|
|
|
import org.apache.hadoop.yarn.util.Clock;
|
|
|
import org.apache.hadoop.yarn.util.MonotonicClock;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Comparator;
|
|
@@ -63,8 +64,6 @@ import java.util.Set;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
-import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
|
|
|
- .NO_LABEL;
|
|
|
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
|
|
|
.capacity.CSQueueUtils.EPSILON;
|
|
|
|
|
@@ -85,8 +84,6 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
private static final Log LOG = LogFactory.getLog(
|
|
|
GuaranteedOrZeroCapacityOverTimePolicy.class);
|
|
|
|
|
|
- private AutoCreatedLeafQueueConfig ZERO_CAPACITY_ENTITLEMENT;
|
|
|
-
|
|
|
private ReentrantReadWriteLock.WriteLock writeLock;
|
|
|
|
|
|
private ReentrantReadWriteLock.ReadLock readLock;
|
|
@@ -97,12 +94,70 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
|
|
|
private QueueCapacities leafQueueTemplateCapacities;
|
|
|
|
|
|
- private Map<String, LeafQueueState> leafQueueStateMap = new HashMap<>();
|
|
|
+ private Set<String> leafQueueTemplateNodeLabels;
|
|
|
+
|
|
|
+ private LeafQueueState leafQueueState = new LeafQueueState();
|
|
|
|
|
|
private Clock clock = new MonotonicClock();
|
|
|
|
|
|
private class LeafQueueState {
|
|
|
|
|
|
+ //map of partition-> queueName->{leaf queue's state}
|
|
|
+ private Map<String, Map<String, LeafQueueStatePerPartition>>
|
|
|
+ leafQueueStateMap = new HashMap<>();
|
|
|
+
|
|
|
+ public boolean containsLeafQueue(String leafQueueName, String partition) {
|
|
|
+ if (leafQueueStateMap.containsKey(partition)) {
|
|
|
+ return leafQueueStateMap.get(partition).containsKey(leafQueueName);
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean containsPartition(String partition) {
|
|
|
+ if (leafQueueStateMap.containsKey(partition)) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean addLeafQueueStateIfNotExists(String leafQueueName,
|
|
|
+ String partition, LeafQueueStatePerPartition leafQueueState) {
|
|
|
+ if (!containsPartition(partition)) {
|
|
|
+ leafQueueStateMap.put(partition, new HashMap<>());
|
|
|
+ }
|
|
|
+ if (!containsLeafQueue(leafQueueName, partition)) {
|
|
|
+ leafQueueStateMap.get(partition).put(leafQueueName, leafQueueState);
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ public boolean createLeafQueueStateIfNotExists(LeafQueue leafQueue,
|
|
|
+ String partition) {
|
|
|
+ return addLeafQueueStateIfNotExists(leafQueue.getQueueName(), partition,
|
|
|
+ new LeafQueueStatePerPartition());
|
|
|
+ }
|
|
|
+
|
|
|
+ public LeafQueueStatePerPartition getLeafQueueStatePerPartition(
|
|
|
+ String leafQueueName, String partition) {
|
|
|
+ if (leafQueueStateMap.get(partition) != null) {
|
|
|
+ return leafQueueStateMap.get(partition).get(leafQueueName);
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Map<String, Map<String, LeafQueueStatePerPartition>>
|
|
|
+ getLeafQueueStateMap() {
|
|
|
+ return leafQueueStateMap;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void clear() {
|
|
|
+ leafQueueStateMap.clear();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private class LeafQueueStatePerPartition {
|
|
|
+
|
|
|
private AtomicBoolean isActive = new AtomicBoolean(false);
|
|
|
|
|
|
private long mostRecentActivationTime;
|
|
@@ -139,41 +194,16 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private boolean containsLeafQueue(String leafQueueName) {
|
|
|
- return leafQueueStateMap.containsKey(leafQueueName);
|
|
|
- }
|
|
|
-
|
|
|
- private boolean addLeafQueueStateIfNotExists(String leafQueueName,
|
|
|
- LeafQueueState leafQueueState) {
|
|
|
- if (!containsLeafQueue(leafQueueName)) {
|
|
|
- leafQueueStateMap.put(leafQueueName, leafQueueState);
|
|
|
- return true;
|
|
|
- }
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- private boolean addLeafQueueStateIfNotExists(LeafQueue leafQueue) {
|
|
|
- return addLeafQueueStateIfNotExists(leafQueue.getQueueName(),
|
|
|
- new LeafQueueState());
|
|
|
- }
|
|
|
-
|
|
|
- private void clearLeafQueueState() {
|
|
|
- leafQueueStateMap.clear();
|
|
|
- }
|
|
|
-
|
|
|
private class ParentQueueState {
|
|
|
|
|
|
private Map<String, Float> totalAbsoluteActivatedChildQueueCapacityByLabel =
|
|
|
new HashMap<String, Float>();
|
|
|
|
|
|
- private float getAbsoluteActivatedChildQueueCapacity() {
|
|
|
- return getAbsoluteActivatedChildQueueCapacity(NO_LABEL);
|
|
|
- }
|
|
|
-
|
|
|
private float getAbsoluteActivatedChildQueueCapacity(String nodeLabel) {
|
|
|
try {
|
|
|
readLock.lock();
|
|
|
- Float totalActivatedCapacity = getByLabel(nodeLabel);
|
|
|
+ Float totalActivatedCapacity = getAbsActivatedChildQueueCapacityByLabel(
|
|
|
+ nodeLabel);
|
|
|
if (totalActivatedCapacity != null) {
|
|
|
return totalActivatedCapacity;
|
|
|
} else{
|
|
@@ -188,11 +218,14 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
float childQueueCapacity) {
|
|
|
try {
|
|
|
writeLock.lock();
|
|
|
- Float activatedChildCapacity = getByLabel(nodeLabel);
|
|
|
+ Float activatedChildCapacity = getAbsActivatedChildQueueCapacityByLabel(
|
|
|
+ nodeLabel);
|
|
|
if (activatedChildCapacity != null) {
|
|
|
- setByLabel(nodeLabel, activatedChildCapacity + childQueueCapacity);
|
|
|
+ setAbsActivatedChildQueueCapacityByLabel(nodeLabel,
|
|
|
+ activatedChildCapacity + childQueueCapacity);
|
|
|
} else{
|
|
|
- setByLabel(nodeLabel, childQueueCapacity);
|
|
|
+ setAbsActivatedChildQueueCapacityByLabel(nodeLabel,
|
|
|
+ childQueueCapacity);
|
|
|
}
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
@@ -203,22 +236,25 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
float childQueueCapacity) {
|
|
|
try {
|
|
|
writeLock.lock();
|
|
|
- Float activatedChildCapacity = getByLabel(nodeLabel);
|
|
|
+ Float activatedChildCapacity = getAbsActivatedChildQueueCapacityByLabel(
|
|
|
+ nodeLabel);
|
|
|
if (activatedChildCapacity != null) {
|
|
|
- setByLabel(nodeLabel, activatedChildCapacity - childQueueCapacity);
|
|
|
+ setAbsActivatedChildQueueCapacityByLabel(nodeLabel,
|
|
|
+ activatedChildCapacity - childQueueCapacity);
|
|
|
} else{
|
|
|
- setByLabel(nodeLabel, childQueueCapacity);
|
|
|
+ setAbsActivatedChildQueueCapacityByLabel(nodeLabel,
|
|
|
+ childQueueCapacity);
|
|
|
}
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- Float getByLabel(String label) {
|
|
|
+ Float getAbsActivatedChildQueueCapacityByLabel(String label) {
|
|
|
return totalAbsoluteActivatedChildQueueCapacityByLabel.get(label);
|
|
|
}
|
|
|
|
|
|
- Float setByLabel(String label, float val) {
|
|
|
+ Float setAbsActivatedChildQueueCapacityByLabel(String label, float val) {
|
|
|
return totalAbsoluteActivatedChildQueueCapacityByLabel.put(label, val);
|
|
|
}
|
|
|
|
|
@@ -256,13 +292,12 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
|
|
|
@Override
|
|
|
public void init(final CapacitySchedulerContext schedulerContext,
|
|
|
- final ParentQueue parentQueue) {
|
|
|
+ final ParentQueue parentQueue) throws IOException {
|
|
|
this.scheduler = schedulerContext;
|
|
|
|
|
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
|
readLock = lock.readLock();
|
|
|
writeLock = lock.writeLock();
|
|
|
-
|
|
|
if (!(parentQueue instanceof ManagedParentQueue)) {
|
|
|
throw new IllegalArgumentException(
|
|
|
"Expected instance of type " + ManagedParentQueue.class);
|
|
@@ -278,15 +313,43 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
+ leafQueueTemplate.getQueueCapacities() + "]");
|
|
|
}
|
|
|
|
|
|
- private void initializeLeafQueueTemplate(ManagedParentQueue parentQueue) {
|
|
|
+ private void initializeLeafQueueTemplate(ManagedParentQueue parentQueue)
|
|
|
+ throws IOException {
|
|
|
leafQueueTemplate = parentQueue.getLeafQueueTemplate();
|
|
|
|
|
|
leafQueueTemplateCapacities = leafQueueTemplate.getQueueCapacities();
|
|
|
|
|
|
- ZERO_CAPACITY_ENTITLEMENT = buildTemplate(0.0f,
|
|
|
- leafQueueTemplateCapacities.getMaximumCapacity());
|
|
|
+ Set<String> parentQueueLabels = parentQueue.getNodeLabelsForQueue();
|
|
|
+ for (String nodeLabel : leafQueueTemplateCapacities
|
|
|
+ .getExistingNodeLabels()) {
|
|
|
+
|
|
|
+ if (!parentQueueLabels.contains(nodeLabel)) {
|
|
|
+ LOG.error("Invalid node label " + nodeLabel
|
|
|
+ + " on configured leaf template on parent" + " queue " + parentQueue
|
|
|
+ .getQueueName());
|
|
|
+ throw new IOException("Invalid node label " + nodeLabel
|
|
|
+ + " on configured leaf template on parent" + " queue " + parentQueue
|
|
|
+ .getQueueName());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ leafQueueTemplateNodeLabels =
|
|
|
+ leafQueueTemplateCapacities.getExistingNodeLabels();
|
|
|
+
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Compute/Adjust child queue capacities
|
|
|
+ * for auto created leaf queues
|
|
|
+ * This computes queue entitlements but does not update LeafQueueState or
|
|
|
+ * queue capacities. Scheduler calls commitQueueManagemetChanges after
|
|
|
+ * validation after applying queue changes and commits to LeafQueueState
|
|
|
+ * are done in commitQueueManagementChanges.
|
|
|
+ *
|
|
|
+ * @return List of Queue Management change suggestions which could potentially
|
|
|
+ * be committed/rejected by the scheduler due to validation failures
|
|
|
+ * @throws SchedulerDynamicEditException
|
|
|
+ */
|
|
|
@Override
|
|
|
public List<QueueManagementChange> computeQueueManagementChanges()
|
|
|
throws SchedulerDynamicEditException {
|
|
@@ -298,70 +361,92 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
try {
|
|
|
readLock.lock();
|
|
|
List<QueueManagementChange> queueManagementChanges = new ArrayList<>();
|
|
|
+ List<FiCaSchedulerApp> pendingApps = getSortedPendingApplications();
|
|
|
+
|
|
|
+ //Map of LeafQueue->QueueCapacities - keep adding the computed
|
|
|
+ // entitlements to this map and finally
|
|
|
+ // build the leaf queue configuration Template for all identified leaf
|
|
|
+ // queues
|
|
|
+ Map<String, QueueCapacities> leafQueueEntitlements = new HashMap<>();
|
|
|
+ for (String nodeLabel : leafQueueTemplateNodeLabels) {
|
|
|
+ // check if any leaf queues need to be deactivated based on pending
|
|
|
+ // applications
|
|
|
+ float parentAbsoluteCapacity =
|
|
|
+ managedParentQueue.getQueueCapacities().getAbsoluteCapacity(
|
|
|
+ nodeLabel);
|
|
|
+ float leafQueueTemplateAbsoluteCapacity =
|
|
|
+ leafQueueTemplateCapacities.getAbsoluteCapacity(nodeLabel);
|
|
|
+ Map<String, QueueCapacities> deactivatedLeafQueues =
|
|
|
+ deactivateLeafQueuesIfInActive(managedParentQueue, nodeLabel,
|
|
|
+ leafQueueEntitlements);
|
|
|
+
|
|
|
+ float deactivatedCapacity = getTotalDeactivatedCapacity(
|
|
|
+ deactivatedLeafQueues, nodeLabel);
|
|
|
+
|
|
|
+ float sumOfChildQueueActivatedCapacity = parentQueueState.
|
|
|
+ getAbsoluteActivatedChildQueueCapacity(nodeLabel);
|
|
|
+
|
|
|
+ //Check if we need to activate anything at all?
|
|
|
+ float availableCapacity =
|
|
|
+ parentAbsoluteCapacity - sumOfChildQueueActivatedCapacity
|
|
|
+ + deactivatedCapacity + EPSILON;
|
|
|
|
|
|
- // check if any leaf queues need to be deactivated based on pending
|
|
|
- // applications and
|
|
|
- float parentAbsoluteCapacity =
|
|
|
- managedParentQueue.getQueueCapacities().getAbsoluteCapacity();
|
|
|
-
|
|
|
- float leafQueueTemplateAbsoluteCapacity =
|
|
|
- leafQueueTemplateCapacities.getAbsoluteCapacity();
|
|
|
- Map<String, QueueCapacities> deactivatedLeafQueues =
|
|
|
- deactivateLeafQueuesIfInActive(managedParentQueue, queueManagementChanges);
|
|
|
-
|
|
|
- float deactivatedCapacity = getTotalDeactivatedCapacity(
|
|
|
- deactivatedLeafQueues);
|
|
|
-
|
|
|
- float sumOfChildQueueActivatedCapacity = parentQueueState.
|
|
|
- getAbsoluteActivatedChildQueueCapacity();
|
|
|
-
|
|
|
- //Check if we need to activate anything at all?
|
|
|
- float availableCapacity = getAvailableCapacity(parentAbsoluteCapacity,
|
|
|
- deactivatedCapacity, sumOfChildQueueActivatedCapacity);
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug(
|
|
|
- "Parent queue : " + managedParentQueue.getQueueName() + " absCapacity = "
|
|
|
- + parentAbsoluteCapacity + ", leafQueueAbsoluteCapacity = "
|
|
|
- + leafQueueTemplateAbsoluteCapacity + ", deactivatedCapacity = "
|
|
|
- + deactivatedCapacity + " , absChildActivatedCapacity = "
|
|
|
- + sumOfChildQueueActivatedCapacity + ", availableCapacity = "
|
|
|
- + availableCapacity);
|
|
|
- }
|
|
|
-
|
|
|
- if (availableCapacity >= leafQueueTemplateAbsoluteCapacity) {
|
|
|
- //sort applications across leaf queues by submit time
|
|
|
- List<FiCaSchedulerApp> pendingApps = getSortedPendingApplications();
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Parent queue : " + managedParentQueue.getQueueName()
|
|
|
+ + ", nodeLabel = " + nodeLabel + ", absCapacity = "
|
|
|
+ + parentAbsoluteCapacity + ", leafQueueAbsoluteCapacity = "
|
|
|
+ + leafQueueTemplateAbsoluteCapacity + ", deactivatedCapacity = "
|
|
|
+ + deactivatedCapacity + " , absChildActivatedCapacity = "
|
|
|
+ + sumOfChildQueueActivatedCapacity + ", availableCapacity = "
|
|
|
+ + availableCapacity);
|
|
|
+ }
|
|
|
|
|
|
- if (pendingApps.size() > 0) {
|
|
|
- int maxLeafQueuesTobeActivated = getMaxLeavesToBeActivated(
|
|
|
- availableCapacity, leafQueueTemplateAbsoluteCapacity,
|
|
|
- pendingApps.size());
|
|
|
+ if (availableCapacity >= leafQueueTemplateAbsoluteCapacity) {
|
|
|
+ //sort applications across leaf queues by submit time
|
|
|
+ if (pendingApps.size() > 0) {
|
|
|
+ int maxLeafQueuesTobeActivated = getMaxLeavesToBeActivated(
|
|
|
+ availableCapacity, leafQueueTemplateAbsoluteCapacity,
|
|
|
+ pendingApps.size());
|
|
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Found " + maxLeafQueuesTobeActivated
|
|
|
- + " leaf queues to be activated with " + pendingApps.size()
|
|
|
- + " apps ");
|
|
|
- }
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Found " + maxLeafQueuesTobeActivated + " leaf queues"
|
|
|
+ + " to be activated with " + pendingApps.size() + " apps ");
|
|
|
+ }
|
|
|
|
|
|
- LinkedHashSet<String> leafQueuesToBeActivated = getSortedLeafQueues(
|
|
|
- pendingApps, maxLeafQueuesTobeActivated,
|
|
|
- deactivatedLeafQueues.keySet());
|
|
|
+ LinkedHashSet<String> leafQueuesToBeActivated = getSortedLeafQueues(
|
|
|
+ nodeLabel, pendingApps, maxLeafQueuesTobeActivated,
|
|
|
+ deactivatedLeafQueues.keySet());
|
|
|
|
|
|
- //Compute entitlement changes for the identified leaf queues
|
|
|
- // which is appended to the List of queueManagementChanges
|
|
|
- computeQueueManagementChanges(leafQueuesToBeActivated,
|
|
|
- queueManagementChanges, availableCapacity,
|
|
|
- leafQueueTemplateAbsoluteCapacity);
|
|
|
+ //Compute entitlement changes for the identified leaf queues
|
|
|
+ // which is appended to the List of computedEntitlements
|
|
|
+ updateLeafQueueCapacitiesByLabel(nodeLabel, leafQueuesToBeActivated,
|
|
|
+ leafQueueEntitlements);
|
|
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- if (leafQueuesToBeActivated.size() > 0) {
|
|
|
- LOG.debug(
|
|
|
- "Activated leaf queues : [" + leafQueuesToBeActivated + "]");
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ if (leafQueuesToBeActivated.size() > 0) {
|
|
|
+ LOG.debug("Activated leaf queues : [" + leafQueuesToBeActivated
|
|
|
+ + "]");
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ //Populate new entitlements
|
|
|
+
|
|
|
+ for (final Iterator<Map.Entry<String, QueueCapacities>> iterator =
|
|
|
+ leafQueueEntitlements.entrySet().iterator(); iterator.hasNext(); ) {
|
|
|
+ Map.Entry<String, QueueCapacities> queueCapacities = iterator.next();
|
|
|
+ String leafQueueName = queueCapacities.getKey();
|
|
|
+ AutoCreatedLeafQueue leafQueue =
|
|
|
+ (AutoCreatedLeafQueue) scheduler.getCapacitySchedulerQueueManager()
|
|
|
+ .getQueue(leafQueueName);
|
|
|
+ AutoCreatedLeafQueueConfig newTemplate = buildTemplate(
|
|
|
+ queueCapacities.getValue());
|
|
|
+ queueManagementChanges.add(
|
|
|
+ new QueueManagementChange.UpdateQueue(leafQueue, newTemplate));
|
|
|
+
|
|
|
+ }
|
|
|
return queueManagementChanges;
|
|
|
} finally {
|
|
|
readLock.unlock();
|
|
@@ -369,14 +454,14 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
}
|
|
|
|
|
|
private float getTotalDeactivatedCapacity(
|
|
|
- Map<String, QueueCapacities> deactivatedLeafQueues) {
|
|
|
+ Map<String, QueueCapacities> deactivatedLeafQueues, String nodeLabel) {
|
|
|
float deactivatedCapacity = 0;
|
|
|
for (Iterator<Map.Entry<String, QueueCapacities>> iterator =
|
|
|
deactivatedLeafQueues.entrySet().iterator(); iterator.hasNext(); ) {
|
|
|
Map.Entry<String, QueueCapacities> deactivatedQueueCapacity =
|
|
|
iterator.next();
|
|
|
deactivatedCapacity +=
|
|
|
- deactivatedQueueCapacity.getValue().getAbsoluteCapacity();
|
|
|
+ deactivatedQueueCapacity.getValue().getAbsoluteCapacity(nodeLabel);
|
|
|
}
|
|
|
return deactivatedCapacity;
|
|
|
}
|
|
@@ -385,20 +470,42 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
void updateLeafQueueState() {
|
|
|
try {
|
|
|
writeLock.lock();
|
|
|
+ Set<String> newPartitions = new HashSet<>();
|
|
|
Set<String> newQueues = new HashSet<>();
|
|
|
+
|
|
|
for (CSQueue newQueue : managedParentQueue.getChildQueues()) {
|
|
|
if (newQueue instanceof LeafQueue) {
|
|
|
- addLeafQueueStateIfNotExists((LeafQueue) newQueue);
|
|
|
+ for (String nodeLabel : leafQueueTemplateNodeLabels) {
|
|
|
+ leafQueueState.createLeafQueueStateIfNotExists((LeafQueue) newQueue,
|
|
|
+ nodeLabel);
|
|
|
+ newPartitions.add(nodeLabel);
|
|
|
+ }
|
|
|
newQueues.add(newQueue.getQueueName());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- for (Iterator<Map.Entry<String, LeafQueueState>> itr =
|
|
|
- leafQueueStateMap.entrySet().iterator(); itr.hasNext(); ) {
|
|
|
- Map.Entry<String, LeafQueueState> e = itr.next();
|
|
|
- String queueName = e.getKey();
|
|
|
- if (!newQueues.contains(queueName)) {
|
|
|
+ for (Iterator<Map.Entry<String, Map<String, LeafQueueStatePerPartition>>>
|
|
|
+ itr = leafQueueState.getLeafQueueStateMap().entrySet().iterator();
|
|
|
+ itr.hasNext(); ) {
|
|
|
+ Map.Entry<String, Map<String, LeafQueueStatePerPartition>> e =
|
|
|
+ itr.next();
|
|
|
+ String partition = e.getKey();
|
|
|
+ if (!newPartitions.contains(partition)) {
|
|
|
itr.remove();
|
|
|
+ LOG.info(
|
|
|
+ "Removed partition " + partition + " from leaf queue " + "state");
|
|
|
+ } else{
|
|
|
+ Map<String, LeafQueueStatePerPartition> queues = e.getValue();
|
|
|
+ for (
|
|
|
+ Iterator<Map.Entry<String, LeafQueueStatePerPartition>> queueItr =
|
|
|
+ queues.entrySet().iterator(); queueItr.hasNext(); ) {
|
|
|
+ String queue = queueItr.next().getKey();
|
|
|
+ if (!newQueues.contains(queue)) {
|
|
|
+ queueItr.remove();
|
|
|
+ LOG.info("Removed queue " + queue + " from leaf queue "
|
|
|
+ + "state from partition " + partition);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
} finally {
|
|
@@ -406,22 +513,20 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private LinkedHashSet<String> getSortedLeafQueues(
|
|
|
+ private LinkedHashSet<String> getSortedLeafQueues(String nodeLabel,
|
|
|
final List<FiCaSchedulerApp> pendingApps, int leafQueuesNeeded,
|
|
|
Set<String> deactivatedQueues) throws SchedulerDynamicEditException {
|
|
|
|
|
|
LinkedHashSet<String> leafQueues = new LinkedHashSet<>(leafQueuesNeeded);
|
|
|
int ctr = 0;
|
|
|
for (FiCaSchedulerApp app : pendingApps) {
|
|
|
-
|
|
|
AutoCreatedLeafQueue leafQueue =
|
|
|
(AutoCreatedLeafQueue) app.getCSLeafQueue();
|
|
|
String leafQueueName = leafQueue.getQueueName();
|
|
|
|
|
|
//Check if leafQueue is not active already and has any pending apps
|
|
|
if (ctr < leafQueuesNeeded) {
|
|
|
-
|
|
|
- if (!isActive(leafQueue)) {
|
|
|
+ if (!isActive(leafQueue, nodeLabel)) {
|
|
|
if (!deactivatedQueues.contains(leafQueueName)) {
|
|
|
if (addLeafQueueIfNotExists(leafQueues, leafQueueName)) {
|
|
|
ctr++;
|
|
@@ -445,11 +550,12 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- public boolean isActive(final AutoCreatedLeafQueue leafQueue)
|
|
|
- throws SchedulerDynamicEditException {
|
|
|
+ public boolean isActive(final AutoCreatedLeafQueue leafQueue,
|
|
|
+ String nodeLabel) throws SchedulerDynamicEditException {
|
|
|
try {
|
|
|
readLock.lock();
|
|
|
- LeafQueueState leafQueueStatus = getLeafQueueState(leafQueue);
|
|
|
+ LeafQueueStatePerPartition leafQueueStatus = getLeafQueueState(leafQueue,
|
|
|
+ nodeLabel);
|
|
|
return leafQueueStatus.isActive();
|
|
|
} finally {
|
|
|
readLock.unlock();
|
|
@@ -457,64 +563,52 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
}
|
|
|
|
|
|
private Map<String, QueueCapacities> deactivateLeafQueuesIfInActive(
|
|
|
- ParentQueue parentQueue,
|
|
|
- List<QueueManagementChange> queueManagementChanges)
|
|
|
+ ParentQueue parentQueue, String nodeLabel,
|
|
|
+ Map<String, QueueCapacities> leafQueueEntitlements)
|
|
|
throws SchedulerDynamicEditException {
|
|
|
Map<String, QueueCapacities> deactivatedQueues = new HashMap<>();
|
|
|
|
|
|
for (CSQueue childQueue : parentQueue.getChildQueues()) {
|
|
|
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue;
|
|
|
+ if (leafQueue != null) {
|
|
|
+ if (isActive(leafQueue, nodeLabel) && !hasPendingApps(leafQueue)) {
|
|
|
+ if (!leafQueueEntitlements.containsKey(leafQueue.getQueueName())) {
|
|
|
+ leafQueueEntitlements.put(leafQueue.getQueueName(),
|
|
|
+ new QueueCapacities(false));
|
|
|
+ }
|
|
|
|
|
|
- if (isActive(leafQueue) && !hasPendingApps(leafQueue)) {
|
|
|
- queueManagementChanges.add(
|
|
|
- new QueueManagementChange.UpdateQueue(leafQueue,
|
|
|
- ZERO_CAPACITY_ENTITLEMENT));
|
|
|
- deactivatedQueues.put(leafQueue.getQueueName(),
|
|
|
- leafQueueTemplateCapacities);
|
|
|
- } else{
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug(" Leaf queue has pending applications : " + leafQueue
|
|
|
- .getNumApplications() + ".Skipping deactivation for "
|
|
|
- + leafQueue);
|
|
|
+ QueueCapacities capacities = leafQueueEntitlements.get(
|
|
|
+ leafQueue.getQueueName());
|
|
|
+ updateToZeroCapacity(capacities, nodeLabel);
|
|
|
+ deactivatedQueues.put(leafQueue.getQueueName(),
|
|
|
+ leafQueueTemplateCapacities);
|
|
|
+ } else{
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug(" Leaf queue has pending applications or is " + "inactive"
|
|
|
+ + " : " + leafQueue.getNumApplications()
|
|
|
+ + ".Skipping deactivation for " + leafQueue);
|
|
|
+ }
|
|
|
}
|
|
|
+ } else{
|
|
|
+ LOG.warn("Could not find queue in scheduler while trying" + " to "
|
|
|
+ + "deactivate for " + parentQueue);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- if (deactivatedQueues.size() > 0) {
|
|
|
- LOG.debug("Deactivated leaf queues : " + deactivatedQueues);
|
|
|
- }
|
|
|
- }
|
|
|
return deactivatedQueues;
|
|
|
}
|
|
|
|
|
|
- private void computeQueueManagementChanges(
|
|
|
+ private void updateLeafQueueCapacitiesByLabel(String nodeLabel,
|
|
|
Set<String> leafQueuesToBeActivated,
|
|
|
- List<QueueManagementChange> queueManagementChanges,
|
|
|
- final float availableCapacity,
|
|
|
- final float leafQueueTemplateAbsoluteCapacity) {
|
|
|
-
|
|
|
- float curAvailableCapacity = availableCapacity;
|
|
|
-
|
|
|
+ Map<String, QueueCapacities> leafQueueEntitlements) {
|
|
|
for (String curLeafQueue : leafQueuesToBeActivated) {
|
|
|
- // Activate queues if capacity is available
|
|
|
- if (curAvailableCapacity >= leafQueueTemplateAbsoluteCapacity) {
|
|
|
- AutoCreatedLeafQueue leafQueue =
|
|
|
- (AutoCreatedLeafQueue) scheduler.getCapacitySchedulerQueueManager()
|
|
|
- .getQueue(curLeafQueue);
|
|
|
- if (leafQueue != null) {
|
|
|
- AutoCreatedLeafQueueConfig newTemplate = buildTemplate(
|
|
|
- leafQueueTemplateCapacities.getCapacity(),
|
|
|
- leafQueueTemplateCapacities.getMaximumCapacity());
|
|
|
- queueManagementChanges.add(
|
|
|
- new QueueManagementChange.UpdateQueue(leafQueue, newTemplate));
|
|
|
- curAvailableCapacity -= leafQueueTemplateAbsoluteCapacity;
|
|
|
- } else{
|
|
|
- LOG.warn(
|
|
|
- "Could not find queue in scheduler while trying to deactivate "
|
|
|
- + curLeafQueue);
|
|
|
- }
|
|
|
+ if (!leafQueueEntitlements.containsKey(curLeafQueue)) {
|
|
|
+ leafQueueEntitlements.put(curLeafQueue, new QueueCapacities(false));
|
|
|
+ // Activate queues if capacity is available
|
|
|
}
|
|
|
+
|
|
|
+ QueueCapacities capacities = leafQueueEntitlements.get(curLeafQueue);
|
|
|
+ updateCapacityFromTemplate(capacities, nodeLabel);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -528,17 +622,8 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
availableCapacity / childQueueAbsoluteCapacity);
|
|
|
|
|
|
return Math.min(numLeafQueuesNeeded, numPendingApps);
|
|
|
- } else{
|
|
|
- throw new SchedulerDynamicEditException("Child queue absolute capacity "
|
|
|
- + "is initialized to 0. Check parent queue's " + managedParentQueue
|
|
|
- .getQueueName() + " leaf queue template configuration");
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- private float getAvailableCapacity(float parentAbsCapacity,
|
|
|
- float deactivatedAbsCapacity, float totalChildQueueActivatedCapacity) {
|
|
|
- return parentAbsCapacity - totalChildQueueActivatedCapacity
|
|
|
- + deactivatedAbsCapacity + EPSILON;
|
|
|
+ return 0;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -567,25 +652,27 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
|
|
|
AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) queue;
|
|
|
|
|
|
- if (updatedQueueTemplate.getQueueCapacities().getCapacity() > 0) {
|
|
|
- if (isActive(leafQueue)) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug(
|
|
|
- "Queue is already active. Skipping activation : " + queue
|
|
|
- .getQueuePath());
|
|
|
+ for (String nodeLabel : updatedQueueTemplate.getQueueCapacities()
|
|
|
+ .getExistingNodeLabels()) {
|
|
|
+ if (updatedQueueTemplate.getQueueCapacities().
|
|
|
+ getCapacity(nodeLabel) > 0) {
|
|
|
+ if (isActive(leafQueue, nodeLabel)) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Queue is already active." + " Skipping activation : "
|
|
|
+ + queue.getQueuePath());
|
|
|
+ }
|
|
|
+ } else{
|
|
|
+ activate(leafQueue, nodeLabel);
|
|
|
}
|
|
|
} else{
|
|
|
- activate(leafQueue);
|
|
|
- }
|
|
|
- } else{
|
|
|
- if (!isActive(leafQueue)) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug(
|
|
|
- "Queue is already de-activated. " + "Skipping de-activation "
|
|
|
- + ": " + leafQueue.getQueuePath());
|
|
|
+ if (!isActive(leafQueue, nodeLabel)) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Queue is already de-activated. Skipping "
|
|
|
+ + "de-activation : " + leafQueue.getQueuePath());
|
|
|
+ }
|
|
|
+ } else{
|
|
|
+ deactivate(leafQueue, nodeLabel);
|
|
|
}
|
|
|
- } else{
|
|
|
- deactivate(leafQueue);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -594,30 +681,26 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void activate(final AutoCreatedLeafQueue leafQueue)
|
|
|
- throws SchedulerDynamicEditException {
|
|
|
+ private void activate(final AbstractAutoCreatedLeafQueue leafQueue,
|
|
|
+ String nodeLabel) throws SchedulerDynamicEditException {
|
|
|
try {
|
|
|
writeLock.lock();
|
|
|
- getLeafQueueState(leafQueue).activate();
|
|
|
-
|
|
|
- parentQueueState.incAbsoluteActivatedChildCapacity(NO_LABEL,
|
|
|
- leafQueueTemplateCapacities.getAbsoluteCapacity());
|
|
|
+ getLeafQueueState(leafQueue, nodeLabel).activate();
|
|
|
+ parentQueueState.incAbsoluteActivatedChildCapacity(nodeLabel,
|
|
|
+ leafQueueTemplateCapacities.getAbsoluteCapacity(nodeLabel));
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void deactivate(final AutoCreatedLeafQueue leafQueue)
|
|
|
- throws SchedulerDynamicEditException {
|
|
|
+ private void deactivate(final AbstractAutoCreatedLeafQueue leafQueue,
|
|
|
+ String nodeLabel) throws SchedulerDynamicEditException {
|
|
|
try {
|
|
|
writeLock.lock();
|
|
|
- getLeafQueueState(leafQueue).deactivate();
|
|
|
+ getLeafQueueState(leafQueue, nodeLabel).deactivate();
|
|
|
|
|
|
- for (String nodeLabel : managedParentQueue.getQueueCapacities()
|
|
|
- .getExistingNodeLabels()) {
|
|
|
- parentQueueState.decAbsoluteActivatedChildCapacity(nodeLabel,
|
|
|
- leafQueueTemplateCapacities.getAbsoluteCapacity());
|
|
|
- }
|
|
|
+ parentQueueState.decAbsoluteActivatedChildCapacity(nodeLabel,
|
|
|
+ leafQueueTemplateCapacities.getAbsoluteCapacity(nodeLabel));
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
@@ -629,7 +712,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
|
|
|
@Override
|
|
|
public void reinitialize(CapacitySchedulerContext schedulerContext,
|
|
|
- final ParentQueue parentQueue) {
|
|
|
+ final ParentQueue parentQueue) throws IOException {
|
|
|
if (!(parentQueue instanceof ManagedParentQueue)) {
|
|
|
throw new IllegalStateException(
|
|
|
"Expected instance of type " + ManagedParentQueue.class + " found "
|
|
@@ -649,12 +732,11 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
|
|
|
//clear state
|
|
|
parentQueueState.clear();
|
|
|
- clearLeafQueueState();
|
|
|
+ leafQueueState.clear();
|
|
|
|
|
|
LOG.info(
|
|
|
- "Reinitialized queue management policy for parent queue "
|
|
|
- + parentQueue.getQueueName() +" with leaf queue template "
|
|
|
- + "capacities : ["
|
|
|
+ "Reinitialized queue management policy for parent queue " + parentQueue
|
|
|
+ .getQueueName() + " with leaf queue template " + "capacities : ["
|
|
|
+ leafQueueTemplate.getQueueCapacities() + "]");
|
|
|
}
|
|
|
|
|
@@ -663,51 +745,74 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
AbstractAutoCreatedLeafQueue leafQueue)
|
|
|
throws SchedulerDynamicEditException {
|
|
|
|
|
|
- if ( !(leafQueue instanceof AutoCreatedLeafQueue)) {
|
|
|
- throw new SchedulerDynamicEditException("Not an instance of "
|
|
|
- + "AutoCreatedLeafQueue : " + leafQueue.getClass());
|
|
|
+ AutoCreatedLeafQueueConfig template;
|
|
|
+
|
|
|
+ if (!(leafQueue instanceof AutoCreatedLeafQueue)) {
|
|
|
+ throw new SchedulerDynamicEditException(
|
|
|
+ "Not an instance of " + "AutoCreatedLeafQueue : " + leafQueue
|
|
|
+ .getClass());
|
|
|
}
|
|
|
|
|
|
- AutoCreatedLeafQueue autoCreatedLeafQueue =
|
|
|
- (AutoCreatedLeafQueue) leafQueue;
|
|
|
- AutoCreatedLeafQueueConfig template = ZERO_CAPACITY_ENTITLEMENT;
|
|
|
try {
|
|
|
writeLock.lock();
|
|
|
- if (!addLeafQueueStateIfNotExists(leafQueue)) {
|
|
|
- LOG.error("Leaf queue already exists in state : " + getLeafQueueState(
|
|
|
- leafQueue));
|
|
|
- throw new SchedulerDynamicEditException(
|
|
|
- "Leaf queue already exists in state : " + getLeafQueueState(
|
|
|
- leafQueue));
|
|
|
- }
|
|
|
|
|
|
- float availableCapacity = getAvailableCapacity(
|
|
|
- managedParentQueue.getQueueCapacities().getAbsoluteCapacity(), 0,
|
|
|
- parentQueueState.getAbsoluteActivatedChildQueueCapacity());
|
|
|
+ QueueCapacities capacities = new QueueCapacities(false);
|
|
|
+ for (String nodeLabel : leafQueueTemplateNodeLabels) {
|
|
|
+ if (!leafQueueState.createLeafQueueStateIfNotExists(leafQueue,
|
|
|
+ nodeLabel)) {
|
|
|
+ String message =
|
|
|
+ "Leaf queue already exists in state : " + getLeafQueueState(
|
|
|
+ leafQueue, nodeLabel);
|
|
|
+ LOG.error(message);
|
|
|
+ }
|
|
|
|
|
|
- if (availableCapacity >= leafQueueTemplateCapacities
|
|
|
- .getAbsoluteCapacity()) {
|
|
|
- activate(autoCreatedLeafQueue);
|
|
|
- template = buildTemplate(leafQueueTemplateCapacities.getCapacity(),
|
|
|
- leafQueueTemplateCapacities.getMaximumCapacity());
|
|
|
+ float availableCapacity = managedParentQueue.getQueueCapacities().
|
|
|
+ getAbsoluteCapacity(nodeLabel) - parentQueueState.
|
|
|
+ getAbsoluteActivatedChildQueueCapacity(nodeLabel) + EPSILON;
|
|
|
+
|
|
|
+ if (availableCapacity >= leafQueueTemplateCapacities
|
|
|
+ .getAbsoluteCapacity(nodeLabel)) {
|
|
|
+ updateCapacityFromTemplate(capacities, nodeLabel);
|
|
|
+ activate(leafQueue, nodeLabel);
|
|
|
+ } else{
|
|
|
+ updateToZeroCapacity(capacities, nodeLabel);
|
|
|
+ }
|
|
|
}
|
|
|
+
|
|
|
+ template = buildTemplate(capacities);
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
|
return template;
|
|
|
}
|
|
|
|
|
|
+ private void updateToZeroCapacity(QueueCapacities capacities,
|
|
|
+ String nodeLabel) {
|
|
|
+ capacities.setCapacity(nodeLabel, 0.0f);
|
|
|
+ capacities.setMaximumCapacity(nodeLabel,
|
|
|
+ leafQueueTemplateCapacities.getMaximumCapacity(nodeLabel));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void updateCapacityFromTemplate(QueueCapacities capacities,
|
|
|
+ String nodeLabel) {
|
|
|
+ capacities.setCapacity(nodeLabel,
|
|
|
+ leafQueueTemplateCapacities.getCapacity(nodeLabel));
|
|
|
+ capacities.setMaximumCapacity(nodeLabel,
|
|
|
+ leafQueueTemplateCapacities.getMaximumCapacity(nodeLabel));
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
- LeafQueueState getLeafQueueState(LeafQueue queue)
|
|
|
- throws SchedulerDynamicEditException {
|
|
|
+ LeafQueueStatePerPartition getLeafQueueState(LeafQueue queue,
|
|
|
+ String partition) throws SchedulerDynamicEditException {
|
|
|
try {
|
|
|
readLock.lock();
|
|
|
String queueName = queue.getQueueName();
|
|
|
- if (!containsLeafQueue(queueName)) {
|
|
|
+ if (!leafQueueState.containsLeafQueue(queueName, partition)) {
|
|
|
throw new SchedulerDynamicEditException(
|
|
|
"Could not find leaf queue in " + "state " + queueName);
|
|
|
} else{
|
|
|
- return leafQueueStateMap.get(queueName);
|
|
|
+ return leafQueueState.
|
|
|
+ getLeafQueueStatePerPartition(queueName, partition);
|
|
|
}
|
|
|
} finally {
|
|
|
readLock.unlock();
|
|
@@ -715,8 +820,8 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- public float getAbsoluteActivatedChildQueueCapacity() {
|
|
|
- return parentQueueState.getAbsoluteActivatedChildQueueCapacity();
|
|
|
+ public float getAbsoluteActivatedChildQueueCapacity(String nodeLabel) {
|
|
|
+ return parentQueueState.getAbsoluteActivatedChildQueueCapacity(nodeLabel);
|
|
|
}
|
|
|
|
|
|
private List<FiCaSchedulerApp> getSortedPendingApplications() {
|
|
@@ -726,20 +831,10 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
return apps;
|
|
|
}
|
|
|
|
|
|
- private AutoCreatedLeafQueueConfig buildTemplate(float capacity,
|
|
|
- float maxCapacity) {
|
|
|
+ private AutoCreatedLeafQueueConfig buildTemplate(QueueCapacities capacities) {
|
|
|
AutoCreatedLeafQueueConfig.Builder templateBuilder =
|
|
|
new AutoCreatedLeafQueueConfig.Builder();
|
|
|
-
|
|
|
- QueueCapacities capacities = new QueueCapacities(false);
|
|
|
templateBuilder.capacities(capacities);
|
|
|
-
|
|
|
- for (String nodeLabel : managedParentQueue.getQueueCapacities()
|
|
|
- .getExistingNodeLabels()) {
|
|
|
- capacities.setCapacity(nodeLabel, capacity);
|
|
|
- capacities.setMaximumCapacity(nodeLabel, maxCapacity);
|
|
|
- }
|
|
|
-
|
|
|
return new AutoCreatedLeafQueueConfig(templateBuilder);
|
|
|
}
|
|
|
}
|