YARN-3139. Improve locks in AbstractYarnScheduler/CapacityScheduler/FairScheduler. Contributed by Wangda Tan

This commit is contained in:
Jian He 2016-10-04 17:23:13 -07:00
parent 44f48ee96e
commit 31f8da22d0
7 changed files with 1784 additions and 1520 deletions

View File

@ -211,10 +211,7 @@ public class RMServerUtils {
} }
/** /**
* Validate increase/decrease request. This function must be called under * Validate increase/decrease request.
* the queue lock to make sure that the access to container resource is
* atomic. Refer to LeafQueue.decreaseContainer() and
* CapacityScheduelr.updateIncreaseRequests()
* <pre> * <pre>
* - Throw exception when any other error happens * - Throw exception when any other error happens
* </pre> * </pre>

View File

@ -28,6 +28,7 @@ import java.util.Set;
import java.util.Timer; import java.util.Timer;
import java.util.TimerTask; import java.util.TimerTask;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
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;
@ -72,8 +73,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReco
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
.LeafQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -94,7 +93,7 @@ public abstract class AbstractYarnScheduler
protected Resource minimumAllocation; protected Resource minimumAllocation;
protected RMContext rmContext; protected volatile RMContext rmContext;
private volatile Priority maxClusterLevelAppPriority; private volatile Priority maxClusterLevelAppPriority;
@ -112,6 +111,18 @@ public abstract class AbstractYarnScheduler
protected static final Allocation EMPTY_ALLOCATION = new Allocation( protected static final Allocation EMPTY_ALLOCATION = new Allocation(
EMPTY_CONTAINER_LIST, Resources.createResource(0), null, null, null); EMPTY_CONTAINER_LIST, Resources.createResource(0), null, null, null);
protected final ReentrantReadWriteLock.ReadLock readLock;
/*
* Use writeLock for any of operations below:
* - queue change (hierarchy / configuration / container allocation)
* - application(add/remove/allocate-container, but not include container
* finish)
* - node (add/remove/change-resource/container-allocation, but not include
* container finish)
*/
protected final ReentrantReadWriteLock.WriteLock writeLock;
/** /**
* Construct the service. * Construct the service.
* *
@ -119,6 +130,9 @@ public abstract class AbstractYarnScheduler
*/ */
public AbstractYarnScheduler(String name) { public AbstractYarnScheduler(String name) {
super(name); super(name);
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
readLock = lock.readLock();
writeLock = lock.writeLock();
} }
@Override @Override
@ -141,6 +155,10 @@ public abstract class AbstractYarnScheduler
return nodeTracker; return nodeTracker;
} }
/*
* YARN-3136 removed synchronized lock for this method for performance
* purposes
*/
public List<Container> getTransferredContainers( public List<Container> getTransferredContainers(
ApplicationAttemptId currentAttempt) { ApplicationAttemptId currentAttempt) {
ApplicationId appId = currentAttempt.getApplicationId(); ApplicationId appId = currentAttempt.getApplicationId();
@ -155,9 +173,8 @@ public abstract class AbstractYarnScheduler
} }
Collection<RMContainer> liveContainers = Collection<RMContainer> liveContainers =
app.getCurrentAppAttempt().getLiveContainers(); app.getCurrentAppAttempt().getLiveContainers();
ContainerId amContainerId = ContainerId amContainerId = rmContext.getRMApps().get(appId)
rmContext.getRMApps().get(appId).getCurrentAppAttempt() .getCurrentAppAttempt().getMasterContainer().getId();
.getMasterContainer().getId();
for (RMContainer rmContainer : liveContainers) { for (RMContainer rmContainer : liveContainers) {
if (!rmContainer.getContainerId().equals(amContainerId)) { if (!rmContainer.getContainerId().equals(amContainerId)) {
containerList.add(rmContainer.getContainer()); containerList.add(rmContainer.getContainer());
@ -211,54 +228,59 @@ public abstract class AbstractYarnScheduler
nodeTracker.setConfiguredMaxAllocation(maximumAllocation); nodeTracker.setConfiguredMaxAllocation(maximumAllocation);
} }
protected synchronized void containerLaunchedOnNode( protected void containerLaunchedOnNode(
ContainerId containerId, SchedulerNode node) { ContainerId containerId, SchedulerNode node) {
// Get the application for the finished container try {
SchedulerApplicationAttempt application = readLock.lock();
getCurrentAttemptForContainer(containerId); // Get the application for the finished container
if (application == null) { SchedulerApplicationAttempt application = getCurrentAttemptForContainer(
LOG.info("Unknown application " + containerId.getApplicationAttemptId() containerId);
.getApplicationId() + " launched container " + containerId if (application == null) {
+ " on node: " + node); LOG.info("Unknown application " + containerId.getApplicationAttemptId()
this.rmContext.getDispatcher().getEventHandler() .getApplicationId() + " launched container " + containerId
.handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId)); + " on node: " + node);
return; this.rmContext.getDispatcher().getEventHandler().handle(
} new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
return;
}
application.containerLaunchedOnNode(containerId, node.getNodeID()); application.containerLaunchedOnNode(containerId, node.getNodeID());
} finally {
readLock.unlock();
}
} }
protected void containerIncreasedOnNode(ContainerId containerId, protected void containerIncreasedOnNode(ContainerId containerId,
SchedulerNode node, Container increasedContainerReportedByNM) { SchedulerNode node, Container increasedContainerReportedByNM) {
/*
* No lock is required, as this method is protected by scheduler's writeLock
*/
// Get the application for the finished container // Get the application for the finished container
SchedulerApplicationAttempt application = SchedulerApplicationAttempt application = getCurrentAttemptForContainer(
getCurrentAttemptForContainer(containerId); containerId);
if (application == null) { if (application == null) {
LOG.info("Unknown application " LOG.info("Unknown application " + containerId.getApplicationAttemptId()
+ containerId.getApplicationAttemptId().getApplicationId() .getApplicationId() + " increased container " + containerId
+ " increased container " + containerId + " on node: " + node); + " on node: " + node);
this.rmContext.getDispatcher().getEventHandler() this.rmContext.getDispatcher().getEventHandler().handle(
.handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId)); new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
return; return;
} }
LeafQueue leafQueue = (LeafQueue) application.getQueue();
synchronized (leafQueue) { RMContainer rmContainer = getRMContainer(containerId);
RMContainer rmContainer = getRMContainer(containerId); if (rmContainer == null) {
if (rmContainer == null) { // Some unknown container sneaked into the system. Kill it.
// Some unknown container sneaked into the system. Kill it. this.rmContext.getDispatcher().getEventHandler().handle(
this.rmContext.getDispatcher().getEventHandler() new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
.handle(new RMNodeCleanContainerEvent( return;
node.getNodeID(), containerId));
return;
}
rmContainer.handle(new RMContainerNMDoneChangeResourceEvent(
containerId, increasedContainerReportedByNM.getResource()));
} }
rmContainer.handle(new RMContainerNMDoneChangeResourceEvent(containerId,
increasedContainerReportedByNM.getResource()));
} }
public T getApplicationAttempt(ApplicationAttemptId applicationAttemptId) { public T getApplicationAttempt(ApplicationAttemptId applicationAttemptId) {
SchedulerApplication<T> app = SchedulerApplication<T> app = applications.get(
applications.get(applicationAttemptId.getApplicationId()); applicationAttemptId.getApplicationId());
return app == null ? null : app.getCurrentAppAttempt(); return app == null ? null : app.getCurrentAppAttempt();
} }
@ -338,96 +360,101 @@ public abstract class AbstractYarnScheduler
} }
} }
public synchronized void recoverContainersOnNode( public void recoverContainersOnNode(
List<NMContainerStatus> containerReports, RMNode nm) { List<NMContainerStatus> containerReports, RMNode nm) {
if (!rmContext.isWorkPreservingRecoveryEnabled() try {
|| containerReports == null writeLock.lock();
|| (containerReports != null && containerReports.isEmpty())) { if (!rmContext.isWorkPreservingRecoveryEnabled()
return; || containerReports == null || (containerReports != null
} && containerReports.isEmpty())) {
return;
for (NMContainerStatus container : containerReports) {
ApplicationId appId =
container.getContainerId().getApplicationAttemptId().getApplicationId();
RMApp rmApp = rmContext.getRMApps().get(appId);
if (rmApp == null) {
LOG.error("Skip recovering container " + container
+ " for unknown application.");
killOrphanContainerOnNode(nm, container);
continue;
} }
SchedulerApplication<T> schedulerApp = applications.get(appId); for (NMContainerStatus container : containerReports) {
if (schedulerApp == null) { ApplicationId appId =
LOG.info("Skip recovering container " + container container.getContainerId().getApplicationAttemptId()
+ " for unknown SchedulerApplication. Application current state is " .getApplicationId();
+ rmApp.getState()); RMApp rmApp = rmContext.getRMApps().get(appId);
killOrphanContainerOnNode(nm, container); if (rmApp == null) {
continue; LOG.error("Skip recovering container " + container
} + " for unknown application.");
LOG.info("Recovering container " + container);
SchedulerApplicationAttempt schedulerAttempt =
schedulerApp.getCurrentAppAttempt();
if (!rmApp.getApplicationSubmissionContext()
.getKeepContainersAcrossApplicationAttempts()) {
// Do not recover containers for stopped attempt or previous attempt.
if (schedulerAttempt.isStopped()
|| !schedulerAttempt.getApplicationAttemptId().equals(
container.getContainerId().getApplicationAttemptId())) {
LOG.info("Skip recovering container " + container
+ " for already stopped attempt.");
killOrphanContainerOnNode(nm, container); killOrphanContainerOnNode(nm, container);
continue; continue;
} }
}
// create container SchedulerApplication<T> schedulerApp = applications.get(appId);
RMContainer rmContainer = recoverAndCreateContainer(container, nm); if (schedulerApp == null) {
LOG.info("Skip recovering container " + container
// recover RMContainer + " for unknown SchedulerApplication. "
rmContainer.handle(new RMContainerRecoverEvent(container.getContainerId(), + "Application current state is " + rmApp.getState());
container)); killOrphanContainerOnNode(nm, container);
continue;
// recover scheduler node
SchedulerNode schedulerNode = nodeTracker.getNode(nm.getNodeID());
schedulerNode.recoverContainer(rmContainer);
// recover queue: update headroom etc.
Queue queue = schedulerAttempt.getQueue();
queue.recoverContainer(
getClusterResource(), schedulerAttempt, rmContainer);
// recover scheduler attempt
schedulerAttempt.recoverContainer(schedulerNode, rmContainer);
// set master container for the current running AMContainer for this
// attempt.
RMAppAttempt appAttempt = rmApp.getCurrentAppAttempt();
if (appAttempt != null) {
Container masterContainer = appAttempt.getMasterContainer();
// Mark current running AMContainer's RMContainer based on the master
// container ID stored in AppAttempt.
if (masterContainer != null
&& masterContainer.getId().equals(rmContainer.getContainerId())) {
((RMContainerImpl)rmContainer).setAMContainer(true);
} }
}
synchronized (schedulerAttempt) { LOG.info("Recovering container " + container);
Set<ContainerId> releases = schedulerAttempt.getPendingRelease(); SchedulerApplicationAttempt schedulerAttempt =
if (releases.contains(container.getContainerId())) { schedulerApp.getCurrentAppAttempt();
if (!rmApp.getApplicationSubmissionContext()
.getKeepContainersAcrossApplicationAttempts()) {
// Do not recover containers for stopped attempt or previous attempt.
if (schedulerAttempt.isStopped() || !schedulerAttempt
.getApplicationAttemptId().equals(
container.getContainerId().getApplicationAttemptId())) {
LOG.info("Skip recovering container " + container
+ " for already stopped attempt.");
killOrphanContainerOnNode(nm, container);
continue;
}
}
// create container
RMContainer rmContainer = recoverAndCreateContainer(container, nm);
// recover RMContainer
rmContainer.handle(
new RMContainerRecoverEvent(container.getContainerId(), container));
// recover scheduler node
SchedulerNode schedulerNode = nodeTracker.getNode(nm.getNodeID());
schedulerNode.recoverContainer(rmContainer);
// recover queue: update headroom etc.
Queue queue = schedulerAttempt.getQueue();
queue.recoverContainer(getClusterResource(), schedulerAttempt,
rmContainer);
// recover scheduler attempt
schedulerAttempt.recoverContainer(schedulerNode, rmContainer);
// set master container for the current running AMContainer for this
// attempt.
RMAppAttempt appAttempt = rmApp.getCurrentAppAttempt();
if (appAttempt != null) {
Container masterContainer = appAttempt.getMasterContainer();
// Mark current running AMContainer's RMContainer based on the master
// container ID stored in AppAttempt.
if (masterContainer != null && masterContainer.getId().equals(
rmContainer.getContainerId())) {
((RMContainerImpl) rmContainer).setAMContainer(true);
}
}
if (schedulerAttempt.getPendingRelease().remove(
container.getContainerId())) {
// release the container // release the container
rmContainer.handle(new RMContainerFinishedEvent(container rmContainer.handle(
.getContainerId(), SchedulerUtils.createAbnormalContainerStatus( new RMContainerFinishedEvent(container.getContainerId(),
container.getContainerId(), SchedulerUtils.RELEASED_CONTAINER), SchedulerUtils
RMContainerEventType.RELEASED)); .createAbnormalContainerStatus(container.getContainerId(),
releases.remove(container.getContainerId()); SchedulerUtils.RELEASED_CONTAINER),
RMContainerEventType.RELEASED));
LOG.info(container.getContainerId() + " is released by application."); LOG.info(container.getContainerId() + " is released by application.");
} }
} }
} finally {
writeLock.unlock();
} }
} }
@ -492,17 +519,15 @@ public abstract class AbstractYarnScheduler
for (SchedulerApplication<T> app : applications.values()) { for (SchedulerApplication<T> app : applications.values()) {
T attempt = app.getCurrentAppAttempt(); T attempt = app.getCurrentAppAttempt();
if (attempt != null) { if (attempt != null) {
synchronized (attempt) { for (ContainerId containerId : attempt.getPendingRelease()) {
for (ContainerId containerId : attempt.getPendingRelease()) { RMAuditLogger.logFailure(app.getUser(),
RMAuditLogger.logFailure(app.getUser(), AuditConstants.RELEASE_CONTAINER,
AuditConstants.RELEASE_CONTAINER, "Unauthorized access or invalid container", "Scheduler",
"Unauthorized access or invalid container", "Scheduler", "Trying to release container not owned by app "
"Trying to release container not owned by app " + "or with invalid id.", attempt.getApplicationId(),
+ "or with invalid id.", attempt.getApplicationId(), containerId, null);
containerId, null);
}
attempt.getPendingRelease().clear();
} }
attempt.getPendingRelease().clear();
} }
} }
} }
@ -558,9 +583,7 @@ public abstract class AbstractYarnScheduler
< nmExpireInterval) { < nmExpireInterval) {
LOG.info(containerId + " doesn't exist. Add the container" LOG.info(containerId + " doesn't exist. Add the container"
+ " to the release request cache as it maybe on recovery."); + " to the release request cache as it maybe on recovery.");
synchronized (attempt) { attempt.getPendingRelease().add(containerId);
attempt.getPendingRelease().add(containerId);
}
} else { } else {
RMAuditLogger.logFailure(attempt.getUser(), RMAuditLogger.logFailure(attempt.getUser(),
AuditConstants.RELEASE_CONTAINER, AuditConstants.RELEASE_CONTAINER,
@ -603,81 +626,92 @@ public abstract class AbstractYarnScheduler
} }
@Override @Override
public synchronized void moveAllApps(String sourceQueue, String destQueue) public void moveAllApps(String sourceQueue, String destQueue)
throws YarnException { throws YarnException {
// check if destination queue is a valid leaf queue
try { try {
getQueueInfo(destQueue, false, false); writeLock.lock();
} catch (IOException e) { // check if destination queue is a valid leaf queue
LOG.warn(e); try {
throw new YarnException(e); getQueueInfo(destQueue, false, false);
} } catch (IOException e) {
// check if source queue is a valid LOG.warn(e);
List<ApplicationAttemptId> apps = getAppsInQueue(sourceQueue); throw new YarnException(e);
if (apps == null) { }
String errMsg = "The specified Queue: " + sourceQueue + " doesn't exist"; // check if source queue is a valid
LOG.warn(errMsg); List<ApplicationAttemptId> apps = getAppsInQueue(sourceQueue);
throw new YarnException(errMsg); if (apps == null) {
} String errMsg =
// generate move events for each pending/running app "The specified Queue: " + sourceQueue + " doesn't exist";
for (ApplicationAttemptId app : apps) { LOG.warn(errMsg);
SettableFuture<Object> future = SettableFuture.create(); throw new YarnException(errMsg);
this.rmContext }
.getDispatcher() // generate move events for each pending/running app
.getEventHandler() for (ApplicationAttemptId app : apps) {
.handle(new RMAppMoveEvent(app.getApplicationId(), destQueue, future)); SettableFuture<Object> future = SettableFuture.create();
this.rmContext.getDispatcher().getEventHandler().handle(
new RMAppMoveEvent(app.getApplicationId(), destQueue, future));
}
} finally {
writeLock.unlock();
} }
} }
@Override @Override
public synchronized void killAllAppsInQueue(String queueName) public void killAllAppsInQueue(String queueName)
throws YarnException { throws YarnException {
// check if queue is a valid try {
List<ApplicationAttemptId> apps = getAppsInQueue(queueName); writeLock.lock();
if (apps == null) { // check if queue is a valid
String errMsg = "The specified Queue: " + queueName + " doesn't exist"; List<ApplicationAttemptId> apps = getAppsInQueue(queueName);
LOG.warn(errMsg); if (apps == null) {
throw new YarnException(errMsg); String errMsg = "The specified Queue: " + queueName + " doesn't exist";
} LOG.warn(errMsg);
// generate kill events for each pending/running app throw new YarnException(errMsg);
for (ApplicationAttemptId app : apps) { }
this.rmContext // generate kill events for each pending/running app
.getDispatcher() for (ApplicationAttemptId app : apps) {
.getEventHandler() this.rmContext.getDispatcher().getEventHandler().handle(
.handle(new RMAppEvent(app.getApplicationId(), RMAppEventType.KILL, new RMAppEvent(app.getApplicationId(), RMAppEventType.KILL,
"Application killed due to expiry of reservation queue " + "Application killed due to expiry of reservation queue "
queueName + ".")); + queueName + "."));
}
} finally {
writeLock.unlock();
} }
} }
/** /**
* Process resource update on a node. * Process resource update on a node.
*/ */
public synchronized void updateNodeResource(RMNode nm, public void updateNodeResource(RMNode nm,
ResourceOption resourceOption) { ResourceOption resourceOption) {
SchedulerNode node = getSchedulerNode(nm.getNodeID()); try {
Resource newResource = resourceOption.getResource(); writeLock.lock();
Resource oldResource = node.getTotalResource(); SchedulerNode node = getSchedulerNode(nm.getNodeID());
if(!oldResource.equals(newResource)) { Resource newResource = resourceOption.getResource();
// Notify NodeLabelsManager about this change Resource oldResource = node.getTotalResource();
rmContext.getNodeLabelManager().updateNodeResource(nm.getNodeID(), if (!oldResource.equals(newResource)) {
newResource); // Notify NodeLabelsManager about this change
rmContext.getNodeLabelManager().updateNodeResource(nm.getNodeID(),
// Log resource change newResource);
LOG.info("Update resource on node: " + node.getNodeName()
+ " from: " + oldResource + ", to: "
+ newResource);
nodeTracker.removeNode(nm.getNodeID()); // Log resource change
LOG.info("Update resource on node: " + node.getNodeName() + " from: "
+ oldResource + ", to: " + newResource);
// update resource to node nodeTracker.removeNode(nm.getNodeID());
node.updateTotalResource(newResource);
nodeTracker.addNode((N) node); // update resource to node
} else { node.updateTotalResource(newResource);
// Log resource change
LOG.warn("Update resource on node: " + node.getNodeName() nodeTracker.addNode((N) node);
+ " with the same resource: " + newResource); } else{
// Log resource change
LOG.warn("Update resource on node: " + node.getNodeName()
+ " with the same resource: " + newResource);
}
} finally {
writeLock.unlock();
} }
} }
@ -735,7 +769,7 @@ public abstract class AbstractYarnScheduler
} }
@Override @Override
public synchronized void setClusterMaxPriority(Configuration conf) public void setClusterMaxPriority(Configuration conf)
throws YarnException { throws YarnException {
try { try {
maxClusterLevelAppPriority = getMaxPriorityFromConf(conf); maxClusterLevelAppPriority = getMaxPriorityFromConf(conf);

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
@ -186,7 +187,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
new AppSchedulingInfo(applicationAttemptId, user, queue, new AppSchedulingInfo(applicationAttemptId, user, queue,
activeUsersManager, rmContext.getEpoch(), attemptResourceUsage); activeUsersManager, rmContext.getEpoch(), attemptResourceUsage);
this.queue = queue; this.queue = queue;
this.pendingRelease = new HashSet<ContainerId>(); this.pendingRelease = Collections.newSetFromMap(
new ConcurrentHashMap<ContainerId, Boolean>());
this.attemptId = applicationAttemptId; this.attemptId = applicationAttemptId;
if (rmContext.getRMApps() != null && if (rmContext.getRMApps() != null &&
rmContext.getRMApps() rmContext.getRMApps()
@ -1191,6 +1193,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
// queue's resource usage for specific partition // queue's resource usage for specific partition
} }
public ReentrantReadWriteLock.WriteLock getWriteLock() {
return writeLock;
}
@Override @Override
public boolean isRecovering() { public boolean isRecovering() {
return isAttemptRecovering; return isAttemptRecovering;

View File

@ -2227,6 +2227,22 @@ public class LeafQueue extends AbstractCSQueue {
} }
} }
public void updateApplicationPriority(SchedulerApplication<FiCaSchedulerApp> app,
Priority newAppPriority) {
try {
writeLock.lock();
FiCaSchedulerApp attempt = app.getCurrentAppAttempt();
getOrderingPolicy().removeSchedulableEntity(attempt);
// Update new priority in SchedulerApplication
attempt.setPriority(newAppPriority);
getOrderingPolicy().addSchedulableEntity(attempt);
} finally {
writeLock.unlock();
}
}
public OrderingPolicy<FiCaSchedulerApp> public OrderingPolicy<FiCaSchedulerApp>
getPendingAppsOrderingPolicy() { getPendingAppsOrderingPolicy() {
return pendingOrderingPolicy; return pendingOrderingPolicy;

View File

@ -23,6 +23,7 @@ import java.util.HashSet;
import java.util.List; import java.util.List;
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.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -666,6 +667,9 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
} finally { } finally {
writeLock.unlock(); writeLock.unlock();
} }
}
public ReentrantReadWriteLock.WriteLock getWriteLock() {
return this.writeLock;
} }
} }