YARN-3142. Improve locks in AppSchedulingInfo. (Varun Saxena via wangda)
This commit is contained in:
parent
875062b5bc
commit
1831be8e73
@ -30,6 +30,7 @@
|
|||||||
import java.util.concurrent.ConcurrentSkipListMap;
|
import java.util.concurrent.ConcurrentSkipListMap;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
@ -67,7 +68,8 @@ public class AppSchedulingInfo {
|
|||||||
|
|
||||||
private Queue queue;
|
private Queue queue;
|
||||||
private ActiveUsersManager activeUsersManager;
|
private ActiveUsersManager activeUsersManager;
|
||||||
private boolean pending = true; // whether accepted/allocated by scheduler
|
// whether accepted/allocated by scheduler
|
||||||
|
private volatile boolean pending = true;
|
||||||
private ResourceUsage appResourceUsage;
|
private ResourceUsage appResourceUsage;
|
||||||
|
|
||||||
private AtomicBoolean userBlacklistChanged = new AtomicBoolean(false);
|
private AtomicBoolean userBlacklistChanged = new AtomicBoolean(false);
|
||||||
@ -86,6 +88,9 @@ public class AppSchedulingInfo {
|
|||||||
SchedContainerChangeRequest>>> containerIncreaseRequestMap =
|
SchedContainerChangeRequest>>> containerIncreaseRequestMap =
|
||||||
new ConcurrentHashMap<>();
|
new ConcurrentHashMap<>();
|
||||||
|
|
||||||
|
private final ReentrantReadWriteLock.ReadLock readLock;
|
||||||
|
private final ReentrantReadWriteLock.WriteLock writeLock;
|
||||||
|
|
||||||
public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
|
public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
|
||||||
String user, Queue queue, ActiveUsersManager activeUsersManager,
|
String user, Queue queue, ActiveUsersManager activeUsersManager,
|
||||||
long epoch, ResourceUsage appResourceUsage) {
|
long epoch, ResourceUsage appResourceUsage) {
|
||||||
@ -97,6 +102,10 @@ public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
|
|||||||
this.containerIdCounter = new AtomicLong(
|
this.containerIdCounter = new AtomicLong(
|
||||||
epoch << ResourceManager.EPOCH_BIT_SHIFT);
|
epoch << ResourceManager.EPOCH_BIT_SHIFT);
|
||||||
this.appResourceUsage = appResourceUsage;
|
this.appResourceUsage = appResourceUsage;
|
||||||
|
|
||||||
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||||
|
readLock = lock.readLock();
|
||||||
|
writeLock = lock.writeLock();
|
||||||
}
|
}
|
||||||
|
|
||||||
public ApplicationId getApplicationId() {
|
public ApplicationId getApplicationId() {
|
||||||
@ -115,11 +124,16 @@ public long getNewContainerId() {
|
|||||||
return this.containerIdCounter.incrementAndGet();
|
return this.containerIdCounter.incrementAndGet();
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized String getQueueName() {
|
public String getQueueName() {
|
||||||
|
try {
|
||||||
|
this.readLock.lock();
|
||||||
return queue.getQueueName();
|
return queue.getQueueName();
|
||||||
|
} finally {
|
||||||
|
this.readLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized boolean isPending() {
|
public boolean isPending() {
|
||||||
return pending;
|
return pending;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -130,38 +144,50 @@ public Set<String> getRequestedPartitions() {
|
|||||||
/**
|
/**
|
||||||
* Clear any pending requests from this application.
|
* Clear any pending requests from this application.
|
||||||
*/
|
*/
|
||||||
private synchronized void clearRequests() {
|
private void clearRequests() {
|
||||||
schedulerKeys.clear();
|
schedulerKeys.clear();
|
||||||
resourceRequestMap.clear();
|
resourceRequestMap.clear();
|
||||||
LOG.info("Application " + applicationId + " requests cleared");
|
LOG.info("Application " + applicationId + " requests cleared");
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized boolean hasIncreaseRequest(NodeId nodeId) {
|
public boolean hasIncreaseRequest(NodeId nodeId) {
|
||||||
|
try {
|
||||||
|
this.readLock.lock();
|
||||||
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
|
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
|
||||||
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
|
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
|
||||||
return requestsOnNode == null ? false : requestsOnNode.size() > 0;
|
return requestsOnNode == null ? false : requestsOnNode.size() > 0;
|
||||||
|
} finally {
|
||||||
|
this.readLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized Map<ContainerId, SchedContainerChangeRequest>
|
public Map<ContainerId, SchedContainerChangeRequest>
|
||||||
getIncreaseRequests(NodeId nodeId, SchedulerRequestKey schedulerKey) {
|
getIncreaseRequests(NodeId nodeId, SchedulerRequestKey schedulerKey) {
|
||||||
|
try {
|
||||||
|
this.readLock.lock();
|
||||||
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
|
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
|
||||||
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
|
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
|
||||||
return requestsOnNode == null ? null : requestsOnNode.get(
|
return requestsOnNode == null ? null : requestsOnNode.get(
|
||||||
schedulerKey);
|
schedulerKey);
|
||||||
|
} finally {
|
||||||
|
this.readLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* return true if any of the existing increase requests are updated,
|
* return true if any of the existing increase requests are updated,
|
||||||
* false if none of them are updated
|
* false if none of them are updated
|
||||||
*/
|
*/
|
||||||
public synchronized boolean updateIncreaseRequests(
|
public boolean updateIncreaseRequests(
|
||||||
List<SchedContainerChangeRequest> increaseRequests) {
|
List<SchedContainerChangeRequest> increaseRequests) {
|
||||||
boolean resourceUpdated = false;
|
boolean resourceUpdated = false;
|
||||||
|
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
for (SchedContainerChangeRequest r : increaseRequests) {
|
for (SchedContainerChangeRequest r : increaseRequests) {
|
||||||
if (r.getRMContainer().getState() != RMContainerState.RUNNING) {
|
if (r.getRMContainer().getState() != RMContainerState.RUNNING) {
|
||||||
LOG.warn("rmContainer's state is not RUNNING, for increase request with"
|
LOG.warn("rmContainer's state is not RUNNING, for increase request"
|
||||||
+ " container-id=" + r.getContainerId());
|
+ " with container-id=" + r.getContainerId());
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
@ -212,6 +238,9 @@ public synchronized boolean updateIncreaseRequests(
|
|||||||
insertIncreaseRequest(r);
|
insertIncreaseRequest(r);
|
||||||
}
|
}
|
||||||
return resourceUpdated;
|
return resourceUpdated;
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -275,8 +304,10 @@ private void decrementSchedulerKeyReference(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized boolean removeIncreaseRequest(NodeId nodeId,
|
public boolean removeIncreaseRequest(NodeId nodeId,
|
||||||
SchedulerRequestKey schedulerKey, ContainerId containerId) {
|
SchedulerRequestKey schedulerKey, ContainerId containerId) {
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
|
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
|
||||||
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
|
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
|
||||||
if (null == requestsOnNode) {
|
if (null == requestsOnNode) {
|
||||||
@ -316,10 +347,15 @@ public synchronized boolean removeIncreaseRequest(NodeId nodeId,
|
|||||||
}
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
|
public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
|
||||||
SchedulerRequestKey schedulerKey, ContainerId containerId) {
|
SchedulerRequestKey schedulerKey, ContainerId containerId) {
|
||||||
|
try {
|
||||||
|
this.readLock.lock();
|
||||||
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
|
Map<SchedulerRequestKey, Map<ContainerId, SchedContainerChangeRequest>>
|
||||||
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
|
requestsOnNode = containerIncreaseRequestMap.get(nodeId);
|
||||||
if (null == requestsOnNode) {
|
if (null == requestsOnNode) {
|
||||||
@ -330,6 +366,9 @@ public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
|
|||||||
requestsOnNode.get(schedulerKey);
|
requestsOnNode.get(schedulerKey);
|
||||||
return requestsOnNodeWithPriority == null ? null
|
return requestsOnNodeWithPriority == null ? null
|
||||||
: requestsOnNodeWithPriority.get(containerId);
|
: requestsOnNodeWithPriority.get(containerId);
|
||||||
|
} finally {
|
||||||
|
this.readLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -343,12 +382,13 @@ public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
|
|||||||
* recover ResourceRequest on preemption
|
* recover ResourceRequest on preemption
|
||||||
* @return true if any resource was updated, false otherwise
|
* @return true if any resource was updated, false otherwise
|
||||||
*/
|
*/
|
||||||
public synchronized boolean updateResourceRequests(
|
public boolean updateResourceRequests(List<ResourceRequest> requests,
|
||||||
List<ResourceRequest> requests,
|
|
||||||
boolean recoverPreemptedRequestForAContainer) {
|
boolean recoverPreemptedRequestForAContainer) {
|
||||||
// Flag to track if any incoming requests update "ANY" requests
|
// Flag to track if any incoming requests update "ANY" requests
|
||||||
boolean anyResourcesUpdated = false;
|
boolean anyResourcesUpdated = false;
|
||||||
|
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
// Update resource requests
|
// Update resource requests
|
||||||
for (ResourceRequest request : requests) {
|
for (ResourceRequest request : requests) {
|
||||||
SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request);
|
SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request);
|
||||||
@ -376,7 +416,8 @@ public synchronized boolean updateResourceRequests(
|
|||||||
if (resourceName.equals(ResourceRequest.ANY)) {
|
if (resourceName.equals(ResourceRequest.ANY)) {
|
||||||
//update the applications requested labels set
|
//update the applications requested labels set
|
||||||
requestedPartitions.add(request.getNodeLabelExpression() == null
|
requestedPartitions.add(request.getNodeLabelExpression() == null
|
||||||
? RMNodeLabelsManager.NO_LABEL : request.getNodeLabelExpression());
|
? RMNodeLabelsManager.NO_LABEL :
|
||||||
|
request.getNodeLabelExpression());
|
||||||
|
|
||||||
anyResourcesUpdated = true;
|
anyResourcesUpdated = true;
|
||||||
|
|
||||||
@ -386,6 +427,9 @@ public synchronized boolean updateResourceRequests(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
return anyResourcesUpdated;
|
return anyResourcesUpdated;
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void updatePendingResources(ResourceRequest lastRequest,
|
private void updatePendingResources(ResourceRequest lastRequest,
|
||||||
@ -529,34 +573,49 @@ public boolean getAndResetBlacklistChanged() {
|
|||||||
return userBlacklistChanged.getAndSet(false);
|
return userBlacklistChanged.getAndSet(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized Collection<SchedulerRequestKey> getSchedulerKeys() {
|
public Collection<SchedulerRequestKey> getSchedulerKeys() {
|
||||||
return schedulerKeys.keySet();
|
return schedulerKeys.keySet();
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized Map<String, ResourceRequest> getResourceRequests(
|
public Map<String, ResourceRequest> getResourceRequests(
|
||||||
SchedulerRequestKey schedulerKey) {
|
SchedulerRequestKey schedulerKey) {
|
||||||
return resourceRequestMap.get(schedulerKey);
|
return resourceRequestMap.get(schedulerKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized List<ResourceRequest> getAllResourceRequests() {
|
public List<ResourceRequest> getAllResourceRequests() {
|
||||||
List<ResourceRequest> ret = new ArrayList<>();
|
List<ResourceRequest> ret = new ArrayList<>();
|
||||||
|
try {
|
||||||
|
this.readLock.lock();
|
||||||
for (Map<String, ResourceRequest> r : resourceRequestMap.values()) {
|
for (Map<String, ResourceRequest> r : resourceRequestMap.values()) {
|
||||||
ret.addAll(r.values());
|
ret.addAll(r.values());
|
||||||
}
|
}
|
||||||
|
} finally {
|
||||||
|
this.readLock.unlock();
|
||||||
|
}
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized ResourceRequest getResourceRequest(
|
public ResourceRequest getResourceRequest(SchedulerRequestKey schedulerKey,
|
||||||
SchedulerRequestKey schedulerKey, String resourceName) {
|
String resourceName) {
|
||||||
|
try {
|
||||||
|
this.readLock.lock();
|
||||||
Map<String, ResourceRequest> nodeRequests =
|
Map<String, ResourceRequest> nodeRequests =
|
||||||
resourceRequestMap.get(schedulerKey);
|
resourceRequestMap.get(schedulerKey);
|
||||||
return (nodeRequests == null) ? null : nodeRequests.get(resourceName);
|
return (nodeRequests == null) ? null : nodeRequests.get(resourceName);
|
||||||
|
} finally {
|
||||||
|
this.readLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized Resource getResource(SchedulerRequestKey schedulerKey) {
|
public Resource getResource(SchedulerRequestKey schedulerKey) {
|
||||||
|
try {
|
||||||
|
this.readLock.lock();
|
||||||
ResourceRequest request =
|
ResourceRequest request =
|
||||||
getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
||||||
return (request == null) ? null : request.getCapability();
|
return (request == null) ? null : request.getCapability();
|
||||||
|
} finally {
|
||||||
|
this.readLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -582,8 +641,7 @@ public boolean isPlaceBlacklisted(String resourceName,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void increaseContainer(
|
public void increaseContainer(SchedContainerChangeRequest increaseRequest) {
|
||||||
SchedContainerChangeRequest increaseRequest) {
|
|
||||||
NodeId nodeId = increaseRequest.getNodeId();
|
NodeId nodeId = increaseRequest.getNodeId();
|
||||||
SchedulerRequestKey schedulerKey =
|
SchedulerRequestKey schedulerKey =
|
||||||
increaseRequest.getRMContainer().getAllocatedSchedulerKey();
|
increaseRequest.getRMContainer().getAllocatedSchedulerKey();
|
||||||
@ -596,16 +654,21 @@ public synchronized void increaseContainer(
|
|||||||
+ increaseRequest.getNodeId() + " user=" + user + " resource="
|
+ increaseRequest.getNodeId() + " user=" + user + " resource="
|
||||||
+ deltaCapacity);
|
+ deltaCapacity);
|
||||||
}
|
}
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
// Set queue metrics
|
// Set queue metrics
|
||||||
queue.getMetrics().allocateResources(user, deltaCapacity);
|
queue.getMetrics().allocateResources(user, deltaCapacity);
|
||||||
// remove the increase request from pending increase request map
|
// remove the increase request from pending increase request map
|
||||||
removeIncreaseRequest(nodeId, schedulerKey, containerId);
|
removeIncreaseRequest(nodeId, schedulerKey, containerId);
|
||||||
// update usage
|
// update usage
|
||||||
appResourceUsage.incUsed(increaseRequest.getNodePartition(), deltaCapacity);
|
appResourceUsage.incUsed(increaseRequest.getNodePartition(),
|
||||||
|
deltaCapacity);
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void decreaseContainer(
|
public void decreaseContainer(SchedContainerChangeRequest decreaseRequest) {
|
||||||
SchedContainerChangeRequest decreaseRequest) {
|
|
||||||
// Delta is negative when it's a decrease request
|
// Delta is negative when it's a decrease request
|
||||||
Resource absDelta = Resources.negate(decreaseRequest.getDeltaCapacity());
|
Resource absDelta = Resources.negate(decreaseRequest.getDeltaCapacity());
|
||||||
|
|
||||||
@ -616,11 +679,16 @@ public synchronized void decreaseContainer(
|
|||||||
+ absDelta);
|
+ absDelta);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
// Set queue metrics
|
// Set queue metrics
|
||||||
queue.getMetrics().releaseResources(user, absDelta);
|
queue.getMetrics().releaseResources(user, absDelta);
|
||||||
|
|
||||||
// update usage
|
// update usage
|
||||||
appResourceUsage.decUsed(decreaseRequest.getNodePartition(), absDelta);
|
appResourceUsage.decUsed(decreaseRequest.getNodePartition(), absDelta);
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -633,10 +701,12 @@ public synchronized void decreaseContainer(
|
|||||||
* @param containerAllocated Container Allocated
|
* @param containerAllocated Container Allocated
|
||||||
* @return List of ResourceRequests
|
* @return List of ResourceRequests
|
||||||
*/
|
*/
|
||||||
public synchronized List<ResourceRequest> allocate(NodeType type,
|
public List<ResourceRequest> allocate(NodeType type, SchedulerNode node,
|
||||||
SchedulerNode node, SchedulerRequestKey schedulerKey,
|
SchedulerRequestKey schedulerKey, ResourceRequest request,
|
||||||
ResourceRequest request, Container containerAllocated) {
|
Container containerAllocated) {
|
||||||
List<ResourceRequest> resourceRequests = new ArrayList<>();
|
List<ResourceRequest> resourceRequests = new ArrayList<>();
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
if (type == NodeType.NODE_LOCAL) {
|
if (type == NodeType.NODE_LOCAL) {
|
||||||
allocateNodeLocal(node, schedulerKey, request, resourceRequests);
|
allocateNodeLocal(node, schedulerKey, request, resourceRequests);
|
||||||
} else if (type == NodeType.RACK_LOCAL) {
|
} else if (type == NodeType.RACK_LOCAL) {
|
||||||
@ -663,13 +733,16 @@ public synchronized List<ResourceRequest> allocate(NodeType type,
|
|||||||
metrics.allocateResources(user, 1, request.getCapability(), true);
|
metrics.allocateResources(user, 1, request.getCapability(), true);
|
||||||
metrics.incrNodeTypeAggregations(user, type);
|
metrics.incrNodeTypeAggregations(user, type);
|
||||||
return resourceRequests;
|
return resourceRequests;
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The {@link ResourceScheduler} is allocating data-local resources to the
|
* The {@link ResourceScheduler} is allocating data-local resources to the
|
||||||
* application.
|
* application.
|
||||||
*/
|
*/
|
||||||
private synchronized void allocateNodeLocal(SchedulerNode node,
|
private void allocateNodeLocal(SchedulerNode node,
|
||||||
SchedulerRequestKey schedulerKey, ResourceRequest nodeLocalRequest,
|
SchedulerRequestKey schedulerKey, ResourceRequest nodeLocalRequest,
|
||||||
List<ResourceRequest> resourceRequests) {
|
List<ResourceRequest> resourceRequests) {
|
||||||
// Update future requirements
|
// Update future requirements
|
||||||
@ -701,7 +774,7 @@ private void decResourceRequest(String resourceName,
|
|||||||
* The {@link ResourceScheduler} is allocating data-local resources to the
|
* The {@link ResourceScheduler} is allocating data-local resources to the
|
||||||
* application.
|
* application.
|
||||||
*/
|
*/
|
||||||
private synchronized void allocateRackLocal(SchedulerNode node,
|
private void allocateRackLocal(SchedulerNode node,
|
||||||
SchedulerRequestKey schedulerKey, ResourceRequest rackLocalRequest,
|
SchedulerRequestKey schedulerKey, ResourceRequest rackLocalRequest,
|
||||||
List<ResourceRequest> resourceRequests) {
|
List<ResourceRequest> resourceRequests) {
|
||||||
// Update future requirements
|
// Update future requirements
|
||||||
@ -720,8 +793,8 @@ private synchronized void allocateRackLocal(SchedulerNode node,
|
|||||||
* The {@link ResourceScheduler} is allocating data-local resources to the
|
* The {@link ResourceScheduler} is allocating data-local resources to the
|
||||||
* application.
|
* application.
|
||||||
*/
|
*/
|
||||||
private synchronized void allocateOffSwitch(
|
private void allocateOffSwitch(ResourceRequest offSwitchRequest,
|
||||||
ResourceRequest offSwitchRequest, List<ResourceRequest> resourceRequests,
|
List<ResourceRequest> resourceRequests,
|
||||||
SchedulerRequestKey schedulerKey) {
|
SchedulerRequestKey schedulerKey) {
|
||||||
// Update future requirements
|
// Update future requirements
|
||||||
decrementOutstanding(offSwitchRequest, schedulerKey);
|
decrementOutstanding(offSwitchRequest, schedulerKey);
|
||||||
@ -729,8 +802,8 @@ private synchronized void allocateOffSwitch(
|
|||||||
resourceRequests.add(cloneResourceRequest(offSwitchRequest));
|
resourceRequests.add(cloneResourceRequest(offSwitchRequest));
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void decrementOutstanding(
|
private void decrementOutstanding(ResourceRequest offSwitchRequest,
|
||||||
ResourceRequest offSwitchRequest, SchedulerRequestKey schedulerKey) {
|
SchedulerRequestKey schedulerKey) {
|
||||||
int numOffSwitchContainers = offSwitchRequest.getNumContainers() - 1;
|
int numOffSwitchContainers = offSwitchRequest.getNumContainers() - 1;
|
||||||
|
|
||||||
// Do not remove ANY
|
// Do not remove ANY
|
||||||
@ -749,13 +822,15 @@ private synchronized void decrementOutstanding(
|
|||||||
offSwitchRequest.getCapability());
|
offSwitchRequest.getCapability());
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void checkForDeactivation() {
|
private void checkForDeactivation() {
|
||||||
if (schedulerKeys.isEmpty()) {
|
if (schedulerKeys.isEmpty()) {
|
||||||
activeUsersManager.deactivateApplication(user, applicationId);
|
activeUsersManager.deactivateApplication(user, applicationId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void move(Queue newQueue) {
|
public void move(Queue newQueue) {
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
QueueMetrics oldMetrics = queue.getMetrics();
|
QueueMetrics oldMetrics = queue.getMetrics();
|
||||||
QueueMetrics newMetrics = newQueue.getMetrics();
|
QueueMetrics newMetrics = newQueue.getMetrics();
|
||||||
for (Map<String, ResourceRequest> asks : resourceRequestMap.values()) {
|
for (Map<String, ResourceRequest> asks : resourceRequestMap.values()) {
|
||||||
@ -779,10 +854,15 @@ public synchronized void move(Queue newQueue) {
|
|||||||
activeUsersManager = newQueue.getActiveUsersManager();
|
activeUsersManager = newQueue.getActiveUsersManager();
|
||||||
activeUsersManager.activateApplication(user, applicationId);
|
activeUsersManager.activateApplication(user, applicationId);
|
||||||
this.queue = newQueue;
|
this.queue = newQueue;
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void stop() {
|
public void stop() {
|
||||||
// clear pending resources metrics for the application
|
// clear pending resources metrics for the application
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
QueueMetrics metrics = queue.getMetrics();
|
QueueMetrics metrics = queue.getMetrics();
|
||||||
for (Map<String, ResourceRequest> asks : resourceRequestMap.values()) {
|
for (Map<String, ResourceRequest> asks : resourceRequestMap.values()) {
|
||||||
ResourceRequest request = asks.get(ResourceRequest.ANY);
|
ResourceRequest request = asks.get(ResourceRequest.ANY);
|
||||||
@ -801,13 +881,21 @@ public synchronized void stop() {
|
|||||||
|
|
||||||
// Clear requests themselves
|
// Clear requests themselves
|
||||||
clearRequests();
|
clearRequests();
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void setQueue(Queue queue) {
|
public void setQueue(Queue queue) {
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
this.queue = queue;
|
this.queue = queue;
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public Set<String> getBlackList() {
|
private Set<String> getBlackList() {
|
||||||
return this.placesBlacklistedByApp;
|
return this.placesBlacklistedByApp;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -817,14 +905,16 @@ public Set<String> getBlackListCopy() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void transferStateFromPreviousAppSchedulingInfo(
|
public void transferStateFromPreviousAppSchedulingInfo(
|
||||||
AppSchedulingInfo appInfo) {
|
AppSchedulingInfo appInfo) {
|
||||||
// This should not require locking the userBlacklist since it will not be
|
// This should not require locking the placesBlacklistedByApp since it will
|
||||||
// used by this instance until after setCurrentAppAttempt.
|
// not be used by this instance until after setCurrentAppAttempt.
|
||||||
this.placesBlacklistedByApp = appInfo.getBlackList();
|
this.placesBlacklistedByApp = appInfo.getBlackList();
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void recoverContainer(RMContainer rmContainer) {
|
public void recoverContainer(RMContainer rmContainer) {
|
||||||
|
try {
|
||||||
|
this.writeLock.lock();
|
||||||
QueueMetrics metrics = queue.getMetrics();
|
QueueMetrics metrics = queue.getMetrics();
|
||||||
if (pending) {
|
if (pending) {
|
||||||
// If there was any container to recover, the application was
|
// If there was any container to recover, the application was
|
||||||
@ -840,6 +930,9 @@ public synchronized void recoverContainer(RMContainer rmContainer) {
|
|||||||
|
|
||||||
metrics.allocateResources(user, 1, rmContainer.getAllocatedResource(),
|
metrics.allocateResources(user, 1, rmContainer.getAllocatedResource(),
|
||||||
false);
|
false);
|
||||||
|
} finally {
|
||||||
|
this.writeLock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public ResourceRequest cloneResourceRequest(ResourceRequest request) {
|
public ResourceRequest cloneResourceRequest(ResourceRequest request) {
|
||||||
|
Loading…
x
Reference in New Issue
Block a user