|
@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.api.records.QueueState;
|
|
import org.apache.hadoop.yarn.api.records.QueueStatistics;
|
|
import org.apache.hadoop.yarn.api.records.QueueStatistics;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
import org.apache.hadoop.yarn.security.AccessRequest;
|
|
import org.apache.hadoop.yarn.security.AccessRequest;
|
|
@@ -77,7 +78,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
final Resource minimumAllocation;
|
|
final Resource minimumAllocation;
|
|
volatile Resource maximumAllocation;
|
|
volatile Resource maximumAllocation;
|
|
- volatile QueueState state;
|
|
|
|
|
|
+ private volatile QueueState state = null;
|
|
final CSQueueMetrics metrics;
|
|
final CSQueueMetrics metrics;
|
|
protected final PrivilegedEntity queueEntity;
|
|
protected final PrivilegedEntity queueEntity;
|
|
|
|
|
|
@@ -292,9 +293,15 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
csContext.getConfiguration().getMaximumAllocationPerQueue(
|
|
csContext.getConfiguration().getMaximumAllocationPerQueue(
|
|
getQueuePath());
|
|
getQueuePath());
|
|
|
|
|
|
- authorizer = YarnAuthorizationProvider.getInstance(csContext.getConf());
|
|
|
|
|
|
+ // initialized the queue state based on previous state, configured state
|
|
|
|
+ // and its parent state.
|
|
|
|
+ QueueState previous = getState();
|
|
|
|
+ QueueState configuredState = csContext.getConfiguration()
|
|
|
|
+ .getConfiguredState(getQueuePath());
|
|
|
|
+ QueueState parentState = (parent == null) ? null : parent.getState();
|
|
|
|
+ initializeQueueState(previous, configuredState, parentState);
|
|
|
|
|
|
- initializeQueueState();
|
|
|
|
|
|
+ authorizer = YarnAuthorizationProvider.getInstance(csContext.getConf());
|
|
|
|
|
|
this.acls = csContext.getConfiguration().getAcls(getQueuePath());
|
|
this.acls = csContext.getConfiguration().getAcls(getQueuePath());
|
|
|
|
|
|
@@ -334,26 +341,53 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private void initializeQueueState() {
|
|
|
|
- // inherit from parent if state not set, only do this when we are not root
|
|
|
|
- if (parent != null) {
|
|
|
|
- QueueState configuredState = csContext.getConfiguration()
|
|
|
|
- .getConfiguredState(getQueuePath());
|
|
|
|
- QueueState parentState = parent.getState();
|
|
|
|
- if (configuredState == null) {
|
|
|
|
- this.state = parentState;
|
|
|
|
- } else if (configuredState == QueueState.RUNNING
|
|
|
|
- && parentState == QueueState.STOPPED) {
|
|
|
|
- throw new IllegalArgumentException(
|
|
|
|
- "The parent queue:" + parent.getQueueName() + " state is STOPPED, "
|
|
|
|
- + "child queue:" + queueName + " state cannot be RUNNING.");
|
|
|
|
|
|
+ private void initializeQueueState(QueueState previousState,
|
|
|
|
+ QueueState configuredState, QueueState parentState) {
|
|
|
|
+ // verify that we can not any value for State other than RUNNING/STOPPED
|
|
|
|
+ if (configuredState != null && configuredState != QueueState.RUNNING
|
|
|
|
+ && configuredState != QueueState.STOPPED) {
|
|
|
|
+ throw new IllegalArgumentException("Invalid queue state configuration."
|
|
|
|
+ + " We can only use RUNNING or STOPPED.");
|
|
|
|
+ }
|
|
|
|
+ // If we did not set state in configuration, use Running as default state
|
|
|
|
+ QueueState defaultState = QueueState.RUNNING;
|
|
|
|
+
|
|
|
|
+ if (previousState == null) {
|
|
|
|
+ // If current state of the queue is null, we would inherit the state
|
|
|
|
+ // from its parent. If this queue does not has parent, such as root queue,
|
|
|
|
+ // we would use the configured state.
|
|
|
|
+ if (parentState == null) {
|
|
|
|
+ updateQueueState((configuredState == null) ? defaultState
|
|
|
|
+ : configuredState);
|
|
} else {
|
|
} else {
|
|
- this.state = configuredState;
|
|
|
|
|
|
+ if (configuredState == null) {
|
|
|
|
+ updateQueueState((parentState == QueueState.DRAINING) ?
|
|
|
|
+ QueueState.STOPPED : parentState);
|
|
|
|
+ } else if (configuredState == QueueState.RUNNING
|
|
|
|
+ && parentState != QueueState.RUNNING) {
|
|
|
|
+ throw new IllegalArgumentException(
|
|
|
|
+ "The parent queue:" + parent.getQueueName()
|
|
|
|
+ + " state is STOPPED, child queue:" + queueName
|
|
|
|
+ + " state cannot be RUNNING.");
|
|
|
|
+ } else {
|
|
|
|
+ updateQueueState(configuredState);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
- // if this is the root queue, get the state from the configuration.
|
|
|
|
- // if the state is not set, use RUNNING as default state.
|
|
|
|
- this.state = csContext.getConfiguration().getState(getQueuePath());
|
|
|
|
|
|
+ // when we get a refreshQueue request from AdminService,
|
|
|
|
+ if (previousState == QueueState.RUNNING) {
|
|
|
|
+ if (configuredState == QueueState.STOPPED) {
|
|
|
|
+ stopQueue();
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ if (configuredState == QueueState.RUNNING) {
|
|
|
|
+ try {
|
|
|
|
+ activeQueue();
|
|
|
|
+ } catch (YarnException ex) {
|
|
|
|
+ throw new IllegalArgumentException(ex.getMessage());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -367,7 +401,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
queueInfo.setAccessibleNodeLabels(accessibleLabels);
|
|
queueInfo.setAccessibleNodeLabels(accessibleLabels);
|
|
queueInfo.setCapacity(queueCapacities.getCapacity());
|
|
queueInfo.setCapacity(queueCapacities.getCapacity());
|
|
queueInfo.setMaximumCapacity(queueCapacities.getMaximumCapacity());
|
|
queueInfo.setMaximumCapacity(queueCapacities.getMaximumCapacity());
|
|
- queueInfo.setQueueState(state);
|
|
|
|
|
|
+ queueInfo.setQueueState(getState());
|
|
queueInfo.setDefaultNodeLabelExpression(defaultLabelExpression);
|
|
queueInfo.setDefaultNodeLabelExpression(defaultLabelExpression);
|
|
queueInfo.setCurrentCapacity(getUsedCapacity());
|
|
queueInfo.setCurrentCapacity(getUsedCapacity());
|
|
queueInfo.setQueueStatistics(getQueueStatistics());
|
|
queueInfo.setQueueStatistics(getQueueStatistics());
|
|
@@ -846,4 +880,47 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
String userName, String queue) throws AccessControlException {
|
|
String userName, String queue) throws AccessControlException {
|
|
// Dummy implementation
|
|
// Dummy implementation
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void updateQueueState(QueueState queueState) {
|
|
|
|
+ this.state = queueState;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void activeQueue() throws YarnException {
|
|
|
|
+ try {
|
|
|
|
+ this.writeLock.lock();
|
|
|
|
+ if (getState() == QueueState.RUNNING) {
|
|
|
|
+ LOG.info("The specified queue:" + queueName
|
|
|
|
+ + " is already in the RUNNING state.");
|
|
|
|
+ } else if (getState() == QueueState.DRAINING) {
|
|
|
|
+ throw new YarnException(
|
|
|
|
+ "The queue:" + queueName + " is in the Stopping process. "
|
|
|
|
+ + "Please wait for the queue getting fully STOPPED.");
|
|
|
|
+ } else {
|
|
|
|
+ CSQueue parent = getParent();
|
|
|
|
+ if (parent == null || parent.getState() == QueueState.RUNNING) {
|
|
|
|
+ updateQueueState(QueueState.RUNNING);
|
|
|
|
+ } else {
|
|
|
|
+ throw new YarnException("The parent Queue:" + parent.getQueueName()
|
|
|
|
+ + " is not running. Please activate the parent queue first");
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ this.writeLock.unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected void appFinished() {
|
|
|
|
+ try {
|
|
|
|
+ this.writeLock.lock();
|
|
|
|
+ if (getState() == QueueState.DRAINING) {
|
|
|
|
+ if (getNumApplications() == 0) {
|
|
|
|
+ updateQueueState(QueueState.STOPPED);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ this.writeLock.unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|