YARN-3140. Improve locks in AbstractCSQueue/LeafQueue/ParentQueue. Contributed by Wangda Tan
(cherry picked from commit 2b66d9ec5b
)
This commit is contained in:
parent
4e376f162f
commit
3acd30df71
|
@ -537,4 +537,14 @@
|
||||||
</Or>
|
</Or>
|
||||||
<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD" />
|
<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD" />
|
||||||
</Match>
|
</Match>
|
||||||
|
|
||||||
|
<!-- Ignore VO_VOLATILE_INCREMENT, they will be protected by writeLock -->
|
||||||
|
<Match>
|
||||||
|
<Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue$User" />
|
||||||
|
<Or>
|
||||||
|
<Field name="pendingApplications" />
|
||||||
|
<Field name="activeApplications" />
|
||||||
|
</Or>
|
||||||
|
<Bug pattern="VO_VOLATILE_INCREMENT" />
|
||||||
|
</Match>
|
||||||
</FindBugsFilter>
|
</FindBugsFilter>
|
||||||
|
|
|
@ -24,6 +24,7 @@ import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
import org.apache.commons.lang.StringUtils;
|
import org.apache.commons.lang.StringUtils;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
|
@ -60,25 +61,25 @@ import com.google.common.collect.Sets;
|
||||||
|
|
||||||
public abstract class AbstractCSQueue implements CSQueue {
|
public abstract class AbstractCSQueue implements CSQueue {
|
||||||
private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class);
|
private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class);
|
||||||
CSQueue parent;
|
volatile CSQueue parent;
|
||||||
final String queueName;
|
final String queueName;
|
||||||
volatile int numContainers;
|
volatile int numContainers;
|
||||||
|
|
||||||
final Resource minimumAllocation;
|
final Resource minimumAllocation;
|
||||||
volatile Resource maximumAllocation;
|
volatile Resource maximumAllocation;
|
||||||
QueueState state;
|
volatile QueueState state;
|
||||||
final CSQueueMetrics metrics;
|
final CSQueueMetrics metrics;
|
||||||
protected final PrivilegedEntity queueEntity;
|
protected final PrivilegedEntity queueEntity;
|
||||||
|
|
||||||
final ResourceCalculator resourceCalculator;
|
final ResourceCalculator resourceCalculator;
|
||||||
Set<String> accessibleLabels;
|
Set<String> accessibleLabels;
|
||||||
RMNodeLabelsManager labelManager;
|
final RMNodeLabelsManager labelManager;
|
||||||
String defaultLabelExpression;
|
String defaultLabelExpression;
|
||||||
|
|
||||||
Map<AccessType, AccessControlList> acls =
|
Map<AccessType, AccessControlList> acls =
|
||||||
new HashMap<AccessType, AccessControlList>();
|
new HashMap<AccessType, AccessControlList>();
|
||||||
volatile boolean reservationsContinueLooking;
|
volatile boolean reservationsContinueLooking;
|
||||||
private boolean preemptionDisabled;
|
private volatile boolean preemptionDisabled;
|
||||||
|
|
||||||
// Track resource usage-by-label like used-resource/pending-resource, etc.
|
// Track resource usage-by-label like used-resource/pending-resource, etc.
|
||||||
volatile ResourceUsage queueUsage;
|
volatile ResourceUsage queueUsage;
|
||||||
|
@ -94,6 +95,9 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
|
|
||||||
protected ActivitiesManager activitiesManager;
|
protected ActivitiesManager activitiesManager;
|
||||||
|
|
||||||
|
protected ReentrantReadWriteLock.ReadLock readLock;
|
||||||
|
protected ReentrantReadWriteLock.WriteLock writeLock;
|
||||||
|
|
||||||
public AbstractCSQueue(CapacitySchedulerContext cs,
|
public AbstractCSQueue(CapacitySchedulerContext cs,
|
||||||
String queueName, CSQueue parent, CSQueue old) throws IOException {
|
String queueName, CSQueue parent, CSQueue old) throws IOException {
|
||||||
this.labelManager = cs.getRMContext().getNodeLabelManager();
|
this.labelManager = cs.getRMContext().getNodeLabelManager();
|
||||||
|
@ -117,6 +121,10 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
|
|
||||||
// initialize QueueCapacities
|
// initialize QueueCapacities
|
||||||
queueCapacities = new QueueCapacities(parent == null);
|
queueCapacities = new QueueCapacities(parent == null);
|
||||||
|
|
||||||
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||||
|
readLock = lock.readLock();
|
||||||
|
writeLock = lock.writeLock();
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void setupConfigurableCapacities() {
|
protected void setupConfigurableCapacities() {
|
||||||
|
@ -128,12 +136,12 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized float getCapacity() {
|
public float getCapacity() {
|
||||||
return queueCapacities.getCapacity();
|
return queueCapacities.getCapacity();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized float getAbsoluteCapacity() {
|
public float getAbsoluteCapacity() {
|
||||||
return queueCapacities.getAbsoluteCapacity();
|
return queueCapacities.getAbsoluteCapacity();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -167,7 +175,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized QueueState getState() {
|
public QueueState getState() {
|
||||||
return state;
|
return state;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -187,13 +195,13 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized CSQueue getParent() {
|
public CSQueue getParent() {
|
||||||
return parent;
|
return parent;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void setParent(CSQueue newParentQueue) {
|
public void setParent(CSQueue newParentQueue) {
|
||||||
this.parent = (ParentQueue)newParentQueue;
|
this.parent = newParentQueue;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Set<String> getAccessibleNodeLabels() {
|
public Set<String> getAccessibleNodeLabels() {
|
||||||
|
@ -221,18 +229,22 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
* Set maximum capacity - used only for testing.
|
* Set maximum capacity - used only for testing.
|
||||||
* @param maximumCapacity new max capacity
|
* @param maximumCapacity new max capacity
|
||||||
*/
|
*/
|
||||||
synchronized void setMaxCapacity(float maximumCapacity) {
|
void setMaxCapacity(float maximumCapacity) {
|
||||||
// Sanity check
|
try {
|
||||||
CSQueueUtils.checkMaxCapacity(getQueueName(),
|
writeLock.lock();
|
||||||
queueCapacities.getCapacity(), maximumCapacity);
|
// Sanity check
|
||||||
float absMaxCapacity =
|
CSQueueUtils.checkMaxCapacity(getQueueName(),
|
||||||
CSQueueUtils.computeAbsoluteMaximumCapacity(maximumCapacity, parent);
|
queueCapacities.getCapacity(), maximumCapacity);
|
||||||
CSQueueUtils.checkAbsoluteCapacity(getQueueName(),
|
float absMaxCapacity = CSQueueUtils.computeAbsoluteMaximumCapacity(
|
||||||
queueCapacities.getAbsoluteCapacity(),
|
maximumCapacity, parent);
|
||||||
absMaxCapacity);
|
CSQueueUtils.checkAbsoluteCapacity(getQueueName(),
|
||||||
|
queueCapacities.getAbsoluteCapacity(), absMaxCapacity);
|
||||||
|
|
||||||
queueCapacities.setMaximumCapacity(maximumCapacity);
|
queueCapacities.setMaximumCapacity(maximumCapacity);
|
||||||
queueCapacities.setAbsoluteMaximumCapacity(absMaxCapacity);
|
queueCapacities.setAbsoluteMaximumCapacity(absMaxCapacity);
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -240,70 +252,82 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
return defaultLabelExpression;
|
return defaultLabelExpression;
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized void setupQueueConfigs(Resource clusterResource)
|
void setupQueueConfigs(Resource clusterResource)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
// get labels
|
try {
|
||||||
this.accessibleLabels =
|
writeLock.lock();
|
||||||
csContext.getConfiguration().getAccessibleNodeLabels(getQueuePath());
|
// get labels
|
||||||
this.defaultLabelExpression = csContext.getConfiguration()
|
this.accessibleLabels =
|
||||||
.getDefaultNodeLabelExpression(getQueuePath());
|
csContext.getConfiguration().getAccessibleNodeLabels(getQueuePath());
|
||||||
|
this.defaultLabelExpression =
|
||||||
|
csContext.getConfiguration().getDefaultNodeLabelExpression(
|
||||||
|
getQueuePath());
|
||||||
|
|
||||||
// inherit from parent if labels not set
|
// inherit from parent if labels not set
|
||||||
if (this.accessibleLabels == null && parent != null) {
|
if (this.accessibleLabels == null && parent != null) {
|
||||||
this.accessibleLabels = parent.getAccessibleNodeLabels();
|
this.accessibleLabels = parent.getAccessibleNodeLabels();
|
||||||
}
|
}
|
||||||
|
|
||||||
// inherit from parent if labels not set
|
// inherit from parent if labels not set
|
||||||
if (this.defaultLabelExpression == null && parent != null
|
if (this.defaultLabelExpression == null && parent != null
|
||||||
&& this.accessibleLabels.containsAll(parent.getAccessibleNodeLabels())) {
|
&& this.accessibleLabels.containsAll(
|
||||||
this.defaultLabelExpression = parent.getDefaultNodeLabelExpression();
|
parent.getAccessibleNodeLabels())) {
|
||||||
}
|
this.defaultLabelExpression = parent.getDefaultNodeLabelExpression();
|
||||||
|
}
|
||||||
|
|
||||||
// After we setup labels, we can setup capacities
|
// After we setup labels, we can setup capacities
|
||||||
setupConfigurableCapacities();
|
setupConfigurableCapacities();
|
||||||
|
|
||||||
this.maximumAllocation =
|
this.maximumAllocation =
|
||||||
csContext.getConfiguration().getMaximumAllocationPerQueue(
|
csContext.getConfiguration().getMaximumAllocationPerQueue(
|
||||||
getQueuePath());
|
getQueuePath());
|
||||||
|
|
||||||
authorizer = YarnAuthorizationProvider.getInstance(csContext.getConf());
|
authorizer = YarnAuthorizationProvider.getInstance(csContext.getConf());
|
||||||
|
|
||||||
this.state = csContext.getConfiguration().getState(getQueuePath());
|
this.state = csContext.getConfiguration().getState(getQueuePath());
|
||||||
this.acls = csContext.getConfiguration().getAcls(getQueuePath());
|
this.acls = csContext.getConfiguration().getAcls(getQueuePath());
|
||||||
|
|
||||||
// Update metrics
|
// Update metrics
|
||||||
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
||||||
minimumAllocation, this, labelManager, null);
|
minimumAllocation, this, labelManager, null);
|
||||||
|
|
||||||
// Check if labels of this queue is a subset of parent queue, only do this
|
// Check if labels of this queue is a subset of parent queue, only do this
|
||||||
// when we not root
|
// when we not root
|
||||||
if (parent != null && parent.getParent() != null) {
|
if (parent != null && parent.getParent() != null) {
|
||||||
if (parent.getAccessibleNodeLabels() != null
|
if (parent.getAccessibleNodeLabels() != null && !parent
|
||||||
&& !parent.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
|
.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
|
||||||
// if parent isn't "*", child shouldn't be "*" too
|
// if parent isn't "*", child shouldn't be "*" too
|
||||||
if (this.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
|
if (this.getAccessibleNodeLabels().contains(
|
||||||
throw new IOException("Parent's accessible queue is not ANY(*), "
|
RMNodeLabelsManager.ANY)) {
|
||||||
+ "but child's accessible queue is *");
|
throw new IOException("Parent's accessible queue is not ANY(*), "
|
||||||
} else {
|
+ "but child's accessible queue is *");
|
||||||
Set<String> diff =
|
} else{
|
||||||
Sets.difference(this.getAccessibleNodeLabels(),
|
Set<String> diff = Sets.difference(this.getAccessibleNodeLabels(),
|
||||||
parent.getAccessibleNodeLabels());
|
parent.getAccessibleNodeLabels());
|
||||||
if (!diff.isEmpty()) {
|
if (!diff.isEmpty()) {
|
||||||
throw new IOException("Some labels of child queue is not a subset "
|
throw new IOException(
|
||||||
+ "of parent queue, these labels=["
|
"Some labels of child queue is not a subset "
|
||||||
+ StringUtils.join(diff, ",") + "]");
|
+ "of parent queue, these labels=[" + StringUtils
|
||||||
|
.join(diff, ",") + "]");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
this.reservationsContinueLooking =
|
||||||
|
csContext.getConfiguration().getReservationContinueLook();
|
||||||
|
|
||||||
|
this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(this);
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
this.reservationsContinueLooking = csContext.getConfiguration()
|
|
||||||
.getReservationContinueLook();
|
|
||||||
|
|
||||||
this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(this);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected QueueInfo getQueueInfo() {
|
protected QueueInfo getQueueInfo() {
|
||||||
|
// Deliberately doesn't use lock here, because this method will be invoked
|
||||||
|
// from schedulerApplicationAttempt, to avoid deadlock, sacrifice
|
||||||
|
// consistency here.
|
||||||
|
// TODO, improve this
|
||||||
QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
|
QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
|
||||||
queueInfo.setQueueName(queueName);
|
queueInfo.setQueueName(queueName);
|
||||||
queueInfo.setAccessibleNodeLabels(accessibleLabels);
|
queueInfo.setAccessibleNodeLabels(accessibleLabels);
|
||||||
|
@ -318,8 +342,12 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
public QueueStatistics getQueueStatistics() {
|
public QueueStatistics getQueueStatistics() {
|
||||||
QueueStatistics stats =
|
// Deliberately doesn't use lock here, because this method will be invoked
|
||||||
recordFactory.newRecordInstance(QueueStatistics.class);
|
// from schedulerApplicationAttempt, to avoid deadlock, sacrifice
|
||||||
|
// consistency here.
|
||||||
|
// TODO, improve this
|
||||||
|
QueueStatistics stats = recordFactory.newRecordInstance(
|
||||||
|
QueueStatistics.class);
|
||||||
stats.setNumAppsSubmitted(getMetrics().getAppsSubmitted());
|
stats.setNumAppsSubmitted(getMetrics().getAppsSubmitted());
|
||||||
stats.setNumAppsRunning(getMetrics().getAppsRunning());
|
stats.setNumAppsRunning(getMetrics().getAppsRunning());
|
||||||
stats.setNumAppsPending(getMetrics().getAppsPending());
|
stats.setNumAppsPending(getMetrics().getAppsPending());
|
||||||
|
@ -351,26 +379,36 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
return minimumAllocation;
|
return minimumAllocation;
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized void allocateResource(Resource clusterResource,
|
void allocateResource(Resource clusterResource,
|
||||||
Resource resource, String nodePartition, boolean changeContainerResource) {
|
Resource resource, String nodePartition, boolean changeContainerResource) {
|
||||||
queueUsage.incUsed(nodePartition, resource);
|
try {
|
||||||
|
writeLock.lock();
|
||||||
|
queueUsage.incUsed(nodePartition, resource);
|
||||||
|
|
||||||
if (!changeContainerResource) {
|
if (!changeContainerResource) {
|
||||||
++numContainers;
|
++numContainers;
|
||||||
|
}
|
||||||
|
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
||||||
|
minimumAllocation, this, labelManager, nodePartition);
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
|
||||||
minimumAllocation, this, labelManager, nodePartition);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected synchronized void releaseResource(Resource clusterResource,
|
protected void releaseResource(Resource clusterResource,
|
||||||
Resource resource, String nodePartition, boolean changeContainerResource) {
|
Resource resource, String nodePartition, boolean changeContainerResource) {
|
||||||
queueUsage.decUsed(nodePartition, resource);
|
try {
|
||||||
|
writeLock.lock();
|
||||||
|
queueUsage.decUsed(nodePartition, resource);
|
||||||
|
|
||||||
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
||||||
minimumAllocation, this, labelManager, nodePartition);
|
minimumAllocation, this, labelManager, nodePartition);
|
||||||
|
|
||||||
if (!changeContainerResource) {
|
if (!changeContainerResource) {
|
||||||
--numContainers;
|
--numContainers;
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -381,7 +419,13 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
public Map<AccessType, AccessControlList> getACLs() {
|
public Map<AccessType, AccessControlList> getACLs() {
|
||||||
return acls;
|
try {
|
||||||
|
readLock.lock();
|
||||||
|
return acls;
|
||||||
|
} finally {
|
||||||
|
readLock.unlock();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
|
@ -464,86 +508,88 @@ public abstract class AbstractCSQueue implements CSQueue {
|
||||||
minimumAllocation);
|
minimumAllocation);
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized boolean canAssignToThisQueue(Resource clusterResource,
|
boolean canAssignToThisQueue(Resource clusterResource,
|
||||||
String nodePartition, ResourceLimits currentResourceLimits,
|
String nodePartition, ResourceLimits currentResourceLimits,
|
||||||
Resource resourceCouldBeUnreserved, SchedulingMode schedulingMode) {
|
Resource resourceCouldBeUnreserved, SchedulingMode schedulingMode) {
|
||||||
// Get current limited resource:
|
try {
|
||||||
// - When doing RESPECT_PARTITION_EXCLUSIVITY allocation, we will respect
|
readLock.lock();
|
||||||
// queues' max capacity.
|
// Get current limited resource:
|
||||||
// - When doing IGNORE_PARTITION_EXCLUSIVITY allocation, we will not respect
|
// - When doing RESPECT_PARTITION_EXCLUSIVITY allocation, we will respect
|
||||||
// queue's max capacity, queue's max capacity on the partition will be
|
// queues' max capacity.
|
||||||
// considered to be 100%. Which is a queue can use all resource in the
|
// - When doing IGNORE_PARTITION_EXCLUSIVITY allocation, we will not respect
|
||||||
// partition.
|
// queue's max capacity, queue's max capacity on the partition will be
|
||||||
// Doing this because: for non-exclusive allocation, we make sure there's
|
// considered to be 100%. Which is a queue can use all resource in the
|
||||||
// idle resource on the partition, to avoid wastage, such resource will be
|
// partition.
|
||||||
// leveraged as much as we can, and preemption policy will reclaim it back
|
// Doing this because: for non-exclusive allocation, we make sure there's
|
||||||
// when partitoned-resource-request comes back.
|
// idle resource on the partition, to avoid wastage, such resource will be
|
||||||
Resource currentLimitResource =
|
// leveraged as much as we can, and preemption policy will reclaim it back
|
||||||
getCurrentLimitResource(nodePartition, clusterResource,
|
// when partitoned-resource-request comes back.
|
||||||
currentResourceLimits, schedulingMode);
|
Resource currentLimitResource = getCurrentLimitResource(nodePartition,
|
||||||
|
clusterResource, currentResourceLimits, schedulingMode);
|
||||||
|
|
||||||
Resource nowTotalUsed = queueUsage.getUsed(nodePartition);
|
Resource nowTotalUsed = queueUsage.getUsed(nodePartition);
|
||||||
|
|
||||||
// Set headroom for currentResourceLimits:
|
// Set headroom for currentResourceLimits:
|
||||||
// When queue is a parent queue: Headroom = limit - used + killable
|
// When queue is a parent queue: Headroom = limit - used + killable
|
||||||
// When queue is a leaf queue: Headroom = limit - used (leaf queue cannot preempt itself)
|
// When queue is a leaf queue: Headroom = limit - used (leaf queue cannot preempt itself)
|
||||||
Resource usedExceptKillable = nowTotalUsed;
|
Resource usedExceptKillable = nowTotalUsed;
|
||||||
if (null != getChildQueues() && !getChildQueues().isEmpty()) {
|
if (null != getChildQueues() && !getChildQueues().isEmpty()) {
|
||||||
usedExceptKillable = Resources.subtract(nowTotalUsed,
|
usedExceptKillable = Resources.subtract(nowTotalUsed,
|
||||||
getTotalKillableResource(nodePartition));
|
getTotalKillableResource(nodePartition));
|
||||||
}
|
}
|
||||||
currentResourceLimits.setHeadroom(
|
currentResourceLimits.setHeadroom(
|
||||||
Resources.subtract(currentLimitResource, usedExceptKillable));
|
Resources.subtract(currentLimitResource, usedExceptKillable));
|
||||||
|
|
||||||
if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
|
if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
|
||||||
usedExceptKillable, currentLimitResource)) {
|
usedExceptKillable, currentLimitResource)) {
|
||||||
|
|
||||||
// if reservation continous looking enabled, check to see if could we
|
// if reservation continous looking enabled, check to see if could we
|
||||||
// potentially use this node instead of a reserved node if the application
|
// potentially use this node instead of a reserved node if the application
|
||||||
// has reserved containers.
|
// has reserved containers.
|
||||||
// TODO, now only consider reservation cases when the node has no label
|
// TODO, now only consider reservation cases when the node has no label
|
||||||
if (this.reservationsContinueLooking
|
if (this.reservationsContinueLooking && nodePartition.equals(
|
||||||
&& nodePartition.equals(RMNodeLabelsManager.NO_LABEL)
|
RMNodeLabelsManager.NO_LABEL) && Resources.greaterThan(
|
||||||
&& Resources.greaterThan(resourceCalculator, clusterResource,
|
resourceCalculator, clusterResource, resourceCouldBeUnreserved,
|
||||||
resourceCouldBeUnreserved, Resources.none())) {
|
Resources.none())) {
|
||||||
// resource-without-reserved = used - reserved
|
// resource-without-reserved = used - reserved
|
||||||
Resource newTotalWithoutReservedResource =
|
Resource newTotalWithoutReservedResource = Resources.subtract(
|
||||||
Resources.subtract(usedExceptKillable, resourceCouldBeUnreserved);
|
usedExceptKillable, resourceCouldBeUnreserved);
|
||||||
|
|
||||||
// when total-used-without-reserved-resource < currentLimit, we still
|
// when total-used-without-reserved-resource < currentLimit, we still
|
||||||
// have chance to allocate on this node by unreserving some containers
|
// have chance to allocate on this node by unreserving some containers
|
||||||
if (Resources.lessThan(resourceCalculator, clusterResource,
|
if (Resources.lessThan(resourceCalculator, clusterResource,
|
||||||
newTotalWithoutReservedResource, currentLimitResource)) {
|
newTotalWithoutReservedResource, currentLimitResource)) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("try to use reserved: " + getQueueName()
|
LOG.debug(
|
||||||
+ " usedResources: " + queueUsage.getUsed()
|
"try to use reserved: " + getQueueName() + " usedResources: "
|
||||||
+ ", clusterResources: " + clusterResource
|
+ queueUsage.getUsed() + ", clusterResources: "
|
||||||
+ ", reservedResources: " + resourceCouldBeUnreserved
|
+ clusterResource + ", reservedResources: "
|
||||||
+ ", capacity-without-reserved: "
|
+ resourceCouldBeUnreserved
|
||||||
+ newTotalWithoutReservedResource + ", maxLimitCapacity: "
|
+ ", capacity-without-reserved: "
|
||||||
+ currentLimitResource);
|
+ newTotalWithoutReservedResource + ", maxLimitCapacity: "
|
||||||
|
+ currentLimitResource);
|
||||||
|
}
|
||||||
|
return true;
|
||||||
}
|
}
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug(getQueueName() + "Check assign to queue, nodePartition="
|
||||||
|
+ nodePartition + " usedResources: " + queueUsage
|
||||||
|
.getUsed(nodePartition) + " clusterResources: " + clusterResource
|
||||||
|
+ " currentUsedCapacity " + Resources
|
||||||
|
.divide(resourceCalculator, clusterResource,
|
||||||
|
queueUsage.getUsed(nodePartition), labelManager
|
||||||
|
.getResourceByLabel(nodePartition, clusterResource))
|
||||||
|
+ " max-capacity: " + queueCapacities
|
||||||
|
.getAbsoluteMaximumCapacity(nodePartition) + ")");
|
||||||
|
}
|
||||||
|
return false;
|
||||||
}
|
}
|
||||||
if (LOG.isDebugEnabled()) {
|
return true;
|
||||||
LOG.debug(getQueueName()
|
} finally {
|
||||||
+ "Check assign to queue, nodePartition="
|
readLock.unlock();
|
||||||
+ nodePartition
|
|
||||||
+ " usedResources: "
|
|
||||||
+ queueUsage.getUsed(nodePartition)
|
|
||||||
+ " clusterResources: "
|
|
||||||
+ clusterResource
|
|
||||||
+ " currentUsedCapacity "
|
|
||||||
+ Resources.divide(resourceCalculator, clusterResource,
|
|
||||||
queueUsage.getUsed(nodePartition),
|
|
||||||
labelManager.getResourceByLabel(nodePartition, clusterResource))
|
|
||||||
+ " max-capacity: "
|
|
||||||
+ queueCapacities.getAbsoluteMaximumCapacity(nodePartition) + ")");
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -79,76 +79,98 @@ public class PlanQueue extends ParentQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void reinitialize(CSQueue newlyParsedQueue,
|
public void reinitialize(CSQueue newlyParsedQueue,
|
||||||
Resource clusterResource) throws IOException {
|
Resource clusterResource) throws IOException {
|
||||||
// Sanity check
|
try {
|
||||||
if (!(newlyParsedQueue instanceof PlanQueue)
|
writeLock.lock();
|
||||||
|| !newlyParsedQueue.getQueuePath().equals(getQueuePath())) {
|
// Sanity check
|
||||||
throw new IOException("Trying to reinitialize " + getQueuePath()
|
if (!(newlyParsedQueue instanceof PlanQueue) || !newlyParsedQueue
|
||||||
+ " from " + newlyParsedQueue.getQueuePath());
|
.getQueuePath().equals(getQueuePath())) {
|
||||||
}
|
throw new IOException(
|
||||||
|
"Trying to reinitialize " + getQueuePath() + " from "
|
||||||
|
+ newlyParsedQueue.getQueuePath());
|
||||||
|
}
|
||||||
|
|
||||||
PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
|
PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
|
||||||
|
|
||||||
if (newlyParsedParentQueue.getChildQueues().size() > 0) {
|
if (newlyParsedParentQueue.getChildQueues().size() > 0) {
|
||||||
throw new IOException(
|
throw new IOException(
|
||||||
"Reservable Queue should not have sub-queues in the"
|
"Reservable Queue should not have sub-queues in the"
|
||||||
+ "configuration");
|
+ "configuration");
|
||||||
}
|
}
|
||||||
|
|
||||||
// Set new configs
|
// Set new configs
|
||||||
setupQueueConfigs(clusterResource);
|
setupQueueConfigs(clusterResource);
|
||||||
|
|
||||||
updateQuotas(newlyParsedParentQueue.userLimit,
|
updateQuotas(newlyParsedParentQueue.userLimit,
|
||||||
newlyParsedParentQueue.userLimitFactor,
|
newlyParsedParentQueue.userLimitFactor,
|
||||||
newlyParsedParentQueue.maxAppsForReservation,
|
newlyParsedParentQueue.maxAppsForReservation,
|
||||||
newlyParsedParentQueue.maxAppsPerUserForReservation);
|
newlyParsedParentQueue.maxAppsPerUserForReservation);
|
||||||
|
|
||||||
// run reinitialize on each existing queue, to trigger absolute cap
|
// run reinitialize on each existing queue, to trigger absolute cap
|
||||||
// recomputations
|
// recomputations
|
||||||
for (CSQueue res : this.getChildQueues()) {
|
for (CSQueue res : this.getChildQueues()) {
|
||||||
res.reinitialize(res, clusterResource);
|
res.reinitialize(res, clusterResource);
|
||||||
}
|
}
|
||||||
showReservationsAsQueues = newlyParsedParentQueue.showReservationsAsQueues;
|
showReservationsAsQueues =
|
||||||
}
|
newlyParsedParentQueue.showReservationsAsQueues;
|
||||||
|
} finally {
|
||||||
synchronized void addChildQueue(CSQueue newQueue)
|
writeLock.unlock();
|
||||||
throws SchedulerDynamicEditException {
|
|
||||||
if (newQueue.getCapacity() > 0) {
|
|
||||||
throw new SchedulerDynamicEditException("Queue " + newQueue
|
|
||||||
+ " being added has non zero capacity.");
|
|
||||||
}
|
|
||||||
boolean added = this.childQueues.add(newQueue);
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("updateChildQueues (action: add queue): " + added + " "
|
|
||||||
+ getChildQueuesToPrint());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized void removeChildQueue(CSQueue remQueue)
|
void addChildQueue(CSQueue newQueue)
|
||||||
throws SchedulerDynamicEditException {
|
throws SchedulerDynamicEditException {
|
||||||
if (remQueue.getCapacity() > 0) {
|
try {
|
||||||
throw new SchedulerDynamicEditException("Queue " + remQueue
|
writeLock.lock();
|
||||||
+ " being removed has non zero capacity.");
|
if (newQueue.getCapacity() > 0) {
|
||||||
|
throw new SchedulerDynamicEditException(
|
||||||
|
"Queue " + newQueue + " being added has non zero capacity.");
|
||||||
|
}
|
||||||
|
boolean added = this.childQueues.add(newQueue);
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("updateChildQueues (action: add queue): " + added + " "
|
||||||
|
+ getChildQueuesToPrint());
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
Iterator<CSQueue> qiter = childQueues.iterator();
|
}
|
||||||
while (qiter.hasNext()) {
|
|
||||||
CSQueue cs = qiter.next();
|
void removeChildQueue(CSQueue remQueue)
|
||||||
if (cs.equals(remQueue)) {
|
throws SchedulerDynamicEditException {
|
||||||
qiter.remove();
|
try {
|
||||||
if (LOG.isDebugEnabled()) {
|
writeLock.lock();
|
||||||
LOG.debug("Removed child queue: {}", cs.getQueueName());
|
if (remQueue.getCapacity() > 0) {
|
||||||
|
throw new SchedulerDynamicEditException(
|
||||||
|
"Queue " + remQueue + " being removed has non zero capacity.");
|
||||||
|
}
|
||||||
|
Iterator<CSQueue> qiter = childQueues.iterator();
|
||||||
|
while (qiter.hasNext()) {
|
||||||
|
CSQueue cs = qiter.next();
|
||||||
|
if (cs.equals(remQueue)) {
|
||||||
|
qiter.remove();
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Removed child queue: {}", cs.getQueueName());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected synchronized float sumOfChildCapacities() {
|
protected float sumOfChildCapacities() {
|
||||||
float ret = 0;
|
try {
|
||||||
for (CSQueue l : childQueues) {
|
writeLock.lock();
|
||||||
ret += l.getCapacity();
|
float ret = 0;
|
||||||
|
for (CSQueue l : childQueues) {
|
||||||
|
ret += l.getCapacity();
|
||||||
|
}
|
||||||
|
return ret;
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
return ret;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void updateQuotas(int userLimit, float userLimitFactor,
|
private void updateQuotas(int userLimit, float userLimitFactor,
|
||||||
|
|
|
@ -51,22 +51,28 @@ public class ReservationQueue extends LeafQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void reinitialize(CSQueue newlyParsedQueue,
|
public void reinitialize(CSQueue newlyParsedQueue,
|
||||||
Resource clusterResource) throws IOException {
|
Resource clusterResource) throws IOException {
|
||||||
// Sanity check
|
try {
|
||||||
if (!(newlyParsedQueue instanceof ReservationQueue)
|
writeLock.lock();
|
||||||
|| !newlyParsedQueue.getQueuePath().equals(getQueuePath())) {
|
// Sanity check
|
||||||
throw new IOException("Trying to reinitialize " + getQueuePath()
|
if (!(newlyParsedQueue instanceof ReservationQueue) || !newlyParsedQueue
|
||||||
+ " from " + newlyParsedQueue.getQueuePath());
|
.getQueuePath().equals(getQueuePath())) {
|
||||||
}
|
throw new IOException(
|
||||||
super.reinitialize(newlyParsedQueue, clusterResource);
|
"Trying to reinitialize " + getQueuePath() + " from "
|
||||||
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
+ newlyParsedQueue.getQueuePath());
|
||||||
minimumAllocation, this, labelManager, null);
|
}
|
||||||
|
super.reinitialize(newlyParsedQueue, clusterResource);
|
||||||
|
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
||||||
|
minimumAllocation, this, labelManager, null);
|
||||||
|
|
||||||
updateQuotas(parent.getUserLimitForReservation(),
|
updateQuotas(parent.getUserLimitForReservation(),
|
||||||
parent.getUserLimitFactor(),
|
parent.getUserLimitFactor(),
|
||||||
parent.getMaxApplicationsForReservations(),
|
parent.getMaxApplicationsForReservations(),
|
||||||
parent.getMaxApplicationsPerUserForReservation());
|
parent.getMaxApplicationsPerUserForReservation());
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -77,21 +83,26 @@ public class ReservationQueue extends LeafQueue {
|
||||||
* maxCapacity, etc..)
|
* maxCapacity, etc..)
|
||||||
* @throws SchedulerDynamicEditException
|
* @throws SchedulerDynamicEditException
|
||||||
*/
|
*/
|
||||||
public synchronized void setEntitlement(QueueEntitlement entitlement)
|
public void setEntitlement(QueueEntitlement entitlement)
|
||||||
throws SchedulerDynamicEditException {
|
throws SchedulerDynamicEditException {
|
||||||
float capacity = entitlement.getCapacity();
|
try {
|
||||||
if (capacity < 0 || capacity > 1.0f) {
|
writeLock.lock();
|
||||||
throw new SchedulerDynamicEditException(
|
float capacity = entitlement.getCapacity();
|
||||||
"Capacity demand is not in the [0,1] range: " + capacity);
|
if (capacity < 0 || capacity > 1.0f) {
|
||||||
}
|
throw new SchedulerDynamicEditException(
|
||||||
setCapacity(capacity);
|
"Capacity demand is not in the [0,1] range: " + capacity);
|
||||||
setAbsoluteCapacity(getParent().getAbsoluteCapacity() * getCapacity());
|
}
|
||||||
// note: we currently set maxCapacity to capacity
|
setCapacity(capacity);
|
||||||
// this might be revised later
|
setAbsoluteCapacity(getParent().getAbsoluteCapacity() * getCapacity());
|
||||||
setMaxCapacity(entitlement.getMaxCapacity());
|
// note: we currently set maxCapacity to capacity
|
||||||
if (LOG.isDebugEnabled()) {
|
// this might be revised later
|
||||||
LOG.debug("successfully changed to " + capacity + " for queue "
|
setMaxCapacity(entitlement.getMaxCapacity());
|
||||||
+ this.getQueueName());
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("successfully changed to " + capacity + " for queue " + this
|
||||||
|
.getQueueName());
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -828,8 +828,8 @@ public class TestContainerResizing {
|
||||||
app.getAppAttemptResourceUsage().getPending().getMemorySize());
|
app.getAppAttemptResourceUsage().getPending().getMemorySize());
|
||||||
// Queue/user/application's usage will be updated
|
// Queue/user/application's usage will be updated
|
||||||
checkUsedResource(rm1, "default", 0 * GB, null);
|
checkUsedResource(rm1, "default", 0 * GB, null);
|
||||||
Assert.assertEquals(0 * GB, ((LeafQueue) cs.getQueue("default"))
|
// User will be removed
|
||||||
.getUser("user").getUsed().getMemorySize());
|
Assert.assertNull(((LeafQueue) cs.getQueue("default")).getUser("user"));
|
||||||
Assert.assertEquals(0 * GB,
|
Assert.assertEquals(0 * GB,
|
||||||
app.getAppAttemptResourceUsage().getReserved().getMemorySize());
|
app.getAppAttemptResourceUsage().getReserved().getMemorySize());
|
||||||
Assert.assertEquals(0 * GB,
|
Assert.assertEquals(0 * GB,
|
||||||
|
|
Loading…
Reference in New Issue