|
@@ -17,7 +17,6 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
|
|
|
|
|
-import java.util.HashSet;
|
|
|
import java.util.Set;
|
|
|
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
@@ -181,9 +180,12 @@ class CSQueueUtils {
|
|
|
* used resource for all partitions of this queue.
|
|
|
*/
|
|
|
public static void updateUsedCapacity(final ResourceCalculator rc,
|
|
|
- final Resource totalPartitionResource, final Resource minimumAllocation,
|
|
|
- ResourceUsage queueResourceUsage, QueueCapacities queueCapacities,
|
|
|
- String nodePartition) {
|
|
|
+ final Resource totalPartitionResource, String nodePartition,
|
|
|
+ AbstractCSQueue childQueue) {
|
|
|
+ QueueCapacities queueCapacities = childQueue.getQueueCapacities();
|
|
|
+ CSQueueMetrics queueMetrics = childQueue.getMetrics();
|
|
|
+ ResourceUsage queueResourceUsage = childQueue.getQueueResourceUsage();
|
|
|
+ Resource minimumAllocation = childQueue.getMinimumAllocation();
|
|
|
float absoluteUsedCapacity = 0.0f;
|
|
|
float usedCapacity = 0.0f;
|
|
|
float reservedCapacity = 0.0f;
|
|
@@ -225,8 +227,18 @@ class CSQueueUtils {
|
|
|
queueCapacities.setReservedCapacity(nodePartition, reservedCapacity);
|
|
|
queueCapacities
|
|
|
.setAbsoluteReservedCapacity(nodePartition, absoluteReservedCapacity);
|
|
|
+
|
|
|
+ // QueueMetrics does not support per-label capacities,
|
|
|
+ // so we report values only for the default partition.
|
|
|
+ if (nodePartition.equals(CommonNodeLabelsManager.NO_LABEL)) {
|
|
|
+ queueMetrics.setUsedCapacity(
|
|
|
+ queueCapacities.getUsedCapacity(RMNodeLabelsManager.NO_LABEL));
|
|
|
+ queueMetrics.setAbsoluteUsedCapacity(
|
|
|
+ queueCapacities.getAbsoluteUsedCapacity(
|
|
|
+ RMNodeLabelsManager.NO_LABEL));
|
|
|
+ }
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private static Resource getMaxAvailableResourceToQueue(
|
|
|
final ResourceCalculator rc, RMNodeLabelsManager nlm, CSQueue queue,
|
|
|
Resource cluster) {
|
|
@@ -270,22 +282,22 @@ class CSQueueUtils {
|
|
|
*/
|
|
|
@Lock(CSQueue.class)
|
|
|
public static void updateQueueStatistics(
|
|
|
- final ResourceCalculator rc, final Resource cluster, final Resource minimumAllocation,
|
|
|
- final CSQueue childQueue, final RMNodeLabelsManager nlm,
|
|
|
+ final ResourceCalculator rc, final Resource cluster,
|
|
|
+ final AbstractCSQueue childQueue, final RMNodeLabelsManager nlm,
|
|
|
final String nodePartition) {
|
|
|
QueueCapacities queueCapacities = childQueue.getQueueCapacities();
|
|
|
ResourceUsage queueResourceUsage = childQueue.getQueueResourceUsage();
|
|
|
-
|
|
|
+
|
|
|
if (nodePartition == null) {
|
|
|
for (String partition : Sets.union(
|
|
|
queueCapacities.getNodePartitionsSet(),
|
|
|
queueResourceUsage.getNodePartitionsSet())) {
|
|
|
updateUsedCapacity(rc, nlm.getResourceByLabel(partition, cluster),
|
|
|
- minimumAllocation, queueResourceUsage, queueCapacities, partition);
|
|
|
+ partition, childQueue);
|
|
|
}
|
|
|
} else {
|
|
|
updateUsedCapacity(rc, nlm.getResourceByLabel(nodePartition, cluster),
|
|
|
- minimumAllocation, queueResourceUsage, queueCapacities, nodePartition);
|
|
|
+ nodePartition, childQueue);
|
|
|
}
|
|
|
|
|
|
// Update queue metrics w.r.t node labels. In a generic way, we can
|