YARN-3140. Improve locks in AbstractCSQueue/LeafQueue/ParentQueue. Contributed by Wangda Tan

(cherry picked from commit 2b66d9ec5b)
This commit is contained in:
Jian He 2016-09-20 15:03:07 +08:00
parent 4e376f162f
commit 3acd30df71
7 changed files with 1817 additions and 1468 deletions

View File

@ -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>

View File

@ -24,6 +24,7 @@
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 @@
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 AbstractCSQueue(CapacitySchedulerContext cs,
// 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 @@ protected void setupConfigurableCapacities() {
} }
@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 int getNumContainers() {
} }
@Override @Override
public synchronized QueueState getState() { public QueueState getState() {
return state; return state;
} }
@ -187,13 +195,13 @@ public PrivilegedEntity getPrivilegedEntity() {
} }
@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 void setAbsoluteUsedCapacity(float absUsedCapacity) {
* 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) {
try {
writeLock.lock();
// Sanity check // Sanity check
CSQueueUtils.checkMaxCapacity(getQueueName(), CSQueueUtils.checkMaxCapacity(getQueueName(),
queueCapacities.getCapacity(), maximumCapacity); queueCapacities.getCapacity(), maximumCapacity);
float absMaxCapacity = float absMaxCapacity = CSQueueUtils.computeAbsoluteMaximumCapacity(
CSQueueUtils.computeAbsoluteMaximumCapacity(maximumCapacity, parent); maximumCapacity, parent);
CSQueueUtils.checkAbsoluteCapacity(getQueueName(), CSQueueUtils.checkAbsoluteCapacity(getQueueName(),
queueCapacities.getAbsoluteCapacity(), queueCapacities.getAbsoluteCapacity(), absMaxCapacity);
absMaxCapacity);
queueCapacities.setMaximumCapacity(maximumCapacity); queueCapacities.setMaximumCapacity(maximumCapacity);
queueCapacities.setAbsoluteMaximumCapacity(absMaxCapacity); queueCapacities.setAbsoluteMaximumCapacity(absMaxCapacity);
} finally {
writeLock.unlock();
}
} }
@Override @Override
@ -240,13 +252,16 @@ public String getDefaultNodeLabelExpression() {
return defaultLabelExpression; return defaultLabelExpression;
} }
synchronized void setupQueueConfigs(Resource clusterResource) void setupQueueConfigs(Resource clusterResource)
throws IOException { throws IOException {
try {
writeLock.lock();
// get labels // get labels
this.accessibleLabels = this.accessibleLabels =
csContext.getConfiguration().getAccessibleNodeLabels(getQueuePath()); csContext.getConfiguration().getAccessibleNodeLabels(getQueuePath());
this.defaultLabelExpression = csContext.getConfiguration() this.defaultLabelExpression =
.getDefaultNodeLabelExpression(getQueuePath()); 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) {
@ -255,7 +270,8 @@ synchronized void setupQueueConfigs(Resource clusterResource)
// 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(
parent.getAccessibleNodeLabels())) {
this.defaultLabelExpression = parent.getDefaultNodeLabelExpression(); this.defaultLabelExpression = parent.getDefaultNodeLabelExpression();
} }
@ -278,32 +294,40 @@ synchronized void setupQueueConfigs(Resource clusterResource)
// 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(
RMNodeLabelsManager.ANY)) {
throw new IOException("Parent's accessible queue is not ANY(*), " throw new IOException("Parent's accessible queue is not ANY(*), "
+ "but child's accessible queue is *"); + "but child's accessible queue is *");
} else { } else{
Set<String> diff = Set<String> diff = Sets.difference(this.getAccessibleNodeLabels(),
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() this.reservationsContinueLooking =
.getReservationContinueLook(); csContext.getConfiguration().getReservationContinueLook();
this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(this); this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(this);
} finally {
writeLock.unlock();
}
} }
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 @@ protected QueueInfo getQueueInfo() {
} }
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,8 +379,10 @@ public Resource getMinimumAllocation() {
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) {
try {
writeLock.lock();
queueUsage.incUsed(nodePartition, resource); queueUsage.incUsed(nodePartition, resource);
if (!changeContainerResource) { if (!changeContainerResource) {
@ -360,10 +390,15 @@ synchronized void allocateResource(Resource clusterResource,
} }
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
minimumAllocation, this, labelManager, nodePartition); minimumAllocation, this, labelManager, nodePartition);
} finally {
writeLock.unlock();
}
} }
protected synchronized void releaseResource(Resource clusterResource, protected void releaseResource(Resource clusterResource,
Resource resource, String nodePartition, boolean changeContainerResource) { Resource resource, String nodePartition, boolean changeContainerResource) {
try {
writeLock.lock();
queueUsage.decUsed(nodePartition, resource); queueUsage.decUsed(nodePartition, resource);
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
@ -372,6 +407,9 @@ protected synchronized void releaseResource(Resource clusterResource,
if (!changeContainerResource) { if (!changeContainerResource) {
--numContainers; --numContainers;
} }
} finally {
writeLock.unlock();
}
} }
@Private @Private
@ -381,7 +419,13 @@ public boolean getReservationContinueLooking() {
@Private @Private
public Map<AccessType, AccessControlList> getACLs() { public Map<AccessType, AccessControlList> getACLs() {
try {
readLock.lock();
return acls; return acls;
} finally {
readLock.unlock();
}
} }
@Private @Private
@ -464,9 +508,11 @@ Resource getQueueMaxResource(String nodePartition, Resource clusterResource) {
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) {
try {
readLock.lock();
// Get current limited resource: // Get current limited resource:
// - When doing RESPECT_PARTITION_EXCLUSIVITY allocation, we will respect // - When doing RESPECT_PARTITION_EXCLUSIVITY allocation, we will respect
// queues' max capacity. // queues' max capacity.
@ -478,9 +524,8 @@ synchronized boolean canAssignToThisQueue(Resource clusterResource,
// idle resource on the partition, to avoid wastage, such resource will be // idle resource on the partition, to avoid wastage, such resource will be
// leveraged as much as we can, and preemption policy will reclaim it back // leveraged as much as we can, and preemption policy will reclaim it back
// when partitoned-resource-request comes back. // when partitoned-resource-request comes back.
Resource currentLimitResource = Resource currentLimitResource = getCurrentLimitResource(nodePartition,
getCurrentLimitResource(nodePartition, clusterResource, clusterResource, currentResourceLimits, schedulingMode);
currentResourceLimits, schedulingMode);
Resource nowTotalUsed = queueUsage.getUsed(nodePartition); Resource nowTotalUsed = queueUsage.getUsed(nodePartition);
@ -502,23 +547,24 @@ synchronized boolean canAssignToThisQueue(Resource clusterResource,
// 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: "
+ resourceCouldBeUnreserved
+ ", capacity-without-reserved: " + ", capacity-without-reserved: "
+ newTotalWithoutReservedResource + ", maxLimitCapacity: " + newTotalWithoutReservedResource + ", maxLimitCapacity: "
+ currentLimitResource); + currentLimitResource);
@ -527,23 +573,23 @@ synchronized boolean canAssignToThisQueue(Resource clusterResource,
} }
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(getQueueName() LOG.debug(getQueueName() + "Check assign to queue, nodePartition="
+ "Check assign to queue, nodePartition=" + nodePartition + " usedResources: " + queueUsage
+ nodePartition .getUsed(nodePartition) + " clusterResources: " + clusterResource
+ " usedResources: " + " currentUsedCapacity " + Resources
+ queueUsage.getUsed(nodePartition) .divide(resourceCalculator, clusterResource,
+ " clusterResources: " queueUsage.getUsed(nodePartition), labelManager
+ clusterResource .getResourceByLabel(nodePartition, clusterResource))
+ " currentUsedCapacity " + " max-capacity: " + queueCapacities
+ Resources.divide(resourceCalculator, clusterResource, .getAbsoluteMaximumCapacity(nodePartition) + ")");
queueUsage.getUsed(nodePartition),
labelManager.getResourceByLabel(nodePartition, clusterResource))
+ " max-capacity: "
+ queueCapacities.getAbsoluteMaximumCapacity(nodePartition) + ")");
} }
return false; return false;
} }
return true; return true;
} finally {
readLock.unlock();
}
} }
@Override @Override

