|
@@ -322,8 +322,8 @@ public class CapacityScheduler extends
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
void initScheduler(Configuration configuration) throws
|
|
void initScheduler(Configuration configuration) throws
|
|
IOException {
|
|
IOException {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
String confProviderStr = configuration.get(
|
|
String confProviderStr = configuration.get(
|
|
YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
|
|
YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
|
|
YarnConfiguration.DEFAULT_CONFIGURATION_STORE);
|
|
YarnConfiguration.DEFAULT_CONFIGURATION_STORE);
|
|
@@ -421,8 +421,8 @@ public class CapacityScheduler extends
|
|
}
|
|
}
|
|
|
|
|
|
private void startSchedulerThreads() {
|
|
private void startSchedulerThreads() {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
activitiesManager.start();
|
|
activitiesManager.start();
|
|
if (scheduleAsynchronously) {
|
|
if (scheduleAsynchronously) {
|
|
Preconditions.checkNotNull(asyncSchedulerThreads,
|
|
Preconditions.checkNotNull(asyncSchedulerThreads,
|
|
@@ -455,8 +455,8 @@ public class CapacityScheduler extends
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void serviceStop() throws Exception {
|
|
public void serviceStop() throws Exception {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
this.activitiesManager.stop();
|
|
this.activitiesManager.stop();
|
|
if (scheduleAsynchronously && asyncSchedulerThreads != null) {
|
|
if (scheduleAsynchronously && asyncSchedulerThreads != null) {
|
|
for (Thread t : asyncSchedulerThreads) {
|
|
for (Thread t : asyncSchedulerThreads) {
|
|
@@ -479,8 +479,8 @@ public class CapacityScheduler extends
|
|
@Override
|
|
@Override
|
|
public void reinitialize(Configuration newConf, RMContext rmContext)
|
|
public void reinitialize(Configuration newConf, RMContext rmContext)
|
|
throws IOException {
|
|
throws IOException {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
Configuration configuration = new Configuration(newConf);
|
|
Configuration configuration = new Configuration(newConf);
|
|
CapacitySchedulerConfiguration oldConf = this.conf;
|
|
CapacitySchedulerConfiguration oldConf = this.conf;
|
|
this.conf = csConfProvider.loadConfiguration(configuration);
|
|
this.conf = csConfProvider.loadConfiguration(configuration);
|
|
@@ -656,9 +656,8 @@ public class CapacityScheduler extends
|
|
try {
|
|
try {
|
|
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request =
|
|
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request =
|
|
backlogs.take();
|
|
backlogs.take();
|
|
-
|
|
|
|
|
|
+ cs.writeLock.lock();
|
|
try {
|
|
try {
|
|
- cs.writeLock.lock();
|
|
|
|
cs.tryCommit(cs.getClusterResource(), request, true);
|
|
cs.tryCommit(cs.getClusterResource(), request, true);
|
|
} finally {
|
|
} finally {
|
|
cs.writeLock.unlock();
|
|
cs.writeLock.unlock();
|
|
@@ -684,8 +683,8 @@ public class CapacityScheduler extends
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
public PlacementRule getUserGroupMappingPlacementRule() throws IOException {
|
|
public PlacementRule getUserGroupMappingPlacementRule() throws IOException {
|
|
|
|
+ readLock.lock();
|
|
try {
|
|
try {
|
|
- readLock.lock();
|
|
|
|
UserGroupMappingPlacementRule ugRule = new UserGroupMappingPlacementRule();
|
|
UserGroupMappingPlacementRule ugRule = new UserGroupMappingPlacementRule();
|
|
ugRule.initialize(this);
|
|
ugRule.initialize(this);
|
|
return ugRule;
|
|
return ugRule;
|
|
@@ -695,8 +694,8 @@ public class CapacityScheduler extends
|
|
}
|
|
}
|
|
|
|
|
|
public PlacementRule getAppNameMappingPlacementRule() throws IOException {
|
|
public PlacementRule getAppNameMappingPlacementRule() throws IOException {
|
|
|
|
+ readLock.lock();
|
|
try {
|
|
try {
|
|
- readLock.lock();
|
|
|
|
AppNameMappingPlacementRule anRule = new AppNameMappingPlacementRule();
|
|
AppNameMappingPlacementRule anRule = new AppNameMappingPlacementRule();
|
|
anRule.initialize(this);
|
|
anRule.initialize(this);
|
|
return anRule;
|
|
return anRule;
|
|
@@ -796,8 +795,8 @@ public class CapacityScheduler extends
|
|
private void addApplicationOnRecovery(ApplicationId applicationId,
|
|
private void addApplicationOnRecovery(ApplicationId applicationId,
|
|
String queueName, String user,
|
|
String queueName, String user,
|
|
Priority priority, ApplicationPlacementContext placementContext) {
|
|
Priority priority, ApplicationPlacementContext placementContext) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
//check if the queue needs to be auto-created during recovery
|
|
//check if the queue needs to be auto-created during recovery
|
|
CSQueue queue = getOrCreateQueueFromPlacementContext(applicationId, user,
|
|
CSQueue queue = getOrCreateQueueFromPlacementContext(applicationId, user,
|
|
queueName, placementContext, true);
|
|
queueName, placementContext, true);
|
|
@@ -920,8 +919,8 @@ public class CapacityScheduler extends
|
|
private void addApplication(ApplicationId applicationId, String queueName,
|
|
private void addApplication(ApplicationId applicationId, String queueName,
|
|
String user, Priority priority,
|
|
String user, Priority priority,
|
|
ApplicationPlacementContext placementContext) {
|
|
ApplicationPlacementContext placementContext) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
if (isSystemAppsLimitReached()) {
|
|
if (isSystemAppsLimitReached()) {
|
|
String message = "Maximum system application limit reached,"
|
|
String message = "Maximum system application limit reached,"
|
|
+ "cannot accept submission of application: " + applicationId;
|
|
+ "cannot accept submission of application: " + applicationId;
|
|
@@ -1019,8 +1018,8 @@ public class CapacityScheduler extends
|
|
ApplicationAttemptId applicationAttemptId,
|
|
ApplicationAttemptId applicationAttemptId,
|
|
boolean transferStateFromPreviousAttempt,
|
|
boolean transferStateFromPreviousAttempt,
|
|
boolean isAttemptRecovering) {
|
|
boolean isAttemptRecovering) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
SchedulerApplication<FiCaSchedulerApp> application = applications.get(
|
|
SchedulerApplication<FiCaSchedulerApp> application = applications.get(
|
|
applicationAttemptId.getApplicationId());
|
|
applicationAttemptId.getApplicationId());
|
|
if (application == null) {
|
|
if (application == null) {
|
|
@@ -1072,8 +1071,8 @@ public class CapacityScheduler extends
|
|
|
|
|
|
private void doneApplication(ApplicationId applicationId,
|
|
private void doneApplication(ApplicationId applicationId,
|
|
RMAppState finalState) {
|
|
RMAppState finalState) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
SchedulerApplication<FiCaSchedulerApp> application = applications.get(
|
|
SchedulerApplication<FiCaSchedulerApp> application = applications.get(
|
|
applicationId);
|
|
applicationId);
|
|
if (application == null) {
|
|
if (application == null) {
|
|
@@ -1099,8 +1098,8 @@ public class CapacityScheduler extends
|
|
private void doneApplicationAttempt(
|
|
private void doneApplicationAttempt(
|
|
ApplicationAttemptId applicationAttemptId,
|
|
ApplicationAttemptId applicationAttemptId,
|
|
RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
|
|
RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
LOG.info("Application Attempt " + applicationAttemptId + " is done."
|
|
LOG.info("Application Attempt " + applicationAttemptId + " is done."
|
|
+ " finalState=" + rmAppAttemptFinalState);
|
|
+ " finalState=" + rmAppAttemptFinalState);
|
|
|
|
|
|
@@ -1214,8 +1213,8 @@ public class CapacityScheduler extends
|
|
|
|
|
|
// make sure we aren't stopping/removing the application
|
|
// make sure we aren't stopping/removing the application
|
|
// when the allocate comes in
|
|
// when the allocate comes in
|
|
|
|
+ application.getWriteLock().lock();
|
|
try {
|
|
try {
|
|
- application.getWriteLock().lock();
|
|
|
|
if (application.isStopped()) {
|
|
if (application.isStopped()) {
|
|
return EMPTY_ALLOCATION;
|
|
return EMPTY_ALLOCATION;
|
|
}
|
|
}
|
|
@@ -1292,8 +1291,8 @@ public class CapacityScheduler extends
|
|
@Override
|
|
@Override
|
|
protected void nodeUpdate(RMNode rmNode) {
|
|
protected void nodeUpdate(RMNode rmNode) {
|
|
long begin = System.nanoTime();
|
|
long begin = System.nanoTime();
|
|
|
|
+ readLock.lock();
|
|
try {
|
|
try {
|
|
- readLock.lock();
|
|
|
|
setLastNodeUpdateTime(Time.now());
|
|
setLastNodeUpdateTime(Time.now());
|
|
super.nodeUpdate(rmNode);
|
|
super.nodeUpdate(rmNode);
|
|
} finally {
|
|
} finally {
|
|
@@ -1302,8 +1301,8 @@ public class CapacityScheduler extends
|
|
|
|
|
|
// Try to do scheduling
|
|
// Try to do scheduling
|
|
if (!scheduleAsynchronously) {
|
|
if (!scheduleAsynchronously) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
ActivitiesLogger.NODE.startNodeUpdateRecording(activitiesManager,
|
|
ActivitiesLogger.NODE.startNodeUpdateRecording(activitiesManager,
|
|
rmNode.getNodeID());
|
|
rmNode.getNodeID());
|
|
|
|
|
|
@@ -1329,8 +1328,8 @@ public class CapacityScheduler extends
|
|
*/
|
|
*/
|
|
private void updateNodeAndQueueResource(RMNode nm,
|
|
private void updateNodeAndQueueResource(RMNode nm,
|
|
ResourceOption resourceOption) {
|
|
ResourceOption resourceOption) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
updateNodeResource(nm, resourceOption);
|
|
updateNodeResource(nm, resourceOption);
|
|
Resource clusterResource = getClusterResource();
|
|
Resource clusterResource = getClusterResource();
|
|
getRootQueue().updateClusterResource(clusterResource,
|
|
getRootQueue().updateClusterResource(clusterResource,
|
|
@@ -1917,8 +1916,8 @@ public class CapacityScheduler extends
|
|
|
|
|
|
private void updateNodeAttributes(
|
|
private void updateNodeAttributes(
|
|
NodeAttributesUpdateSchedulerEvent attributeUpdateEvent) {
|
|
NodeAttributesUpdateSchedulerEvent attributeUpdateEvent) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
for (Entry<String, Set<NodeAttribute>> entry : attributeUpdateEvent
|
|
for (Entry<String, Set<NodeAttribute>> entry : attributeUpdateEvent
|
|
.getUpdatedNodeToAttributes().entrySet()) {
|
|
.getUpdatedNodeToAttributes().entrySet()) {
|
|
String hostname = entry.getKey();
|
|
String hostname = entry.getKey();
|
|
@@ -1944,8 +1943,8 @@ public class CapacityScheduler extends
|
|
*/
|
|
*/
|
|
private void updateNodeLabelsAndQueueResource(
|
|
private void updateNodeLabelsAndQueueResource(
|
|
NodeLabelsUpdateSchedulerEvent labelUpdateEvent) {
|
|
NodeLabelsUpdateSchedulerEvent labelUpdateEvent) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
Set<String> updateLabels = new HashSet<String>();
|
|
Set<String> updateLabels = new HashSet<String>();
|
|
for (Entry<NodeId, Set<String>> entry : labelUpdateEvent
|
|
for (Entry<NodeId, Set<String>> entry : labelUpdateEvent
|
|
.getUpdatedNodeToLabels().entrySet()) {
|
|
.getUpdatedNodeToLabels().entrySet()) {
|
|
@@ -1982,8 +1981,8 @@ public class CapacityScheduler extends
|
|
}
|
|
}
|
|
|
|
|
|
private void addNode(RMNode nodeManager) {
|
|
private void addNode(RMNode nodeManager) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
FiCaSchedulerNode schedulerNode = new FiCaSchedulerNode(nodeManager,
|
|
FiCaSchedulerNode schedulerNode = new FiCaSchedulerNode(nodeManager,
|
|
usePortForNodeName, nodeManager.getNodeLabels());
|
|
usePortForNodeName, nodeManager.getNodeLabels());
|
|
nodeTracker.addNode(schedulerNode);
|
|
nodeTracker.addNode(schedulerNode);
|
|
@@ -2019,8 +2018,8 @@ public class CapacityScheduler extends
|
|
}
|
|
}
|
|
|
|
|
|
private void removeNode(RMNode nodeInfo) {
|
|
private void removeNode(RMNode nodeInfo) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
// update this node to node label manager
|
|
// update this node to node label manager
|
|
if (labelManager != null) {
|
|
if (labelManager != null) {
|
|
labelManager.deactivateNode(nodeInfo.getNodeID());
|
|
labelManager.deactivateNode(nodeInfo.getNodeID());
|
|
@@ -2164,8 +2163,8 @@ public class CapacityScheduler extends
|
|
|
|
|
|
public void markContainerForKillable(
|
|
public void markContainerForKillable(
|
|
RMContainer killableContainer) {
|
|
RMContainer killableContainer) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug(SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE + ": container"
|
|
LOG.debug(SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE + ": container"
|
|
+ killableContainer.toString());
|
|
+ killableContainer.toString());
|
|
@@ -2200,8 +2199,8 @@ public class CapacityScheduler extends
|
|
|
|
|
|
private void markContainerForNonKillable(
|
|
private void markContainerForNonKillable(
|
|
RMContainer nonKillableContainer) {
|
|
RMContainer nonKillableContainer) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug(
|
|
LOG.debug(
|
|
SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE + ": container"
|
|
SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE + ": container"
|
|
@@ -2269,8 +2268,8 @@ public class CapacityScheduler extends
|
|
private String resolveReservationQueueName(String queueName,
|
|
private String resolveReservationQueueName(String queueName,
|
|
ApplicationId applicationId, ReservationId reservationID,
|
|
ApplicationId applicationId, ReservationId reservationID,
|
|
boolean isRecovering) {
|
|
boolean isRecovering) {
|
|
|
|
+ readLock.lock();
|
|
try {
|
|
try {
|
|
- readLock.lock();
|
|
|
|
CSQueue queue = getQueue(queueName);
|
|
CSQueue queue = getQueue(queueName);
|
|
// Check if the queue is a plan queue
|
|
// Check if the queue is a plan queue
|
|
if ((queue == null) || !(queue instanceof PlanQueue)) {
|
|
if ((queue == null) || !(queue instanceof PlanQueue)) {
|
|
@@ -2320,8 +2319,8 @@ public class CapacityScheduler extends
|
|
@Override
|
|
@Override
|
|
public void removeQueue(String queueName)
|
|
public void removeQueue(String queueName)
|
|
throws SchedulerDynamicEditException {
|
|
throws SchedulerDynamicEditException {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
LOG.info("Removing queue: " + queueName);
|
|
LOG.info("Removing queue: " + queueName);
|
|
CSQueue q = this.getQueue(queueName);
|
|
CSQueue q = this.getQueue(queueName);
|
|
if (!(AbstractAutoCreatedLeafQueue.class.isAssignableFrom(
|
|
if (!(AbstractAutoCreatedLeafQueue.class.isAssignableFrom(
|
|
@@ -2354,8 +2353,8 @@ public class CapacityScheduler extends
|
|
@Override
|
|
@Override
|
|
public void addQueue(Queue queue)
|
|
public void addQueue(Queue queue)
|
|
throws SchedulerDynamicEditException, IOException {
|
|
throws SchedulerDynamicEditException, IOException {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
if (queue == null) {
|
|
if (queue == null) {
|
|
throw new SchedulerDynamicEditException(
|
|
throw new SchedulerDynamicEditException(
|
|
"Queue specified is null. Should be an implementation of "
|
|
"Queue specified is null. Should be an implementation of "
|
|
@@ -2392,8 +2391,8 @@ public class CapacityScheduler extends
|
|
@Override
|
|
@Override
|
|
public void setEntitlement(String inQueue, QueueEntitlement entitlement)
|
|
public void setEntitlement(String inQueue, QueueEntitlement entitlement)
|
|
throws YarnException {
|
|
throws YarnException {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
LeafQueue queue = this.queueManager.getAndCheckLeafQueue(inQueue);
|
|
LeafQueue queue = this.queueManager.getAndCheckLeafQueue(inQueue);
|
|
AbstractManagedParentQueue parent =
|
|
AbstractManagedParentQueue parent =
|
|
(AbstractManagedParentQueue) queue.getParent();
|
|
(AbstractManagedParentQueue) queue.getParent();
|
|
@@ -2429,8 +2428,8 @@ public class CapacityScheduler extends
|
|
@Override
|
|
@Override
|
|
public String moveApplication(ApplicationId appId,
|
|
public String moveApplication(ApplicationId appId,
|
|
String targetQueueName) throws YarnException {
|
|
String targetQueueName) throws YarnException {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
SchedulerApplication<FiCaSchedulerApp> application =
|
|
SchedulerApplication<FiCaSchedulerApp> application =
|
|
applications.get(appId);
|
|
applications.get(appId);
|
|
if (application == null) {
|
|
if (application == null) {
|
|
@@ -2481,8 +2480,8 @@ public class CapacityScheduler extends
|
|
@Override
|
|
@Override
|
|
public void preValidateMoveApplication(ApplicationId appId,
|
|
public void preValidateMoveApplication(ApplicationId appId,
|
|
String newQueue) throws YarnException {
|
|
String newQueue) throws YarnException {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
SchedulerApplication<FiCaSchedulerApp> application =
|
|
SchedulerApplication<FiCaSchedulerApp> application =
|
|
applications.get(appId);
|
|
applications.get(appId);
|
|
if (application == null) {
|
|
if (application == null) {
|
|
@@ -2604,8 +2603,8 @@ public class CapacityScheduler extends
|
|
public Priority checkAndGetApplicationPriority(
|
|
public Priority checkAndGetApplicationPriority(
|
|
Priority priorityRequestedByApp, UserGroupInformation user,
|
|
Priority priorityRequestedByApp, UserGroupInformation user,
|
|
String queueName, ApplicationId applicationId) throws YarnException {
|
|
String queueName, ApplicationId applicationId) throws YarnException {
|
|
|
|
+ readLock.lock();
|
|
try {
|
|
try {
|
|
- readLock.lock();
|
|
|
|
Priority appPriority = priorityRequestedByApp;
|
|
Priority appPriority = priorityRequestedByApp;
|
|
|
|
|
|
// Verify the scenario where priority is null from submissionContext.
|
|
// Verify the scenario where priority is null from submissionContext.
|
|
@@ -2660,8 +2659,8 @@ public class CapacityScheduler extends
|
|
ApplicationId applicationId, SettableFuture<Object> future,
|
|
ApplicationId applicationId, SettableFuture<Object> future,
|
|
UserGroupInformation user)
|
|
UserGroupInformation user)
|
|
throws YarnException {
|
|
throws YarnException {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
Priority appPriority = null;
|
|
Priority appPriority = null;
|
|
SchedulerApplication<FiCaSchedulerApp> application = applications
|
|
SchedulerApplication<FiCaSchedulerApp> application = applications
|
|
.get(applicationId);
|
|
.get(applicationId);
|
|
@@ -3065,9 +3064,8 @@ public class CapacityScheduler extends
|
|
*/
|
|
*/
|
|
public boolean moveReservedContainer(RMContainer toBeMovedContainer,
|
|
public boolean moveReservedContainer(RMContainer toBeMovedContainer,
|
|
FiCaSchedulerNode targetNode) {
|
|
FiCaSchedulerNode targetNode) {
|
|
|
|
+ writeLock.lock();
|
|
try {
|
|
try {
|
|
- writeLock.lock();
|
|
|
|
-
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Trying to move container=" + toBeMovedContainer + " to node="
|
|
LOG.debug("Trying to move container=" + toBeMovedContainer + " to node="
|
|
+ targetNode.getNodeID());
|
|
+ targetNode.getNodeID());
|
|
@@ -3121,8 +3119,8 @@ public class CapacityScheduler extends
|
|
@Override
|
|
@Override
|
|
public long checkAndGetApplicationLifetime(String queueName,
|
|
public long checkAndGetApplicationLifetime(String queueName,
|
|
long lifetimeRequestedByApp) {
|
|
long lifetimeRequestedByApp) {
|
|
|
|
+ readLock.lock();
|
|
try {
|
|
try {
|
|
- readLock.lock();
|
|
|
|
CSQueue queue = getQueue(queueName);
|
|
CSQueue queue = getQueue(queueName);
|
|
if (queue == null || !(queue instanceof LeafQueue)) {
|
|
if (queue == null || !(queue instanceof LeafQueue)) {
|
|
return lifetimeRequestedByApp;
|
|
return lifetimeRequestedByApp;
|