|
@@ -19,8 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
|
|
.queuemanagement;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
-import org.apache.commons.logging.Log;
|
|
|
-import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
|
|
+ .QueueManagementDynamicEditPolicy;
|
|
|
+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;
|
|
@@ -81,8 +83,8 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
private CapacitySchedulerContext scheduler;
|
|
|
private ManagedParentQueue managedParentQueue;
|
|
|
|
|
|
- private static final Log LOG = LogFactory.getLog(
|
|
|
- GuaranteedOrZeroCapacityOverTimePolicy.class);
|
|
|
+ private static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(GuaranteedOrZeroCapacityOverTimePolicy.class);
|
|
|
|
|
|
private ReentrantReadWriteLock.WriteLock writeLock;
|
|
|
|
|
@@ -380,6 +382,17 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
deactivateLeafQueuesIfInActive(managedParentQueue, nodeLabel,
|
|
|
leafQueueEntitlements);
|
|
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ if ( deactivatedLeafQueues.size() > 0) {
|
|
|
+ LOG.debug("Parent queue = {}, " +
|
|
|
+ ", nodeLabel = {}, deactivated leaf queues = [{}] ",
|
|
|
+ managedParentQueue.getQueueName(), nodeLabel,
|
|
|
+ deactivatedLeafQueues.size() > 25 ? deactivatedLeafQueues
|
|
|
+ .size() : deactivatedLeafQueues);
|
|
|
+
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
float deactivatedCapacity = getTotalDeactivatedCapacity(
|
|
|
deactivatedLeafQueues, nodeLabel);
|
|
|
|
|
@@ -392,7 +405,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
+ deactivatedCapacity + EPSILON;
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Parent queue : " + managedParentQueue.getQueueName()
|
|
|
+ LOG.debug("Parent queue = " + managedParentQueue.getQueueName()
|
|
|
+ ", nodeLabel = " + nodeLabel + ", absCapacity = "
|
|
|
+ parentAbsoluteCapacity + ", leafQueueAbsoluteCapacity = "
|
|
|
+ leafQueueTemplateAbsoluteCapacity + ", deactivatedCapacity = "
|
|
@@ -409,7 +422,8 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
pendingApps.size());
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Found " + maxLeafQueuesTobeActivated + " leaf queues"
|
|
|
+ LOG.debug("Parent queue = " + managedParentQueue.getQueueName()
|
|
|
+ + " : Found " + maxLeafQueuesTobeActivated + " leaf queues"
|
|
|
+ " to be activated with " + pendingApps.size() + " apps ");
|
|
|
}
|
|
|
|
|
@@ -424,8 +438,9 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
if (leafQueuesToBeActivated.size() > 0) {
|
|
|
- LOG.debug("Activated leaf queues : [" + leafQueuesToBeActivated
|
|
|
- + "]");
|
|
|
+ LOG.debug("Activated leaf queues : [{}]",
|
|
|
+ leafQueuesToBeActivated.size() < 25 ?
|
|
|
+ leafQueuesToBeActivated : leafQueuesToBeActivated.size());
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -492,8 +507,9 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
String partition = e.getKey();
|
|
|
if (!newPartitions.contains(partition)) {
|
|
|
itr.remove();
|
|
|
- LOG.info(
|
|
|
- "Removed partition " + partition + " from leaf queue " + "state");
|
|
|
+ LOG.info(managedParentQueue.getQueueName() +
|
|
|
+ " : Removed partition " + partition + " from leaf queue " +
|
|
|
+ "state");
|
|
|
} else{
|
|
|
Map<String, LeafQueueStatePerPartition> queues = e.getValue();
|
|
|
for (
|
|
@@ -502,7 +518,9 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
String queue = queueItr.next().getKey();
|
|
|
if (!newQueues.contains(queue)) {
|
|
|
queueItr.remove();
|
|
|
- LOG.info("Removed queue " + queue + " from leaf queue "
|
|
|
+ LOG.info(managedParentQueue.getQueueName() + " : Removed queue"
|
|
|
+ + queue + " from "
|
|
|
+ + "leaf queue "
|
|
|
+ "state from partition " + partition);
|
|
|
}
|
|
|
}
|
|
@@ -582,12 +600,6 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
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 "
|
|
@@ -659,7 +671,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
if (isActive(leafQueue, nodeLabel)) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Queue is already active." + " Skipping activation : "
|
|
|
- + queue.getQueuePath());
|
|
|
+ + leafQueue.getQueueName());
|
|
|
}
|
|
|
} else{
|
|
|
activate(leafQueue, nodeLabel);
|
|
@@ -668,7 +680,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
|
|
|
if (!isActive(leafQueue, nodeLabel)) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Queue is already de-activated. Skipping "
|
|
|
- + "de-activation : " + leafQueue.getQueuePath());
|
|
|
+ + "de-activation : " + leafQueue.getQueueName());
|
|
|
}
|
|
|
} else{
|
|
|
deactivate(leafQueue, nodeLabel);
|