|
@@ -18,17 +18,8 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
|
-import java.util.ArrayList;
|
|
|
|
-import java.util.Collection;
|
|
|
|
-import java.util.Collections;
|
|
|
|
-import java.util.HashMap;
|
|
|
|
-import java.util.Iterator;
|
|
|
|
-import java.util.List;
|
|
|
|
-import java.util.Map;
|
|
|
|
-import java.util.Set;
|
|
|
|
-import java.util.TreeSet;
|
|
|
|
-
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
|
+import com.google.common.collect.Sets;
|
|
import org.apache.commons.lang.StringUtils;
|
|
import org.apache.commons.lang.StringUtils;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -60,7 +51,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
|
-
|
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
|
|
@@ -75,9 +65,22 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderi
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
|
|
import org.apache.hadoop.yarn.server.utils.Lock;
|
|
import org.apache.hadoop.yarn.server.utils.Lock;
|
|
import org.apache.hadoop.yarn.server.utils.Lock.NoLock;
|
|
import org.apache.hadoop.yarn.server.utils.Lock.NoLock;
|
|
|
|
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
|
+import java.util.ArrayList;
|
|
|
|
+import java.util.Collection;
|
|
|
|
+import java.util.Collections;
|
|
|
|
+import java.util.HashMap;
|
|
|
|
+import java.util.Iterator;
|
|
|
|
+import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
|
|
+import java.util.Set;
|
|
|
|
+import java.util.TreeSet;
|
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
|
|
|
|
|
@Private
|
|
@Private
|
|
@Unstable
|
|
@Unstable
|
|
@@ -116,7 +119,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
// cache last cluster resource to compute actual capacity
|
|
// cache last cluster resource to compute actual capacity
|
|
private Resource lastClusterResource = Resources.none();
|
|
private Resource lastClusterResource = Resources.none();
|
|
-
|
|
|
|
|
|
+
|
|
private final QueueResourceLimitsInfo queueResourceLimitsInfo =
|
|
private final QueueResourceLimitsInfo queueResourceLimitsInfo =
|
|
new QueueResourceLimitsInfo();
|
|
new QueueResourceLimitsInfo();
|
|
|
|
|
|
@@ -124,6 +127,10 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
private OrderingPolicy<FiCaSchedulerApp> orderingPolicy = null;
|
|
private OrderingPolicy<FiCaSchedulerApp> orderingPolicy = null;
|
|
|
|
|
|
|
|
+ // Summation of consumed ratios for all users in queue
|
|
|
|
+ private float totalUserConsumedRatio = 0;
|
|
|
|
+ private UsageRatios qUsageRatios;
|
|
|
|
+
|
|
// record all ignore partition exclusivityRMContainer, this will be used to do
|
|
// record all ignore partition exclusivityRMContainer, this will be used to do
|
|
// preemption, key is the partition of the RMContainer allocated on
|
|
// preemption, key is the partition of the RMContainer allocated on
|
|
private Map<String, TreeSet<RMContainer>> ignorePartitionExclusivityRMContainers =
|
|
private Map<String, TreeSet<RMContainer>> ignorePartitionExclusivityRMContainers =
|
|
@@ -140,6 +147,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
// One time initialization is enough since it is static ordering policy
|
|
// One time initialization is enough since it is static ordering policy
|
|
this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps();
|
|
this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps();
|
|
|
|
|
|
|
|
+ qUsageRatios = new UsageRatios();
|
|
|
|
+
|
|
if(LOG.isDebugEnabled()) {
|
|
if(LOG.isDebugEnabled()) {
|
|
LOG.debug("LeafQueue:" + " name=" + queueName
|
|
LOG.debug("LeafQueue:" + " name=" + queueName
|
|
+ ", fullname=" + getQueuePath());
|
|
+ ", fullname=" + getQueuePath());
|
|
@@ -164,7 +173,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
setQueueResourceLimitsInfo(clusterResource);
|
|
setQueueResourceLimitsInfo(clusterResource);
|
|
|
|
|
|
CapacitySchedulerConfiguration conf = csContext.getConfiguration();
|
|
CapacitySchedulerConfiguration conf = csContext.getConfiguration();
|
|
-
|
|
|
|
|
|
+
|
|
setOrderingPolicy(conf.<FiCaSchedulerApp>getOrderingPolicy(getQueuePath()));
|
|
setOrderingPolicy(conf.<FiCaSchedulerApp>getOrderingPolicy(getQueuePath()));
|
|
|
|
|
|
userLimit = conf.getUserLimit(getQueuePath());
|
|
userLimit = conf.getUserLimit(getQueuePath());
|
|
@@ -1113,6 +1122,9 @@ public class LeafQueue extends AbstractCSQueue {
|
|
private Resource computeUserLimit(FiCaSchedulerApp application,
|
|
private Resource computeUserLimit(FiCaSchedulerApp application,
|
|
Resource clusterResource, User user,
|
|
Resource clusterResource, User user,
|
|
String nodePartition, SchedulingMode schedulingMode) {
|
|
String nodePartition, SchedulingMode schedulingMode) {
|
|
|
|
+ Resource partitionResource = labelManager.getResourceByLabel(nodePartition,
|
|
|
|
+ clusterResource);
|
|
|
|
+
|
|
// What is our current capacity?
|
|
// What is our current capacity?
|
|
// * It is equal to the max(required, queue-capacity) if
|
|
// * It is equal to the max(required, queue-capacity) if
|
|
// we're running below capacity. The 'max' ensures that jobs in queues
|
|
// we're running below capacity. The 'max' ensures that jobs in queues
|
|
@@ -1121,7 +1133,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
// (usedResources + required) (which extra resources we are allocating)
|
|
// (usedResources + required) (which extra resources we are allocating)
|
|
Resource queueCapacity =
|
|
Resource queueCapacity =
|
|
Resources.multiplyAndNormalizeUp(resourceCalculator,
|
|
Resources.multiplyAndNormalizeUp(resourceCalculator,
|
|
- labelManager.getResourceByLabel(nodePartition, clusterResource),
|
|
|
|
|
|
+ partitionResource,
|
|
queueCapacities.getAbsoluteCapacity(nodePartition),
|
|
queueCapacities.getAbsoluteCapacity(nodePartition),
|
|
minimumAllocation);
|
|
minimumAllocation);
|
|
|
|
|
|
@@ -1133,15 +1145,30 @@ public class LeafQueue extends AbstractCSQueue {
|
|
// Allow progress for queues with miniscule capacity
|
|
// Allow progress for queues with miniscule capacity
|
|
queueCapacity =
|
|
queueCapacity =
|
|
Resources.max(
|
|
Resources.max(
|
|
- resourceCalculator, clusterResource,
|
|
|
|
|
|
+ resourceCalculator, partitionResource,
|
|
queueCapacity,
|
|
queueCapacity,
|
|
required);
|
|
required);
|
|
|
|
|
|
|
|
+
|
|
|
|
+ /* We want to base the userLimit calculation on
|
|
|
|
+ * max(queueCapacity, usedResources+required). However, we want
|
|
|
|
+ * usedResources to be based on the combined ratios of all the users in the
|
|
|
|
+ * queue so we use consumedRatio to calculate such.
|
|
|
|
+ * The calculation is dependent on how the resourceCalculator calculates the
|
|
|
|
+ * ratio between two Resources. DRF Example: If usedResources is
|
|
|
|
+ * greater than queueCapacity and users have the following [mem,cpu] usages:
|
|
|
|
+ * User1: [10%,20%] - Dominant resource is 20%
|
|
|
|
+ * User2: [30%,10%] - Dominant resource is 30%
|
|
|
|
+ * Then total consumedRatio is then 20+30=50%. Yes, this value can be
|
|
|
|
+ * larger than 100% but for the purposes of making sure all users are
|
|
|
|
+ * getting their fair share, it works.
|
|
|
|
+ */
|
|
|
|
+ Resource consumed = Resources.multiplyAndNormalizeUp(resourceCalculator,
|
|
|
|
+ partitionResource, qUsageRatios.getUsageRatio(nodePartition),
|
|
|
|
+ minimumAllocation);
|
|
Resource currentCapacity =
|
|
Resource currentCapacity =
|
|
- Resources.lessThan(resourceCalculator, clusterResource,
|
|
|
|
- queueUsage.getUsed(nodePartition), queueCapacity) ? queueCapacity
|
|
|
|
- : Resources.add(queueUsage.getUsed(nodePartition), required);
|
|
|
|
-
|
|
|
|
|
|
+ Resources.lessThan(resourceCalculator, partitionResource, consumed,
|
|
|
|
+ queueCapacity) ? queueCapacity : Resources.add(consumed, required);
|
|
// Never allow a single user to take more than the
|
|
// Never allow a single user to take more than the
|
|
// queue's configured capacity * user-limit-factor.
|
|
// queue's configured capacity * user-limit-factor.
|
|
// Also, the queue's configured capacity should be higher than
|
|
// Also, the queue's configured capacity should be higher than
|
|
@@ -1150,9 +1177,10 @@ public class LeafQueue extends AbstractCSQueue {
|
|
final int activeUsers = activeUsersManager.getNumActiveUsers();
|
|
final int activeUsers = activeUsersManager.getNumActiveUsers();
|
|
|
|
|
|
// User limit resource is determined by:
|
|
// User limit resource is determined by:
|
|
- // max{currentCapacity / #activeUsers, currentCapacity * user-limit-percentage%)
|
|
|
|
|
|
+ // max{currentCapacity / #activeUsers, currentCapacity *
|
|
|
|
+ // user-limit-percentage%)
|
|
Resource userLimitResource = Resources.max(
|
|
Resource userLimitResource = Resources.max(
|
|
- resourceCalculator, clusterResource,
|
|
|
|
|
|
+ resourceCalculator, partitionResource,
|
|
Resources.divideAndCeil(
|
|
Resources.divideAndCeil(
|
|
resourceCalculator, currentCapacity, activeUsers),
|
|
resourceCalculator, currentCapacity, activeUsers),
|
|
Resources.divideAndCeil(
|
|
Resources.divideAndCeil(
|
|
@@ -1176,8 +1204,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
maxUserLimit =
|
|
maxUserLimit =
|
|
Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor);
|
|
Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor);
|
|
} else if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
|
|
} else if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
|
|
- maxUserLimit =
|
|
|
|
- labelManager.getResourceByLabel(nodePartition, clusterResource);
|
|
|
|
|
|
+ maxUserLimit = partitionResource;
|
|
}
|
|
}
|
|
|
|
|
|
// Cap final user limit with maxUserLimit
|
|
// Cap final user limit with maxUserLimit
|
|
@@ -1185,7 +1212,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
Resources.roundUp(
|
|
Resources.roundUp(
|
|
resourceCalculator,
|
|
resourceCalculator,
|
|
Resources.min(
|
|
Resources.min(
|
|
- resourceCalculator, clusterResource,
|
|
|
|
|
|
+ resourceCalculator, partitionResource,
|
|
userLimitResource,
|
|
userLimitResource,
|
|
maxUserLimit
|
|
maxUserLimit
|
|
),
|
|
),
|
|
@@ -1193,18 +1220,22 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
String userName = application.getUser();
|
|
String userName = application.getUser();
|
|
- LOG.debug("User limit computation for " + userName +
|
|
|
|
|
|
+ LOG.debug("User limit computation for " + userName +
|
|
" in queue " + getQueueName() +
|
|
" in queue " + getQueueName() +
|
|
" userLimitPercent=" + userLimit +
|
|
" userLimitPercent=" + userLimit +
|
|
" userLimitFactor=" + userLimitFactor +
|
|
" userLimitFactor=" + userLimitFactor +
|
|
- " required: " + required +
|
|
|
|
- " consumed: " + user.getUsed() +
|
|
|
|
|
|
+ " required: " + required +
|
|
|
|
+ " consumed: " + consumed +
|
|
" user-limit-resource: " + userLimitResource +
|
|
" user-limit-resource: " + userLimitResource +
|
|
- " queueCapacity: " + queueCapacity +
|
|
|
|
|
|
+ " queueCapacity: " + queueCapacity +
|
|
" qconsumed: " + queueUsage.getUsed() +
|
|
" qconsumed: " + queueUsage.getUsed() +
|
|
|
|
+ " consumedRatio: " + totalUserConsumedRatio +
|
|
" currentCapacity: " + currentCapacity +
|
|
" currentCapacity: " + currentCapacity +
|
|
" activeUsers: " + activeUsers +
|
|
" activeUsers: " + activeUsers +
|
|
- " clusterCapacity: " + clusterResource
|
|
|
|
|
|
+ " clusterCapacity: " + clusterResource +
|
|
|
|
+ " resourceByLabel: " + partitionResource +
|
|
|
|
+ " usageratio: " + qUsageRatios.getUsageRatio(nodePartition) +
|
|
|
|
+ " Partition: " + nodePartition
|
|
);
|
|
);
|
|
}
|
|
}
|
|
user.setUserResourceLimit(userLimitResource);
|
|
user.setUserResourceLimit(userLimitResource);
|
|
@@ -1311,6 +1342,42 @@ public class LeafQueue extends AbstractCSQueue {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private synchronized float calculateUserUsageRatio(Resource clusterResource,
|
|
|
|
+ String nodePartition) {
|
|
|
|
+ Resource resourceByLabel =
|
|
|
|
+ labelManager.getResourceByLabel(nodePartition, clusterResource);
|
|
|
|
+ float consumed = 0;
|
|
|
|
+ User user;
|
|
|
|
+ for (Map.Entry<String, User> entry : users.entrySet()) {
|
|
|
|
+ user = entry.getValue();
|
|
|
|
+ consumed += user.resetAndUpdateUsageRatio(resourceCalculator,
|
|
|
|
+ resourceByLabel, nodePartition);
|
|
|
|
+ }
|
|
|
|
+ return consumed;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private synchronized void recalculateQueueUsageRatio(Resource clusterResource,
|
|
|
|
+ String nodePartition) {
|
|
|
|
+ ResourceUsage queueResourceUsage = this.getQueueResourceUsage();
|
|
|
|
+
|
|
|
|
+ if (nodePartition == null) {
|
|
|
|
+ for (String partition : Sets.union(queueCapacities.getNodePartitionsSet(),
|
|
|
|
+ queueResourceUsage.getNodePartitionsSet())) {
|
|
|
|
+ qUsageRatios.setUsageRatio(partition,
|
|
|
|
+ calculateUserUsageRatio(clusterResource, partition));
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ qUsageRatios.setUsageRatio(nodePartition,
|
|
|
|
+ calculateUserUsageRatio(clusterResource, nodePartition));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private synchronized void updateQueueUsageRatio(String nodePartition,
|
|
|
|
+ float delta) {
|
|
|
|
+ qUsageRatios.incUsageRatio(nodePartition, delta);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void completedContainer(Resource clusterResource,
|
|
public void completedContainer(Resource clusterResource,
|
|
FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer,
|
|
FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer,
|
|
@@ -1348,7 +1415,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
removed =
|
|
removed =
|
|
application.containerCompleted(rmContainer, containerStatus,
|
|
application.containerCompleted(rmContainer, containerStatus,
|
|
event, node.getPartition());
|
|
event, node.getPartition());
|
|
-
|
|
|
|
|
|
+
|
|
node.releaseContainer(container);
|
|
node.releaseContainer(container);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1381,6 +1448,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
boolean isIncreasedAllocation) {
|
|
boolean isIncreasedAllocation) {
|
|
super.allocateResource(clusterResource, resource, nodePartition,
|
|
super.allocateResource(clusterResource, resource, nodePartition,
|
|
isIncreasedAllocation);
|
|
isIncreasedAllocation);
|
|
|
|
+ Resource resourceByLabel = labelManager.getResourceByLabel(nodePartition,
|
|
|
|
+ clusterResource);
|
|
|
|
|
|
// handle ignore exclusivity container
|
|
// handle ignore exclusivity container
|
|
if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
|
|
if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
|
|
@@ -1399,6 +1468,12 @@ public class LeafQueue extends AbstractCSQueue {
|
|
String userName = application.getUser();
|
|
String userName = application.getUser();
|
|
User user = getUser(userName);
|
|
User user = getUser(userName);
|
|
user.assignContainer(resource, nodePartition);
|
|
user.assignContainer(resource, nodePartition);
|
|
|
|
+
|
|
|
|
+ // Update usage ratios
|
|
|
|
+ updateQueueUsageRatio(nodePartition,
|
|
|
|
+ user.updateUsageRatio(resourceCalculator, resourceByLabel,
|
|
|
|
+ nodePartition));
|
|
|
|
+
|
|
// Note this is a bit unconventional since it gets the object and modifies
|
|
// Note this is a bit unconventional since it gets the object and modifies
|
|
// it here, rather then using set routine
|
|
// it here, rather then using set routine
|
|
Resources.subtractFrom(application.getHeadroom(), resource); // headroom
|
|
Resources.subtractFrom(application.getHeadroom(), resource); // headroom
|
|
@@ -1419,6 +1494,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
RMContainer rmContainer, boolean isChangeResource) {
|
|
RMContainer rmContainer, boolean isChangeResource) {
|
|
super.releaseResource(clusterResource, resource, nodePartition,
|
|
super.releaseResource(clusterResource, resource, nodePartition,
|
|
isChangeResource);
|
|
isChangeResource);
|
|
|
|
+ Resource resourceByLabel = labelManager.getResourceByLabel(nodePartition,
|
|
|
|
+ clusterResource);
|
|
|
|
|
|
// handle ignore exclusivity container
|
|
// handle ignore exclusivity container
|
|
if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
|
|
if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
|
|
@@ -1438,6 +1515,12 @@ public class LeafQueue extends AbstractCSQueue {
|
|
String userName = application.getUser();
|
|
String userName = application.getUser();
|
|
User user = getUser(userName);
|
|
User user = getUser(userName);
|
|
user.releaseContainer(resource, nodePartition);
|
|
user.releaseContainer(resource, nodePartition);
|
|
|
|
+
|
|
|
|
+ // Update usage ratios
|
|
|
|
+ updateQueueUsageRatio(nodePartition,
|
|
|
|
+ user.updateUsageRatio(resourceCalculator, resourceByLabel,
|
|
|
|
+ nodePartition));
|
|
|
|
+
|
|
metrics.setAvailableResourcesToUser(userName, application.getHeadroom());
|
|
metrics.setAvailableResourcesToUser(userName, application.getHeadroom());
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -1477,7 +1560,10 @@ public class LeafQueue extends AbstractCSQueue {
|
|
// absoluteMaxCapacity now, will be replaced with absoluteMaxAvailCapacity
|
|
// absoluteMaxCapacity now, will be replaced with absoluteMaxAvailCapacity
|
|
// during allocation
|
|
// during allocation
|
|
setQueueResourceLimitsInfo(clusterResource);
|
|
setQueueResourceLimitsInfo(clusterResource);
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ // Update user consumedRatios
|
|
|
|
+ recalculateQueueUsageRatio(clusterResource, null);
|
|
|
|
+
|
|
// Update metrics
|
|
// Update metrics
|
|
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
|
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
|
minimumAllocation, this, labelManager, null);
|
|
minimumAllocation, this, labelManager, null);
|
|
@@ -1529,17 +1615,93 @@ public class LeafQueue extends AbstractCSQueue {
|
|
queueUsage.decAMUsed(nodeLabel, resourceToDec);
|
|
queueUsage.decAMUsed(nodeLabel, resourceToDec);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /*
|
|
|
|
+ * Usage Ratio
|
|
|
|
+ */
|
|
|
|
+ static private class UsageRatios {
|
|
|
|
+ private Map<String, Float> usageRatios;
|
|
|
|
+ private ReadLock readLock;
|
|
|
|
+ private WriteLock writeLock;
|
|
|
|
+
|
|
|
|
+ public UsageRatios() {
|
|
|
|
+ ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
|
|
+ readLock = lock.readLock();
|
|
|
|
+ writeLock = lock.writeLock();
|
|
|
|
+ usageRatios = new HashMap<String, Float>();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void incUsageRatio(String label, float delta) {
|
|
|
|
+ try {
|
|
|
|
+ writeLock.lock();
|
|
|
|
+ Float fl = usageRatios.get(label);
|
|
|
|
+ if (null == fl) {
|
|
|
|
+ fl = new Float(0.0);
|
|
|
|
+ }
|
|
|
|
+ fl += delta;
|
|
|
|
+ usageRatios.put(label, new Float(fl));
|
|
|
|
+ } finally {
|
|
|
|
+ writeLock.unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ float getUsageRatio(String label) {
|
|
|
|
+ try {
|
|
|
|
+ readLock.lock();
|
|
|
|
+ Float f = usageRatios.get(label);
|
|
|
|
+ if (null == f) {
|
|
|
|
+ return 0.0f;
|
|
|
|
+ }
|
|
|
|
+ return f;
|
|
|
|
+ } finally {
|
|
|
|
+ readLock.unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void setUsageRatio(String label, float ratio) {
|
|
|
|
+ try {
|
|
|
|
+ writeLock.lock();
|
|
|
|
+ usageRatios.put(label, new Float(ratio));
|
|
|
|
+ } finally {
|
|
|
|
+ writeLock.unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ public float getUsageRatio(String label) {
|
|
|
|
+ return qUsageRatios.getUsageRatio(label);
|
|
|
|
+ }
|
|
|
|
+
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
public static class User {
|
|
public static class User {
|
|
ResourceUsage userResourceUsage = new ResourceUsage();
|
|
ResourceUsage userResourceUsage = new ResourceUsage();
|
|
volatile Resource userResourceLimit = Resource.newInstance(0, 0);
|
|
volatile Resource userResourceLimit = Resource.newInstance(0, 0);
|
|
int pendingApplications = 0;
|
|
int pendingApplications = 0;
|
|
int activeApplications = 0;
|
|
int activeApplications = 0;
|
|
|
|
+ private UsageRatios userUsageRatios = new UsageRatios();
|
|
|
|
|
|
public ResourceUsage getResourceUsage() {
|
|
public ResourceUsage getResourceUsage() {
|
|
return userResourceUsage;
|
|
return userResourceUsage;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public synchronized float resetAndUpdateUsageRatio(
|
|
|
|
+ ResourceCalculator resourceCalculator,
|
|
|
|
+ Resource resource, String nodePartition) {
|
|
|
|
+ userUsageRatios.setUsageRatio(nodePartition, 0);
|
|
|
|
+ return updateUsageRatio(resourceCalculator, resource, nodePartition);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public synchronized float updateUsageRatio(
|
|
|
|
+ ResourceCalculator resourceCalculator,
|
|
|
|
+ Resource resource, String nodePartition) {
|
|
|
|
+ float delta;
|
|
|
|
+ float newRatio =
|
|
|
|
+ Resources.ratio(resourceCalculator, getUsed(nodePartition), resource);
|
|
|
|
+ delta = newRatio - userUsageRatios.getUsageRatio(nodePartition);
|
|
|
|
+ userUsageRatios.setUsageRatio(nodePartition, newRatio);
|
|
|
|
+ return delta;
|
|
|
|
+ }
|
|
|
|
+
|
|
public Resource getUsed() {
|
|
public Resource getUsed() {
|
|
return userResourceUsage.getUsed();
|
|
return userResourceUsage.getUsed();
|
|
}
|
|
}
|
|
@@ -1677,7 +1839,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
.getSchedulableEntities()) {
|
|
.getSchedulableEntities()) {
|
|
apps.add(pendingApp.getApplicationAttemptId());
|
|
apps.add(pendingApp.getApplicationAttemptId());
|
|
}
|
|
}
|
|
- for (FiCaSchedulerApp app :
|
|
|
|
|
|
+ for (FiCaSchedulerApp app :
|
|
orderingPolicy.getSchedulableEntities()) {
|
|
orderingPolicy.getSchedulableEntities()) {
|
|
apps.add(app.getApplicationAttemptId());
|
|
apps.add(app.getApplicationAttemptId());
|
|
}
|
|
}
|