|
@@ -51,7 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
|
|
|
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.RMContainerState;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractUsersManager;
|
|
|
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.SchedContainerChangeRequest;
|
|
@@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UsersManager.User;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
|
|
@@ -76,7 +77,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPo
|
|
|
import org.apache.hadoop.yarn.server.utils.Lock;
|
|
|
import org.apache.hadoop.yarn.server.utils.Lock.NoLock;
|
|
|
import org.apache.hadoop.yarn.util.SystemClock;
|
|
|
-import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
import java.io.IOException;
|
|
@@ -101,8 +101,6 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
private static final Log LOG = LogFactory.getLog(LeafQueue.class);
|
|
|
|
|
|
private float absoluteUsedCapacity = 0.0f;
|
|
|
- private volatile int userLimit;
|
|
|
- private volatile float userLimitFactor;
|
|
|
|
|
|
protected int maxApplications;
|
|
|
protected volatile int maxApplicationsPerUser;
|
|
@@ -122,14 +120,12 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
private volatile float minimumAllocationFactor;
|
|
|
|
|
|
- private Map<String, User> users = new ConcurrentHashMap<>();
|
|
|
-
|
|
|
private final RecordFactory recordFactory =
|
|
|
RecordFactoryProvider.getRecordFactory(null);
|
|
|
|
|
|
private CapacitySchedulerContext scheduler;
|
|
|
|
|
|
- private final ActiveUsersManager activeUsersManager;
|
|
|
+ private final UsersManager usersManager;
|
|
|
|
|
|
// cache last cluster resource to compute actual capacity
|
|
|
private Resource lastClusterResource = Resources.none();
|
|
@@ -141,10 +137,6 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
private volatile 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
|
|
|
// preemption, key is the partition of the RMContainer allocated on
|
|
|
private Map<String, TreeSet<RMContainer>> ignorePartitionExclusivityRMContainers =
|
|
@@ -159,13 +151,12 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
super(cs, queueName, parent, old);
|
|
|
this.scheduler = cs;
|
|
|
|
|
|
- this.activeUsersManager = new ActiveUsersManager(metrics);
|
|
|
+ this.usersManager = new UsersManager(metrics, this, labelManager, scheduler,
|
|
|
+ resourceCalculator);
|
|
|
|
|
|
// One time initialization is enough since it is static ordering policy
|
|
|
this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps();
|
|
|
|
|
|
- qUsageRatios = new UsageRatios();
|
|
|
-
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("LeafQueue:" + " name=" + queueName
|
|
|
+ ", fullname=" + getQueuePath());
|
|
@@ -197,8 +188,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
setOrderingPolicy(
|
|
|
conf.<FiCaSchedulerApp>getAppOrderingPolicy(getQueuePath()));
|
|
|
|
|
|
- userLimit = conf.getUserLimit(getQueuePath());
|
|
|
- userLimitFactor = conf.getUserLimitFactor(getQueuePath());
|
|
|
+ usersManager.setUserLimit(conf.getUserLimit(getQueuePath()));
|
|
|
+ usersManager.setUserLimitFactor(conf.getUserLimitFactor(getQueuePath()));
|
|
|
|
|
|
maxApplications = conf.getMaximumApplicationsPerQueue(getQueuePath());
|
|
|
if (maxApplications < 0) {
|
|
@@ -212,7 +203,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
}
|
|
|
maxApplicationsPerUser = Math.min(maxApplications,
|
|
|
- (int) (maxApplications * (userLimit / 100.0f) * userLimitFactor));
|
|
|
+ (int) (maxApplications * (usersManager.getUserLimit() / 100.0f)
|
|
|
+ * usersManager.getUserLimitFactor()));
|
|
|
|
|
|
maxAMResourcePerQueuePercent =
|
|
|
conf.getMaximumApplicationMasterResourcePerQueuePercent(
|
|
@@ -271,8 +263,9 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
+ queueCapacities.getAbsoluteMaximumCapacity()
|
|
|
+ " [= 1.0 maximumCapacity undefined, "
|
|
|
+ "(parentAbsoluteMaxCapacity * maximumCapacity) / 100 otherwise ]"
|
|
|
- + "\n" + "userLimit = " + userLimit + " [= configuredUserLimit ]"
|
|
|
- + "\n" + "userLimitFactor = " + userLimitFactor
|
|
|
+ + "\n" + "userLimit = " + usersManager.getUserLimit()
|
|
|
+ + " [= configuredUserLimit ]" + "\n" + "userLimitFactor = "
|
|
|
+ + usersManager.getUserLimitFactor()
|
|
|
+ " [= configuredUserLimitFactor ]" + "\n" + "maxApplications = "
|
|
|
+ maxApplications
|
|
|
+ " [= configuredMaximumSystemApplicationsPerQueue or"
|
|
@@ -336,9 +329,17 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
return maxApplicationsPerUser;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ *
|
|
|
+ * @return UsersManager instance.
|
|
|
+ */
|
|
|
+ public UsersManager getUsersManager() {
|
|
|
+ return usersManager;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
- public ActiveUsersManager getActiveUsersManager() {
|
|
|
- return activeUsersManager;
|
|
|
+ public AbstractUsersManager getAbstractUsersManager() {
|
|
|
+ return usersManager;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -352,7 +353,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
void setUserLimit(int userLimit) {
|
|
|
- this.userLimit = userLimit;
|
|
|
+ usersManager.setUserLimit(userLimit);
|
|
|
+ usersManager.userLimitNeedsRecompute();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -361,7 +363,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
void setUserLimitFactor(float userLimitFactor) {
|
|
|
- this.userLimitFactor = userLimitFactor;
|
|
|
+ usersManager.setUserLimitFactor(userLimitFactor);
|
|
|
+ usersManager.userLimitNeedsRecompute();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -422,12 +425,12 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
@Private
|
|
|
public int getUserLimit() {
|
|
|
- return userLimit;
|
|
|
+ return usersManager.getUserLimit();
|
|
|
}
|
|
|
|
|
|
@Private
|
|
|
public float getUserLimitFactor() {
|
|
|
- return userLimitFactor;
|
|
|
+ return usersManager.getUserLimitFactor();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -477,44 +480,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
@VisibleForTesting
|
|
|
public User getUser(String userName) {
|
|
|
- return users.get(userName);
|
|
|
- }
|
|
|
-
|
|
|
- // Get and add user if absent
|
|
|
- private User getUserAndAddIfAbsent(String userName) {
|
|
|
- try {
|
|
|
- writeLock.lock();
|
|
|
- User u = users.get(userName);
|
|
|
- if (null == u) {
|
|
|
- u = new User();
|
|
|
- users.put(userName, u);
|
|
|
- }
|
|
|
- return u;
|
|
|
- } finally {
|
|
|
- writeLock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return an ArrayList of UserInfo objects who are active in this queue
|
|
|
- */
|
|
|
- public ArrayList<UserInfo> getUsers() {
|
|
|
- try {
|
|
|
- readLock.lock();
|
|
|
- ArrayList<UserInfo> usersToReturn = new ArrayList<UserInfo>();
|
|
|
- for (Map.Entry<String, User> entry : users.entrySet()) {
|
|
|
- User user = entry.getValue();
|
|
|
- usersToReturn.add(
|
|
|
- new UserInfo(entry.getKey(), Resources.clone(user.getAllUsed()),
|
|
|
- user.getActiveApplications(), user.getPendingApplications(),
|
|
|
- Resources.clone(user.getConsumedAMResources()),
|
|
|
- Resources.clone(user.getUserResourceLimit()),
|
|
|
- user.getResourceUsage()));
|
|
|
- }
|
|
|
- return usersToReturn;
|
|
|
- } finally {
|
|
|
- readLock.unlock();
|
|
|
- }
|
|
|
+ return usersManager.getUser(userName);
|
|
|
}
|
|
|
|
|
|
@Private
|
|
@@ -575,7 +541,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
// TODO, should use getUser, use this method just to avoid UT failure
|
|
|
// which is caused by wrong invoking order, will fix UT separately
|
|
|
- User user = getUserAndAddIfAbsent(userName);
|
|
|
+ User user = usersManager.getUserAndAddIfAbsent(userName);
|
|
|
|
|
|
// Add the attempt to our data-structures
|
|
|
addApplicationAttempt(application, user);
|
|
@@ -632,7 +598,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
|
|
|
// Check submission limits for the user on this queue
|
|
|
- User user = getUserAndAddIfAbsent(userName);
|
|
|
+ User user = usersManager.getUserAndAddIfAbsent(userName);
|
|
|
if (user.getTotalApplications() >= getMaxApplicationsPerUser()) {
|
|
|
String msg = "Queue " + getQueuePath() + " already has " + user
|
|
|
.getTotalApplications() + " applications from user " + userName
|
|
@@ -682,19 +648,21 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
* the absolute queue capacity (per partition) instead of the max and is
|
|
|
* modified by the userlimit and the userlimit factor as is the userlimit
|
|
|
*/
|
|
|
- float effectiveUserLimit = Math.max(userLimit / 100.0f,
|
|
|
- 1.0f / Math.max(getActiveUsersManager().getNumActiveUsers(), 1));
|
|
|
+ float effectiveUserLimit = Math.max(usersManager.getUserLimit() / 100.0f,
|
|
|
+ 1.0f / Math.max(getAbstractUsersManager().getNumActiveUsers(), 1));
|
|
|
|
|
|
- Resource queuePartitionResource = Resources.multiplyAndNormalizeUp(
|
|
|
- resourceCalculator,
|
|
|
- labelManager.getResourceByLabel(nodePartition, lastClusterResource),
|
|
|
- queueCapacities.getAbsoluteCapacity(nodePartition),
|
|
|
- minimumAllocation);
|
|
|
+ Resource queuePartitionResource = Resources
|
|
|
+ .multiplyAndNormalizeUp(resourceCalculator,
|
|
|
+ labelManager.getResourceByLabel(nodePartition,
|
|
|
+ lastClusterResource),
|
|
|
+ queueCapacities.getAbsoluteCapacity(nodePartition),
|
|
|
+ minimumAllocation);
|
|
|
|
|
|
Resource userAMLimit = Resources.multiplyAndNormalizeUp(
|
|
|
resourceCalculator, queuePartitionResource,
|
|
|
queueCapacities.getMaxAMResourcePercentage(nodePartition)
|
|
|
- * effectiveUserLimit * userLimitFactor, minimumAllocation);
|
|
|
+ * effectiveUserLimit * usersManager.getUserLimitFactor(),
|
|
|
+ minimumAllocation);
|
|
|
return Resources.lessThanOrEqual(resourceCalculator, lastClusterResource,
|
|
|
userAMLimit, getAMResourceLimitPerPartition(nodePartition)) ?
|
|
|
userAMLimit :
|
|
@@ -910,7 +878,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
@Override
|
|
|
public void finishApplication(ApplicationId application, String user) {
|
|
|
// Inform the activeUsersManager
|
|
|
- activeUsersManager.deactivateApplication(user, application);
|
|
|
+ usersManager.deactivateApplication(user, application);
|
|
|
|
|
|
appFinished();
|
|
|
|
|
@@ -932,7 +900,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
// TODO, should use getUser, use this method just to avoid UT failure
|
|
|
// which is caused by wrong invoking order, will fix UT separately
|
|
|
- User user = getUserAndAddIfAbsent(userName);
|
|
|
+ User user = usersManager.getUserAndAddIfAbsent(userName);
|
|
|
|
|
|
String partitionName = application.getAppAMNodePartitionName();
|
|
|
boolean wasActive = orderingPolicy.removeSchedulableEntity(application);
|
|
@@ -950,7 +918,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
user.finishApplication(wasActive);
|
|
|
if (user.getTotalApplications() == 0) {
|
|
|
- users.remove(application.getUser());
|
|
|
+ usersManager.removeUser(application.getUser());
|
|
|
}
|
|
|
|
|
|
// Check if we can activate more applications
|
|
@@ -1291,7 +1259,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
Resource clusterResource, FiCaSchedulerApp application,
|
|
|
String partition) {
|
|
|
return getHeadroom(user, queueCurrentLimit, clusterResource,
|
|
|
- computeUserLimit(application.getUser(), clusterResource, user,
|
|
|
+ getResourceLimitForActiveUsers(application.getUser(), clusterResource,
|
|
|
partition, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY),
|
|
|
partition);
|
|
|
}
|
|
@@ -1365,7 +1333,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
// Compute user limit respect requested labels,
|
|
|
// TODO, need consider headroom respect labels also
|
|
|
Resource userLimit =
|
|
|
- computeUserLimit(application.getUser(), clusterResource, queueUser,
|
|
|
+ getResourceLimitForActiveUsers(application.getUser(), clusterResource,
|
|
|
nodePartition, schedulingMode);
|
|
|
|
|
|
setQueueResourceLimitsInfo(clusterResource);
|
|
@@ -1375,11 +1343,11 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
clusterResource, userLimit, nodePartition);
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Headroom calculation for user " + user + ": " +
|
|
|
- " userLimit=" + userLimit +
|
|
|
- " queueMaxAvailRes=" + cachedResourceLimitsForHeadroom.getLimit() +
|
|
|
- " consumed=" + queueUser.getUsed() +
|
|
|
- " headroom=" + headroom);
|
|
|
+ LOG.debug("Headroom calculation for user " + user + ": " + " userLimit="
|
|
|
+ + userLimit + " queueMaxAvailRes="
|
|
|
+ + cachedResourceLimitsForHeadroom.getLimit() + " consumed="
|
|
|
+ + queueUser.getUsed() + " headroom=" + headroom + " partition="
|
|
|
+ + nodePartition);
|
|
|
}
|
|
|
|
|
|
CapacityHeadroomProvider headroomProvider = new CapacityHeadroomProvider(
|
|
@@ -1407,129 +1375,46 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
return rackLocalityFullReset;
|
|
|
}
|
|
|
|
|
|
- @Lock(NoLock.class)
|
|
|
- private Resource computeUserLimit(String userName,
|
|
|
- Resource clusterResource, User user,
|
|
|
- String nodePartition, SchedulingMode schedulingMode) {
|
|
|
- Resource partitionResource = labelManager.getResourceByLabel(nodePartition,
|
|
|
- clusterResource);
|
|
|
-
|
|
|
- // What is our current capacity?
|
|
|
- // * It is equal to the max(required, queue-capacity) if
|
|
|
- // we're running below capacity. The 'max' ensures that jobs in queues
|
|
|
- // with miniscule capacity (< 1 slot) make progress
|
|
|
- // * If we're running over capacity, then its
|
|
|
- // (usedResources + required) (which extra resources we are allocating)
|
|
|
- Resource queueCapacity =
|
|
|
- Resources.multiplyAndNormalizeUp(resourceCalculator,
|
|
|
- partitionResource,
|
|
|
- queueCapacities.getAbsoluteCapacity(nodePartition),
|
|
|
- minimumAllocation);
|
|
|
-
|
|
|
- // Assume we have required resource equals to minimumAllocation, this can
|
|
|
- // make sure user limit can continuously increase till queueMaxResource
|
|
|
- // reached.
|
|
|
- Resource required = minimumAllocation;
|
|
|
-
|
|
|
- // Allow progress for queues with miniscule capacity
|
|
|
- queueCapacity =
|
|
|
- Resources.max(
|
|
|
- resourceCalculator, partitionResource,
|
|
|
- queueCapacity,
|
|
|
- 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 =
|
|
|
- Resources.lessThan(resourceCalculator, partitionResource, consumed,
|
|
|
- queueCapacity) ? queueCapacity : Resources.add(consumed, required);
|
|
|
- // Never allow a single user to take more than the
|
|
|
- // queue's configured capacity * user-limit-factor.
|
|
|
- // Also, the queue's configured capacity should be higher than
|
|
|
- // queue-hard-limit * ulMin
|
|
|
-
|
|
|
- final int activeUsers = activeUsersManager.getNumActiveUsers();
|
|
|
-
|
|
|
- // User limit resource is determined by:
|
|
|
- // max{currentCapacity / #activeUsers, currentCapacity *
|
|
|
- // user-limit-percentage%)
|
|
|
- Resource userLimitResource = Resources.max(
|
|
|
- resourceCalculator, partitionResource,
|
|
|
- Resources.divideAndCeil(
|
|
|
- resourceCalculator, currentCapacity, activeUsers),
|
|
|
- Resources.divideAndCeil(
|
|
|
- resourceCalculator,
|
|
|
- Resources.multiplyAndRoundDown(
|
|
|
- currentCapacity, userLimit),
|
|
|
- 100)
|
|
|
- );
|
|
|
-
|
|
|
- // User limit is capped by maxUserLimit
|
|
|
- // - maxUserLimit = queueCapacity * user-limit-factor (RESPECT_PARTITION_EXCLUSIVITY)
|
|
|
- // - maxUserLimit = total-partition-resource (IGNORE_PARTITION_EXCLUSIVITY)
|
|
|
- //
|
|
|
- // In IGNORE_PARTITION_EXCLUSIVITY mode, if a queue cannot access a
|
|
|
- // partition, its guaranteed resource on that partition is 0. And
|
|
|
- // user-limit-factor computation is based on queue's guaranteed capacity. So
|
|
|
- // we will not cap user-limit as well as used resource when doing
|
|
|
- // IGNORE_PARTITION_EXCLUSIVITY allocation.
|
|
|
- Resource maxUserLimit = Resources.none();
|
|
|
- if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
|
|
|
- maxUserLimit =
|
|
|
- Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor);
|
|
|
- } else if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
|
|
|
- maxUserLimit = partitionResource;
|
|
|
- }
|
|
|
-
|
|
|
- // Cap final user limit with maxUserLimit
|
|
|
- userLimitResource =
|
|
|
- Resources.roundUp(
|
|
|
- resourceCalculator,
|
|
|
- Resources.min(
|
|
|
- resourceCalculator, partitionResource,
|
|
|
- userLimitResource,
|
|
|
- maxUserLimit
|
|
|
- ),
|
|
|
- minimumAllocation);
|
|
|
+ /**
|
|
|
+ *
|
|
|
+ * @param userName
|
|
|
+ * Name of user who has submitted one/more app to given queue.
|
|
|
+ * @param clusterResource
|
|
|
+ * total cluster resource
|
|
|
+ * @param nodePartition
|
|
|
+ * partition name
|
|
|
+ * @param schedulingMode
|
|
|
+ * scheduling mode
|
|
|
+ * RESPECT_PARTITION_EXCLUSIVITY/IGNORE_PARTITION_EXCLUSIVITY
|
|
|
+ * @return Computed User Limit
|
|
|
+ */
|
|
|
+ public Resource getResourceLimitForActiveUsers(String userName,
|
|
|
+ Resource clusterResource, String nodePartition,
|
|
|
+ SchedulingMode schedulingMode) {
|
|
|
+ return usersManager.getComputedResourceLimitForActiveUsers(userName,
|
|
|
+ clusterResource, nodePartition, schedulingMode);
|
|
|
+ }
|
|
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("User limit computation for " + userName +
|
|
|
- " in queue " + getQueueName() +
|
|
|
- " userLimitPercent=" + userLimit +
|
|
|
- " userLimitFactor=" + userLimitFactor +
|
|
|
- " required: " + required +
|
|
|
- " consumed: " + consumed +
|
|
|
- " user-limit-resource: " + userLimitResource +
|
|
|
- " queueCapacity: " + queueCapacity +
|
|
|
- " qconsumed: " + queueUsage.getUsed() +
|
|
|
- " consumedRatio: " + totalUserConsumedRatio +
|
|
|
- " currentCapacity: " + currentCapacity +
|
|
|
- " activeUsers: " + activeUsers +
|
|
|
- " clusterCapacity: " + clusterResource +
|
|
|
- " resourceByLabel: " + partitionResource +
|
|
|
- " usageratio: " + qUsageRatios.getUsageRatio(nodePartition) +
|
|
|
- " Partition: " + nodePartition
|
|
|
- );
|
|
|
- }
|
|
|
- user.setUserResourceLimit(userLimitResource);
|
|
|
- return userLimitResource;
|
|
|
+ /**
|
|
|
+ *
|
|
|
+ * @param userName
|
|
|
+ * Name of user who has submitted one/more app to given queue.
|
|
|
+ * @param clusterResource
|
|
|
+ * total cluster resource
|
|
|
+ * @param nodePartition
|
|
|
+ * partition name
|
|
|
+ * @param schedulingMode
|
|
|
+ * scheduling mode
|
|
|
+ * RESPECT_PARTITION_EXCLUSIVITY/IGNORE_PARTITION_EXCLUSIVITY
|
|
|
+ * @return Computed User Limit
|
|
|
+ */
|
|
|
+ public Resource getResourceLimitForAllUsers(String userName,
|
|
|
+ Resource clusterResource, String nodePartition,
|
|
|
+ SchedulingMode schedulingMode) {
|
|
|
+ return usersManager.getComputedResourceLimitForAllUsers(userName,
|
|
|
+ clusterResource, nodePartition, schedulingMode);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Private
|
|
|
protected boolean canAssignToUser(Resource clusterResource,
|
|
|
String userName, Resource limit, FiCaSchedulerApp application,
|
|
@@ -1600,52 +1485,34 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private float calculateUserUsageRatio(Resource clusterResource,
|
|
|
- String nodePartition) {
|
|
|
- try {
|
|
|
- writeLock.lock();
|
|
|
- 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;
|
|
|
- } finally {
|
|
|
- writeLock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void recalculateQueueUsageRatio(Resource clusterResource,
|
|
|
+ /**
|
|
|
+ * Recalculate QueueUsage Ratio.
|
|
|
+ *
|
|
|
+ * @param clusterResource
|
|
|
+ * Total Cluster Resource
|
|
|
+ * @param nodePartition
|
|
|
+ * Partition
|
|
|
+ */
|
|
|
+ public void recalculateQueueUsageRatio(Resource clusterResource,
|
|
|
String nodePartition) {
|
|
|
try {
|
|
|
writeLock.lock();
|
|
|
- ResourceUsage queueResourceUsage = this.getQueueResourceUsage();
|
|
|
+ ResourceUsage queueResourceUsage = getQueueResourceUsage();
|
|
|
|
|
|
if (nodePartition == null) {
|
|
|
for (String partition : Sets.union(
|
|
|
- queueCapacities.getNodePartitionsSet(),
|
|
|
+ getQueueCapacities().getNodePartitionsSet(),
|
|
|
queueResourceUsage.getNodePartitionsSet())) {
|
|
|
- qUsageRatios.setUsageRatio(partition,
|
|
|
- calculateUserUsageRatio(clusterResource, partition));
|
|
|
+ usersManager.updateUsageRatio(partition, clusterResource);
|
|
|
}
|
|
|
- } else{
|
|
|
- qUsageRatios.setUsageRatio(nodePartition,
|
|
|
- calculateUserUsageRatio(clusterResource, nodePartition));
|
|
|
+ } else {
|
|
|
+ usersManager.updateUsageRatio(nodePartition, clusterResource);
|
|
|
}
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void updateQueueUsageRatio(String nodePartition,
|
|
|
- float delta) {
|
|
|
- qUsageRatios.incUsageRatio(nodePartition, delta);
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public void completedContainer(Resource clusterResource,
|
|
|
FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer,
|
|
@@ -1708,8 +1575,6 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
try {
|
|
|
writeLock.lock();
|
|
|
super.allocateResource(clusterResource, resource, nodePartition);
|
|
|
- Resource resourceByLabel = labelManager.getResourceByLabel(nodePartition,
|
|
|
- clusterResource);
|
|
|
|
|
|
// handle ignore exclusivity container
|
|
|
if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
|
|
@@ -1728,16 +1593,9 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
// Update user metrics
|
|
|
String userName = application.getUser();
|
|
|
|
|
|
- // TODO, should use getUser, use this method just to avoid UT failure
|
|
|
- // which is caused by wrong invoking order, will fix UT separately
|
|
|
- User user = getUserAndAddIfAbsent(userName);
|
|
|
-
|
|
|
- user.assignContainer(resource, nodePartition);
|
|
|
-
|
|
|
- // Update usage ratios
|
|
|
- updateQueueUsageRatio(nodePartition,
|
|
|
- user.updateUsageRatio(resourceCalculator, resourceByLabel,
|
|
|
- nodePartition));
|
|
|
+ // Increment user's resource usage.
|
|
|
+ User user = usersManager.updateUserResourceUsage(userName, resource,
|
|
|
+ nodePartition, true);
|
|
|
|
|
|
// Note this is a bit unconventional since it gets the object and modifies
|
|
|
// it here, rather then using set routine
|
|
@@ -1746,9 +1604,10 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
userName, application.getHeadroom());
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug(getQueueName() + " user=" + userName + " used=" + queueUsage
|
|
|
- .getUsed() + " numContainers=" + numContainers + " headroom = "
|
|
|
- + application.getHeadroom() + " user-resources=" + user.getUsed());
|
|
|
+ LOG.debug(getQueueName() + " user=" + userName + " used="
|
|
|
+ + queueUsage.getUsed(nodePartition) + " numContainers="
|
|
|
+ + numContainers + " headroom = " + application.getHeadroom()
|
|
|
+ + " user-resources=" + user.getUsed());
|
|
|
}
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
@@ -1761,8 +1620,6 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
try {
|
|
|
writeLock.lock();
|
|
|
super.releaseResource(clusterResource, resource, nodePartition);
|
|
|
- Resource resourceByLabel = labelManager.getResourceByLabel(nodePartition,
|
|
|
- clusterResource);
|
|
|
|
|
|
// handle ignore exclusivity container
|
|
|
if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
|
|
@@ -1780,13 +1637,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
// Update user metrics
|
|
|
String userName = application.getUser();
|
|
|
- User user = getUserAndAddIfAbsent(userName);
|
|
|
- user.releaseContainer(resource, nodePartition);
|
|
|
-
|
|
|
- // Update usage ratios
|
|
|
- updateQueueUsageRatio(nodePartition,
|
|
|
- user.updateUsageRatio(resourceCalculator, resourceByLabel,
|
|
|
- nodePartition));
|
|
|
+ User user = usersManager.updateUserResourceUsage(userName, resource,
|
|
|
+ nodePartition, false);
|
|
|
|
|
|
metrics.setAvailableResourcesToUser(nodePartition,
|
|
|
userName, application.getHeadroom());
|
|
@@ -1846,6 +1698,10 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
// activate the pending applications if possible
|
|
|
activateApplications();
|
|
|
|
|
|
+ // In case of any resource change, invalidate recalculateULCount to clear
|
|
|
+ // the computed user-limit.
|
|
|
+ usersManager.userLimitNeedsRecompute();
|
|
|
+
|
|
|
// Update application properties
|
|
|
for (FiCaSchedulerApp application : orderingPolicy
|
|
|
.getSchedulableEntities()) {
|
|
@@ -1861,16 +1717,16 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
@Override
|
|
|
public void incUsedResource(String nodeLabel, Resource resourceToInc,
|
|
|
SchedulerApplicationAttempt application) {
|
|
|
- getUser(application.getUser()).getResourceUsage().incUsed(nodeLabel,
|
|
|
- resourceToInc);
|
|
|
+ usersManager.updateUserResourceUsage(application.getUser(), resourceToInc,
|
|
|
+ nodeLabel, true);
|
|
|
super.incUsedResource(nodeLabel, resourceToInc, application);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void decUsedResource(String nodeLabel, Resource resourceToDec,
|
|
|
SchedulerApplicationAttempt application) {
|
|
|
- getUser(application.getUser()).getResourceUsage().decUsed(nodeLabel,
|
|
|
- resourceToDec);
|
|
|
+ usersManager.updateUserResourceUsage(application.getUser(), resourceToDec,
|
|
|
+ nodeLabel, false);
|
|
|
super.decUsedResource(nodeLabel, resourceToDec, application);
|
|
|
}
|
|
|
|
|
@@ -1890,191 +1746,6 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
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
|
|
|
- public static class User {
|
|
|
- ResourceUsage userResourceUsage = new ResourceUsage();
|
|
|
- volatile Resource userResourceLimit = Resource.newInstance(0, 0);
|
|
|
- volatile int pendingApplications = 0;
|
|
|
- volatile int activeApplications = 0;
|
|
|
- private UsageRatios userUsageRatios = new UsageRatios();
|
|
|
- private WriteLock writeLock;
|
|
|
-
|
|
|
- User() {
|
|
|
- ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
|
- // Nobody uses read-lock now, will add it when necessary
|
|
|
- writeLock = lock.writeLock();
|
|
|
- }
|
|
|
-
|
|
|
- public ResourceUsage getResourceUsage() {
|
|
|
- return userResourceUsage;
|
|
|
- }
|
|
|
-
|
|
|
- public float resetAndUpdateUsageRatio(
|
|
|
- ResourceCalculator resourceCalculator,
|
|
|
- Resource resource, String nodePartition) {
|
|
|
- try {
|
|
|
- writeLock.lock();
|
|
|
- userUsageRatios.setUsageRatio(nodePartition, 0);
|
|
|
- return updateUsageRatio(resourceCalculator, resource, nodePartition);
|
|
|
- } finally {
|
|
|
- writeLock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public float updateUsageRatio(
|
|
|
- ResourceCalculator resourceCalculator,
|
|
|
- Resource resource, String nodePartition) {
|
|
|
- try {
|
|
|
- writeLock.lock();
|
|
|
- float delta;
|
|
|
- float newRatio = Resources.ratio(resourceCalculator,
|
|
|
- getUsed(nodePartition), resource);
|
|
|
- delta = newRatio - userUsageRatios.getUsageRatio(nodePartition);
|
|
|
- userUsageRatios.setUsageRatio(nodePartition, newRatio);
|
|
|
- return delta;
|
|
|
- } finally {
|
|
|
- writeLock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public Resource getUsed() {
|
|
|
- return userResourceUsage.getUsed();
|
|
|
- }
|
|
|
-
|
|
|
- public Resource getAllUsed() {
|
|
|
- return userResourceUsage.getAllUsed();
|
|
|
- }
|
|
|
-
|
|
|
- public Resource getUsed(String label) {
|
|
|
- return userResourceUsage.getUsed(label);
|
|
|
- }
|
|
|
-
|
|
|
- public int getPendingApplications() {
|
|
|
- return pendingApplications;
|
|
|
- }
|
|
|
-
|
|
|
- public int getActiveApplications() {
|
|
|
- return activeApplications;
|
|
|
- }
|
|
|
-
|
|
|
- public Resource getConsumedAMResources() {
|
|
|
- return userResourceUsage.getAMUsed();
|
|
|
- }
|
|
|
-
|
|
|
- public Resource getConsumedAMResources(String label) {
|
|
|
- return userResourceUsage.getAMUsed(label);
|
|
|
- }
|
|
|
-
|
|
|
- public int getTotalApplications() {
|
|
|
- return getPendingApplications() + getActiveApplications();
|
|
|
- }
|
|
|
-
|
|
|
- public void submitApplication() {
|
|
|
- try {
|
|
|
- writeLock.lock();
|
|
|
- ++pendingApplications;
|
|
|
- } finally {
|
|
|
- writeLock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public void activateApplication() {
|
|
|
- try {
|
|
|
- writeLock.lock();
|
|
|
- --pendingApplications;
|
|
|
- ++activeApplications;
|
|
|
- } finally {
|
|
|
- writeLock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public void finishApplication(boolean wasActive) {
|
|
|
- try {
|
|
|
- writeLock.lock();
|
|
|
- if (wasActive) {
|
|
|
- --activeApplications;
|
|
|
- } else{
|
|
|
- --pendingApplications;
|
|
|
- }
|
|
|
- } finally {
|
|
|
- writeLock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public void assignContainer(Resource resource, String nodePartition) {
|
|
|
- userResourceUsage.incUsed(nodePartition, resource);
|
|
|
- }
|
|
|
-
|
|
|
- public void releaseContainer(Resource resource, String nodePartition) {
|
|
|
- userResourceUsage.decUsed(nodePartition, resource);
|
|
|
- }
|
|
|
-
|
|
|
- public Resource getUserResourceLimit() {
|
|
|
- return userResourceLimit;
|
|
|
- }
|
|
|
-
|
|
|
- public void setUserResourceLimit(Resource userResourceLimit) {
|
|
|
- this.userResourceLimit = userResourceLimit;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public void recoverContainer(Resource clusterResource,
|
|
|
SchedulerApplicationAttempt attempt, RMContainer rmContainer) {
|
|
@@ -2144,9 +1815,9 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
* excessive preemption.
|
|
|
* @return Total pending resource considering user limit
|
|
|
*/
|
|
|
-
|
|
|
public Resource getTotalPendingResourcesConsideringUserLimit(
|
|
|
- Resource clusterResources, String partition, boolean deductReservedFromPending) {
|
|
|
+ Resource clusterResources, String partition,
|
|
|
+ boolean deductReservedFromPending) {
|
|
|
try {
|
|
|
readLock.lock();
|
|
|
Map<String, Resource> userNameToHeadroom =
|
|
@@ -2157,8 +1828,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
if (!userNameToHeadroom.containsKey(userName)) {
|
|
|
User user = getUser(userName);
|
|
|
Resource headroom = Resources.subtract(
|
|
|
- computeUserLimit(app.getUser(), clusterResources, user, partition,
|
|
|
- SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY),
|
|
|
+ getResourceLimitForActiveUsers(app.getUser(), clusterResources,
|
|
|
+ partition, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY),
|
|
|
user.getUsed(partition));
|
|
|
// Make sure headroom is not negative.
|
|
|
headroom = Resources.componentwiseMax(headroom, Resources.none());
|
|
@@ -2188,16 +1859,6 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
}
|
|
|
|
|
|
- public synchronized Resource getUserLimitPerUser(String userName,
|
|
|
- Resource resources, String partition) {
|
|
|
-
|
|
|
- // Check user resource limit
|
|
|
- User user = getUser(userName);
|
|
|
-
|
|
|
- return computeUserLimit(userName, resources, user, partition,
|
|
|
- SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public void collectSchedulerApplications(
|
|
|
Collection<ApplicationAttemptId> apps) {
|