View File

@ -107,8 +107,10 @@ public ParentQueue(CapacitySchedulerContext cs,
", fullname=" + getQueuePath()); ", fullname=" + getQueuePath());
} }
synchronized void setupQueueConfigs(Resource clusterResource) void setupQueueConfigs(Resource clusterResource)
throws IOException { throws IOException {
try {
writeLock.lock();
super.setupQueueConfigs(clusterResource); super.setupQueueConfigs(clusterResource);
StringBuilder aclsString = new StringBuilder(); StringBuilder aclsString = new StringBuilder();
for (Map.Entry<AccessType, AccessControlList> e : acls.entrySet()) { for (Map.Entry<AccessType, AccessControlList> e : acls.entrySet()) {
@ -123,19 +125,23 @@ synchronized void setupQueueConfigs(Resource clusterResource)
} }
} }
LOG.info(queueName + LOG.info(queueName + ", capacity=" + this.queueCapacities.getCapacity()
", capacity=" + this.queueCapacities.getCapacity() + + ", absoluteCapacity=" + this.queueCapacities.getAbsoluteCapacity()
", absoluteCapacity=" + this.queueCapacities.getAbsoluteCapacity() + + ", maxCapacity=" + this.queueCapacities.getMaximumCapacity()
", maxCapacity=" + this.queueCapacities.getMaximumCapacity() + + ", absoluteMaxCapacity=" + this.queueCapacities
", absoluteMaxCapacity=" + this.queueCapacities.getAbsoluteMaximumCapacity() + .getAbsoluteMaximumCapacity() + ", state=" + state + ", acls="
", state=" + state + + aclsString + ", labels=" + labelStrBuilder.toString() + "\n"
", acls=" + aclsString + + ", reservationsContinueLooking=" + reservationsContinueLooking);
", labels=" + labelStrBuilder.toString() + "\n" + } finally {
", reservationsContinueLooking=" + reservationsContinueLooking); writeLock.unlock();
}
} }
private static float PRECISION = 0.0005f; // 0.05% precision private static float PRECISION = 0.0005f; // 0.05% precision
synchronized void setChildQueues(Collection<CSQueue> childQueues) {
void setChildQueues(Collection<CSQueue> childQueues) {
try {
writeLock.lock();
// Validate // Validate
float childCapacities = 0; float childCapacities = 0;
for (CSQueue queue : childQueues) { for (CSQueue queue : childQueues) {
@ -143,11 +149,11 @@ synchronized void setChildQueues(Collection<CSQueue> childQueues) {
} }
float delta = Math.abs(1.0f - childCapacities); // crude way to check float delta = Math.abs(1.0f - childCapacities); // crude way to check
// allow capacities being set to 0, and enforce child 0 if parent is 0 // allow capacities being set to 0, and enforce child 0 if parent is 0
if (((queueCapacities.getCapacity() > 0) && (delta > PRECISION)) || if (((queueCapacities.getCapacity() > 0) && (delta > PRECISION)) || (
((queueCapacities.getCapacity() == 0) && (childCapacities > 0))) { (queueCapacities.getCapacity() == 0) && (childCapacities > 0))) {
throw new IllegalArgumentException("Illegal" + throw new IllegalArgumentException(
" capacity of " + childCapacities + "Illegal" + " capacity of " + childCapacities
" for children of queue " + queueName); + " for children of queue " + queueName);
} }
// check label capacities // check label capacities
for (String nodeLabel : queueCapacities.getExistingNodeLabels()) { for (String nodeLabel : queueCapacities.getExistingNodeLabels()) {
@ -159,9 +165,9 @@ synchronized void setChildQueues(Collection<CSQueue> childQueues) {
} }
if ((capacityByLabel > 0 && Math.abs(1.0f - sum) > PRECISION) if ((capacityByLabel > 0 && Math.abs(1.0f - sum) > PRECISION)
|| (capacityByLabel == 0) && (sum > 0)) { || (capacityByLabel == 0) && (sum > 0)) {
throw new IllegalArgumentException("Illegal" + " capacity of " throw new IllegalArgumentException(
+ sum + " for children of queue " + queueName "Illegal" + " capacity of " + sum + " for children of queue "
+ " for label=" + nodeLabel); + queueName + " for label=" + nodeLabel);
} }
} }
@ -170,6 +176,9 @@ synchronized void setChildQueues(Collection<CSQueue> childQueues) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("setChildQueues: " + getChildQueuesToPrint()); LOG.debug("setChildQueues: " + getChildQueuesToPrint());
} }
} finally {
writeLock.unlock();
}
} }
@Override @Override
@ -179,27 +188,34 @@ public String getQueuePath() {
} }
@Override @Override
public synchronized QueueInfo getQueueInfo( public QueueInfo getQueueInfo(
boolean includeChildQueues, boolean recursive) { boolean includeChildQueues, boolean recursive) {
try {
readLock.lock();
QueueInfo queueInfo = getQueueInfo(); QueueInfo queueInfo = getQueueInfo();
List<QueueInfo> childQueuesInfo = new ArrayList<QueueInfo>(); List<QueueInfo> childQueuesInfo = new ArrayList<>();
if (includeChildQueues) { if (includeChildQueues) {
for (CSQueue child : childQueues) { for (CSQueue child : childQueues) {
// Get queue information recursively? // Get queue information recursively?
childQueuesInfo.add( childQueuesInfo.add(child.getQueueInfo(recursive, recursive));
child.getQueueInfo(recursive, recursive));
} }
} }
queueInfo.setChildQueues(childQueuesInfo); queueInfo.setChildQueues(childQueuesInfo);
return queueInfo; return queueInfo;
} finally {
readLock.unlock();
} }
private synchronized QueueUserACLInfo getUserAclInfo( }
private QueueUserACLInfo getUserAclInfo(
UserGroupInformation user) { UserGroupInformation user) {
QueueUserACLInfo userAclInfo = try {
recordFactory.newRecordInstance(QueueUserACLInfo.class); readLock.lock();
QueueUserACLInfo userAclInfo = recordFactory.newRecordInstance(
QueueUserACLInfo.class);
List<QueueACL> operations = new ArrayList<QueueACL>(); List<QueueACL> operations = new ArrayList<QueueACL>();
for (QueueACL operation : QueueACL.values()) { for (QueueACL operation : QueueACL.values()) {
if (hasAccess(operation, user)) { if (hasAccess(operation, user)) {
@ -210,12 +226,18 @@ private synchronized QueueUserACLInfo getUserAclInfo(
userAclInfo.setQueueName(getQueueName()); userAclInfo.setQueueName(getQueueName());
userAclInfo.setUserAcls(operations); userAclInfo.setUserAcls(operations);
return userAclInfo; return userAclInfo;
} finally {
readLock.unlock();
}
} }
@Override @Override
public synchronized List<QueueUserACLInfo> getQueueUserAclInfo( public List<QueueUserACLInfo> getQueueUserAclInfo(
UserGroupInformation user) { UserGroupInformation user) {
List<QueueUserACLInfo> userAcls = new ArrayList<QueueUserACLInfo>(); try {
readLock.lock();
List<QueueUserACLInfo> userAcls = new ArrayList<>();
// Add parent queue acls // Add parent queue acls
userAcls.add(getUserAclInfo(user)); userAcls.add(getUserAclInfo(user));
@ -226,6 +248,10 @@ public synchronized List<QueueUserACLInfo> getQueueUserAclInfo(
} }
return userAcls; return userAcls;
} finally {
readLock.unlock();
}
} }
public String toString() { public String toString() {
@ -240,16 +266,19 @@ public String toString() {
} }
@Override @Override
public synchronized void reinitialize(CSQueue newlyParsedQueue, public void reinitialize(CSQueue newlyParsedQueue,
Resource clusterResource) throws IOException { Resource clusterResource) throws IOException {
try {
writeLock.lock();
// Sanity check // Sanity check
if (!(newlyParsedQueue instanceof ParentQueue) || if (!(newlyParsedQueue instanceof ParentQueue) || !newlyParsedQueue
!newlyParsedQueue.getQueuePath().equals(getQueuePath())) { .getQueuePath().equals(getQueuePath())) {
throw new IOException("Trying to reinitialize " + getQueuePath() + throw new IOException(
" from " + newlyParsedQueue.getQueuePath()); "Trying to reinitialize " + getQueuePath() + " from "
+ newlyParsedQueue.getQueuePath());
} }
ParentQueue newlyParsedParentQueue = (ParentQueue)newlyParsedQueue; ParentQueue newlyParsedParentQueue = (ParentQueue) newlyParsedQueue;
// Set new configs // Set new configs
setupQueueConfigs(clusterResource); setupQueueConfigs(clusterResource);
@ -257,8 +286,8 @@ public synchronized void reinitialize(CSQueue newlyParsedQueue,
// Re-configure existing child queues and add new ones // Re-configure existing child queues and add new ones
// The CS has already checked to ensure all existing child queues are present! // The CS has already checked to ensure all existing child queues are present!
Map<String, CSQueue> currentChildQueues = getQueues(childQueues); Map<String, CSQueue> currentChildQueues = getQueues(childQueues);
Map<String, CSQueue> newChildQueues = Map<String, CSQueue> newChildQueues = getQueues(
getQueues(newlyParsedParentQueue.childQueues); newlyParsedParentQueue.childQueues);
for (Map.Entry<String, CSQueue> e : newChildQueues.entrySet()) { for (Map.Entry<String, CSQueue> e : newChildQueues.entrySet()) {
String newChildQueueName = e.getKey(); String newChildQueueName = e.getKey();
CSQueue newChildQueue = e.getValue(); CSQueue newChildQueue = e.getValue();
@ -270,7 +299,7 @@ public synchronized void reinitialize(CSQueue newlyParsedQueue,
// Re-init existing child queues // Re-init existing child queues
childQueue.reinitialize(newChildQueue, clusterResource); childQueue.reinitialize(newChildQueue, clusterResource);
LOG.info(getQueueName() + ": re-configured queue: " + childQueue); LOG.info(getQueueName() + ": re-configured queue: " + childQueue);
} else { } else{
// New child queue, do not re-init // New child queue, do not re-init
// Set parent to 'this' // Set parent to 'this'
@ -279,13 +308,17 @@ public synchronized void reinitialize(CSQueue newlyParsedQueue,
// Save in list of current child queues // Save in list of current child queues
currentChildQueues.put(newChildQueueName, newChildQueue); currentChildQueues.put(newChildQueueName, newChildQueue);
LOG.info(getQueueName() + ": added new child queue: " + newChildQueue); LOG.info(
getQueueName() + ": added new child queue: " + newChildQueue);
} }
} }
// Re-sort all queues // Re-sort all queues
childQueues.clear(); childQueues.clear();
childQueues.addAll(currentChildQueues.values()); childQueues.addAll(currentChildQueues.values());
} finally {
writeLock.unlock();
}
} }
Map<String, CSQueue> getQueues(Set<CSQueue> queues) { Map<String, CSQueue> getQueues(Set<CSQueue> queues) {
@ -300,20 +333,23 @@ Map<String, CSQueue> getQueues(Set<CSQueue> queues) {
public void submitApplication(ApplicationId applicationId, String user, public void submitApplication(ApplicationId applicationId, String user,
String queue) throws AccessControlException { String queue) throws AccessControlException {
synchronized (this) { try {
writeLock.lock();
// Sanity check // Sanity check
if (queue.equals(queueName)) { if (queue.equals(queueName)) {
throw new AccessControlException("Cannot submit application " + throw new AccessControlException(
"to non-leaf queue: " + queueName); "Cannot submit application " + "to non-leaf queue: " + queueName);
} }
if (state != QueueState.RUNNING) { if (state != QueueState.RUNNING) {
throw new AccessControlException("Queue " + getQueuePath() + throw new AccessControlException("Queue " + getQueuePath()
" is STOPPED. Cannot accept submission of application: " + + " is STOPPED. Cannot accept submission of application: "
applicationId); + applicationId);
} }
addApplication(applicationId, user); addApplication(applicationId, user);
} finally {
writeLock.unlock();
} }
// Inform the parent queue // Inform the parent queue
@ -342,24 +378,26 @@ public void finishApplicationAttempt(FiCaSchedulerApp application,
// finish attempt logic. // finish attempt logic.
} }
private synchronized void addApplication(ApplicationId applicationId, private void addApplication(ApplicationId applicationId,
String user) { String user) {
try {
writeLock.lock();
++numApplications; ++numApplications;
LOG.info("Application added -" + LOG.info(
" appId: " + applicationId + "Application added -" + " appId: " + applicationId + " user: " + user
" user: " + user + + " leaf-queue of parent: " + getQueueName() + " #applications: "
" leaf-queue of parent: " + getQueueName() + + getNumApplications());
" #applications: " + getNumApplications()); } finally {
writeLock.unlock();
}
} }
@Override @Override
public void finishApplication(ApplicationId application, String user) { public void finishApplication(ApplicationId application, String user) {
synchronized (this) {
removeApplication(application, user); removeApplication(application, user);
}
// Inform the parent queue // Inform the parent queue
if (parent != null) { if (parent != null) {
@ -367,16 +405,18 @@ public void finishApplication(ApplicationId application, String user) {
} }
} }
private synchronized void removeApplication(ApplicationId applicationId, private void removeApplication(ApplicationId applicationId,
String user) { String user) {
try {
writeLock.lock();
--numApplications; --numApplications;
LOG.info("Application removed -" + LOG.info("Application removed -" + " appId: " + applicationId + " user: "
" appId: " + applicationId + + user + " leaf-queue of parent: " + getQueueName()
" user: " + user + + " #applications: " + getNumApplications());
" leaf-queue of parent: " + getQueueName() + } finally {
" #applications: " + getNumApplications()); writeLock.unlock();
}
} }
private String getParentName() { private String getParentName() {
@ -384,9 +424,11 @@ private String getParentName() {
} }
@Override @Override
public synchronized CSAssignment assignContainers(Resource clusterResource, public CSAssignment assignContainers(Resource clusterResource,
FiCaSchedulerNode node, ResourceLimits resourceLimits, FiCaSchedulerNode node, ResourceLimits resourceLimits,
SchedulingMode schedulingMode) { SchedulingMode schedulingMode) {
try {
writeLock.lock();
// if our queue cannot access this node, just return // if our queue cannot access this node, just return
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
&& !accessibleToPartition(node.getPartition())) { && !accessibleToPartition(node.getPartition())) {
@ -410,12 +452,13 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
// Check if this queue need more resource, simply skip allocation if this // Check if this queue need more resource, simply skip allocation if this
// queue doesn't need more resources. // queue doesn't need more resources.
if (!super.hasPendingResourceRequest(node.getPartition(), if (!super.hasPendingResourceRequest(node.getPartition(), clusterResource,
clusterResource, schedulingMode)) { schedulingMode)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Skip this queue=" + getQueuePath() LOG.debug("Skip this queue=" + getQueuePath()
+ ", because it doesn't need more resource, schedulingMode=" + ", because it doesn't need more resource, schedulingMode="
+ schedulingMode.name() + " node-partition=" + node.getPartition()); + schedulingMode.name() + " node-partition=" + node
.getPartition());
} }
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
@ -429,8 +472,8 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
return CSAssignment.NULL_ASSIGNMENT; return CSAssignment.NULL_ASSIGNMENT;
} }
CSAssignment assignment = CSAssignment assignment = new CSAssignment(Resources.createResource(0, 0),
new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL); NodeType.NODE_LOCAL);
while (canAssign(clusterResource, node)) { while (canAssign(clusterResource, node)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -442,9 +485,9 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
// This will also consider parent's limits and also continuous reservation // This will also consider parent's limits and also continuous reservation
// looking // looking
if (!super.canAssignToThisQueue(clusterResource, node.getPartition(), if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
resourceLimits, Resources.createResource( resourceLimits, Resources
getMetrics().getReservedMB(), getMetrics() .createResource(getMetrics().getReservedMB(),
.getReservedVirtualCores()), schedulingMode)) { getMetrics().getReservedVirtualCores()), schedulingMode)) {
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
getParentName(), getQueueName(), ActivityState.SKIPPED, getParentName(), getQueueName(), ActivityState.SKIPPED,
@ -458,14 +501,12 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
} }
// Schedule // Schedule
CSAssignment assignedToChild = CSAssignment assignedToChild = assignContainersToChildQueues(
assignContainersToChildQueues(clusterResource, node, resourceLimits, clusterResource, node, resourceLimits, schedulingMode);
schedulingMode);
assignment.setType(assignedToChild.getType()); assignment.setType(assignedToChild.getType());
// Done if no child-queue assigned anything // Done if no child-queue assigned anything
if (Resources.greaterThan( if (Resources.greaterThan(resourceCalculator, clusterResource,
resourceCalculator, clusterResource,
assignedToChild.getResource(), Resources.none())) { assignedToChild.getResource(), Resources.none())) {
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
@ -480,7 +521,7 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
.getFirstAllocatedOrReservedContainerId(), .getFirstAllocatedOrReservedContainerId(),
AllocationState.ALLOCATED); AllocationState.ALLOCATED);
} }
} else { } else{
if (rootQueue) { if (rootQueue) {
ActivitiesLogger.NODE.finishAllocatedNodeAllocation( ActivitiesLogger.NODE.finishAllocatedNodeAllocation(
activitiesManager, node, activitiesManager, node,
@ -495,8 +536,8 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
node.getPartition(), assignedToChild.isIncreasedAllocation()); node.getPartition(), assignedToChild.isIncreasedAllocation());
// Track resource utilization in this pass of the scheduler // Track resource utilization in this pass of the scheduler
Resources Resources.addTo(assignment.getResource(),
.addTo(assignment.getResource(), assignedToChild.getResource()); assignedToChild.getResource());
Resources.addTo(assignment.getAssignmentInformation().getAllocated(), Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
assignedToChild.getAssignmentInformation().getAllocated()); assignedToChild.getAssignmentInformation().getAllocated());
Resources.addTo(assignment.getAssignmentInformation().getReserved(), Resources.addTo(assignment.getAssignmentInformation().getReserved(),
@ -505,28 +546,21 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
assignedToChild.getAssignmentInformation().getNumAllocations()); assignedToChild.getAssignmentInformation().getNumAllocations());
assignment.getAssignmentInformation().incrReservations( assignment.getAssignmentInformation().incrReservations(
assignedToChild.getAssignmentInformation().getNumReservations()); assignedToChild.getAssignmentInformation().getNumReservations());
assignment assignment.getAssignmentInformation().getAllocationDetails().addAll(
.getAssignmentInformation() assignedToChild.getAssignmentInformation()
.getAllocationDetails() .getAllocationDetails());
.addAll( assignment.getAssignmentInformation().getReservationDetails().addAll(
assignedToChild.getAssignmentInformation().getAllocationDetails());
assignment
.getAssignmentInformation()
.getReservationDetails()
.addAll(
assignedToChild.getAssignmentInformation() assignedToChild.getAssignmentInformation()
.getReservationDetails()); .getReservationDetails());
assignment.setIncreasedAllocation(assignedToChild assignment.setIncreasedAllocation(
.isIncreasedAllocation()); assignedToChild.isIncreasedAllocation());
LOG.info("assignedContainer" + LOG.info("assignedContainer" + " queue=" + getQueueName()
" queue=" + getQueueName() + + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
" usedCapacity=" + getUsedCapacity() + + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed()
" absoluteUsedCapacity=" + getAbsoluteUsedCapacity() + + " cluster=" + clusterResource);
" used=" + queueUsage.getUsed() +
" cluster=" + clusterResource);
} else { } else{
assignment.setSkippedType(assignedToChild.getSkippedType()); assignment.setSkippedType(assignedToChild.getSkippedType());
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
@ -541,10 +575,11 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("ParentQ=" + getQueueName() LOG.debug(
+ " assignedSoFarInThisIteration=" + assignment.getResource() "ParentQ=" + getQueueName() + " assignedSoFarInThisIteration="
+ " usedCapacity=" + getUsedCapacity() + assignment.getResource() + " usedCapacity="
+ " absoluteUsedCapacity=" + getAbsoluteUsedCapacity()); + getUsedCapacity() + " absoluteUsedCapacity="
+ getAbsoluteUsedCapacity());
} }
// Do not assign more than one container if this isn't the root queue // Do not assign more than one container if this isn't the root queue
@ -552,8 +587,8 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
if (!rootQueue || assignment.getType() == NodeType.OFF_SWITCH) { if (!rootQueue || assignment.getType() == NodeType.OFF_SWITCH) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
if (rootQueue && assignment.getType() == NodeType.OFF_SWITCH) { if (rootQueue && assignment.getType() == NodeType.OFF_SWITCH) {
LOG.debug("Not assigning more than one off-switch container," + LOG.debug("Not assigning more than one off-switch container,"
" assignments so far: " + assignment); + " assignments so far: " + assignment);
} }
} }
break; break;
@ -561,6 +596,9 @@ public synchronized CSAssignment assignContainers(Resource clusterResource,
} }
return assignment; return assignment;
} finally {
writeLock.unlock();
}
} }
private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) { private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) {
@ -628,7 +666,7 @@ private Iterator<CSQueue> sortAndGetChildrenAllocationIterator(FiCaSchedulerNode
return childrenList.iterator(); return childrenList.iterator();
} }
private synchronized CSAssignment assignContainersToChildQueues( private CSAssignment assignContainersToChildQueues(
Resource cluster, FiCaSchedulerNode node, ResourceLimits limits, Resource cluster, FiCaSchedulerNode node, ResourceLimits limits,
SchedulingMode schedulingMode) { SchedulingMode schedulingMode) {
CSAssignment assignment = CSAssignment.NULL_ASSIGNMENT; CSAssignment assignment = CSAssignment.NULL_ASSIGNMENT;
@ -717,15 +755,17 @@ private void printChildQueues() {
} }
} }
private synchronized void internalReleaseResource(Resource clusterResource, private void internalReleaseResource(Resource clusterResource,
FiCaSchedulerNode node, Resource releasedResource, boolean changeResource, FiCaSchedulerNode node, Resource releasedResource, boolean changeResource,
CSQueue completedChildQueue, boolean sortQueues) { CSQueue completedChildQueue, boolean sortQueues) {
super.releaseResource(clusterResource, try {
releasedResource, node.getPartition(), writeLock.lock();
changeResource); super.releaseResource(clusterResource, releasedResource,
node.getPartition(), changeResource);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("completedContainer " + this + ", cluster=" + clusterResource); LOG.debug(
"completedContainer " + this + ", cluster=" + clusterResource);
} }
// Note that this is using an iterator on the childQueues so this can't // Note that this is using an iterator on the childQueues so this can't
@ -733,7 +773,8 @@ private synchronized void internalReleaseResource(Resource clusterResource,
// from assignContainersToChildQueues. // from assignContainersToChildQueues.
if (sortQueues) { if (sortQueues) {
// reinsert the updated queue // reinsert the updated queue
for (Iterator<CSQueue> iter = childQueues.iterator(); iter.hasNext();) { for (Iterator<CSQueue> iter = childQueues.iterator();
iter.hasNext(); ) {
CSQueue csqueue = iter.next(); CSQueue csqueue = iter.next();
if (csqueue.equals(completedChildQueue)) { if (csqueue.equals(completedChildQueue)) {
iter.remove(); iter.remove();
@ -750,6 +791,9 @@ private synchronized void internalReleaseResource(Resource clusterResource,
// sure we allocate from least usage (or order defined by queue policy) // sure we allocate from least usage (or order defined by queue policy)
// queues. // queues.
needToResortQueuesAtNextAllocation = !sortQueues; needToResortQueuesAtNextAllocation = !sortQueues;
} finally {
writeLock.unlock();
}
} }
@Override @Override
@ -806,8 +850,10 @@ public void completedContainer(Resource clusterResource,
} }
@Override @Override
public synchronized void updateClusterResource(Resource clusterResource, public void updateClusterResource(Resource clusterResource,
ResourceLimits resourceLimits) { ResourceLimits resourceLimits) {
try {
writeLock.lock();
// Update all children // Update all children
for (CSQueue childQueue : childQueues) { for (CSQueue childQueue : childQueues) {
// Get ResourceLimits of child queue before assign containers // Get ResourceLimits of child queue before assign containers
@ -819,11 +865,20 @@ public synchronized void updateClusterResource(Resource clusterResource,
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
minimumAllocation, this, labelManager, null); minimumAllocation, this, labelManager, null);
} finally {
writeLock.unlock();
}
} }
@Override @Override
public synchronized List<CSQueue> getChildQueues() { public List<CSQueue> getChildQueues() {
try {
readLock.lock();
return new ArrayList<CSQueue>(childQueues); return new ArrayList<CSQueue>(childQueues);
} finally {
readLock.unlock();
}
} }
@Override @Override
@ -832,13 +887,18 @@ public void recoverContainer(Resource clusterResource,
if (rmContainer.getState().equals(RMContainerState.COMPLETED)) { if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
return; return;
} }
// Careful! Locking order is important! // Careful! Locking order is important!
synchronized (this) { try {
FiCaSchedulerNode node = writeLock.lock();
scheduler.getNode(rmContainer.getContainer().getNodeId()); FiCaSchedulerNode node = scheduler.getNode(
rmContainer.getContainer().getNodeId());
allocateResource(clusterResource, allocateResource(clusterResource,
rmContainer.getContainer().getResource(), node.getPartition(), false); rmContainer.getContainer().getResource(), node.getPartition(), false);
} finally {
writeLock.unlock();
} }
if (parent != null) { if (parent != null) {
parent.recoverContainer(clusterResource, attempt, rmContainer); parent.recoverContainer(clusterResource, attempt, rmContainer);
} }
@ -851,11 +911,17 @@ public ActiveUsersManager getActiveUsersManager() {
} }
@Override @Override
public synchronized void collectSchedulerApplications( public void collectSchedulerApplications(
Collection<ApplicationAttemptId> apps) { Collection<ApplicationAttemptId> apps) {
try {
readLock.lock();
for (CSQueue queue : childQueues) { for (CSQueue queue : childQueues) {
queue.collectSchedulerApplications(apps); queue.collectSchedulerApplications(apps);
} }
} finally {
readLock.unlock();
}
} }
@Override @Override
@ -897,12 +963,14 @@ public void detachContainer(Resource clusterResource,
} }
} }
public synchronized int getNumApplications() { public int getNumApplications() {
return numApplications; return numApplications;
} }
synchronized void allocateResource(Resource clusterResource, void allocateResource(Resource clusterResource,
Resource resource, String nodePartition, boolean changeContainerResource) { Resource resource, String nodePartition, boolean changeContainerResource) {
try {
writeLock.lock();
super.allocateResource(clusterResource, resource, nodePartition, super.allocateResource(clusterResource, resource, nodePartition,
changeContainerResource); changeContainerResource);
@ -936,6 +1004,9 @@ synchronized void allocateResource(Resource clusterResource,
< getQueueCapacities().getAbsoluteUsedCapacity(nodePartition)) { < getQueueCapacities().getAbsoluteUsedCapacity(nodePartition)) {
killContainersToEnforceMaxQueueCapacity(nodePartition, clusterResource); killContainersToEnforceMaxQueueCapacity(nodePartition, clusterResource);
} }
} finally {
writeLock.unlock();
}
} }
private void killContainersToEnforceMaxQueueCapacity(String partition, private void killContainersToEnforceMaxQueueCapacity(String partition,

View File

@ -79,13 +79,16 @@ public PlanQueue(CapacitySchedulerContext cs, String queueName,
} }
@Override @Override
public synchronized void reinitialize(CSQueue newlyParsedQueue, public void reinitialize(CSQueue newlyParsedQueue,
Resource clusterResource) throws IOException { Resource clusterResource) throws IOException {
try {
writeLock.lock();
// Sanity check // Sanity check
if (!(newlyParsedQueue instanceof PlanQueue) if (!(newlyParsedQueue instanceof PlanQueue) || !newlyParsedQueue
|| !newlyParsedQueue.getQueuePath().equals(getQueuePath())) { .getQueuePath().equals(getQueuePath())) {
throw new IOException("Trying to reinitialize " + getQueuePath() throw new IOException(
+ " from " + newlyParsedQueue.getQueuePath()); "Trying to reinitialize " + getQueuePath() + " from "
+ newlyParsedQueue.getQueuePath());
} }
PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue; PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
@ -109,27 +112,38 @@ public synchronized void reinitialize(CSQueue newlyParsedQueue,
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 {
writeLock.unlock();
}
} }
synchronized void addChildQueue(CSQueue newQueue) void addChildQueue(CSQueue newQueue)
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException {
try {
writeLock.lock();
if (newQueue.getCapacity() > 0) { if (newQueue.getCapacity() > 0) {
throw new SchedulerDynamicEditException("Queue " + newQueue throw new SchedulerDynamicEditException(
+ " being added has non zero capacity."); "Queue " + newQueue + " being added has non zero capacity.");
} }
boolean added = this.childQueues.add(newQueue); boolean added = this.childQueues.add(newQueue);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("updateChildQueues (action: add queue): " + added + " " LOG.debug("updateChildQueues (action: add queue): " + added + " "
+ getChildQueuesToPrint()); + getChildQueuesToPrint());
} }
} finally {
writeLock.unlock();
}
} }
synchronized void removeChildQueue(CSQueue remQueue) void removeChildQueue(CSQueue remQueue)
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException {
try {
writeLock.lock();
if (remQueue.getCapacity() > 0) { if (remQueue.getCapacity() > 0) {
throw new SchedulerDynamicEditException("Queue " + remQueue throw new SchedulerDynamicEditException(
+ " being removed has non zero capacity."); "Queue " + remQueue + " being removed has non zero capacity.");
} }
Iterator<CSQueue> qiter = childQueues.iterator(); Iterator<CSQueue> qiter = childQueues.iterator();
while (qiter.hasNext()) { while (qiter.hasNext()) {
@ -141,14 +155,22 @@ synchronized void removeChildQueue(CSQueue remQueue)
} }
} }
} }
} finally {
writeLock.unlock();
}
} }
protected synchronized float sumOfChildCapacities() { protected float sumOfChildCapacities() {
try {
writeLock.lock();
float ret = 0; float ret = 0;
for (CSQueue l : childQueues) { for (CSQueue l : childQueues) {
ret += l.getCapacity(); ret += l.getCapacity();
} }
return ret; return ret;
} finally {
writeLock.unlock();
}
} }
private void updateQuotas(int userLimit, float userLimitFactor, private void updateQuotas(int userLimit, float userLimitFactor,

View File

@ -51,13 +51,16 @@ public ReservationQueue(CapacitySchedulerContext cs, String queueName,
} }
@Override @Override
public synchronized void reinitialize(CSQueue newlyParsedQueue, public void reinitialize(CSQueue newlyParsedQueue,
Resource clusterResource) throws IOException { Resource clusterResource) throws IOException {
try {
writeLock.lock();
// Sanity check // Sanity check
if (!(newlyParsedQueue instanceof ReservationQueue) if (!(newlyParsedQueue instanceof ReservationQueue) || !newlyParsedQueue
|| !newlyParsedQueue.getQueuePath().equals(getQueuePath())) { .getQueuePath().equals(getQueuePath())) {
throw new IOException("Trying to reinitialize " + getQueuePath() throw new IOException(
+ " from " + newlyParsedQueue.getQueuePath()); "Trying to reinitialize " + getQueuePath() + " from "
+ newlyParsedQueue.getQueuePath());
} }
super.reinitialize(newlyParsedQueue, clusterResource); super.reinitialize(newlyParsedQueue, clusterResource);
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
@ -67,6 +70,9 @@ public synchronized void reinitialize(CSQueue newlyParsedQueue,
parent.getUserLimitFactor(), parent.getUserLimitFactor(),
parent.getMaxApplicationsForReservations(), parent.getMaxApplicationsForReservations(),
parent.getMaxApplicationsPerUserForReservation()); parent.getMaxApplicationsPerUserForReservation());
} finally {
writeLock.unlock();
}
} }
/** /**
@ -77,8 +83,10 @@ public synchronized void reinitialize(CSQueue newlyParsedQueue,
* maxCapacity, etc..) * maxCapacity, etc..)
* @throws SchedulerDynamicEditException * @throws SchedulerDynamicEditException
*/ */
public synchronized void setEntitlement(QueueEntitlement entitlement) public void setEntitlement(QueueEntitlement entitlement)
throws SchedulerDynamicEditException { throws SchedulerDynamicEditException {
try {
writeLock.lock();
float capacity = entitlement.getCapacity(); float capacity = entitlement.getCapacity();
if (capacity < 0 || capacity > 1.0f) { if (capacity < 0 || capacity > 1.0f) {
throw new SchedulerDynamicEditException( throw new SchedulerDynamicEditException(
@ -90,8 +98,11 @@ public synchronized void setEntitlement(QueueEntitlement entitlement)
// this might be revised later // this might be revised later
setMaxCapacity(entitlement.getMaxCapacity()); setMaxCapacity(entitlement.getMaxCapacity());
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("successfully changed to " + capacity + " for queue " LOG.debug("successfully changed to " + capacity + " for queue " + this
+ this.getQueueName()); .getQueueName());
}
} finally {
writeLock.unlock();
} }
} }

View File

@ -828,8 +828,8 @@ public RMNodeLabelsManager createNodeLabelManager() {
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,