|
@@ -19,11 +19,9 @@
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.util.Iterator;
|
|
|
|
|
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
|
|
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
@@ -33,189 +31,48 @@ import org.slf4j.LoggerFactory;
|
|
* reservations, but functionality wise is a sub-class of ParentQueue
|
|
* reservations, but functionality wise is a sub-class of ParentQueue
|
|
*
|
|
*
|
|
*/
|
|
*/
|
|
-public class PlanQueue extends ParentQueue {
|
|
|
|
|
|
+public class PlanQueue extends AbstractManagedParentQueue {
|
|
|
|
|
|
private static final Logger LOG = LoggerFactory.getLogger(PlanQueue.class);
|
|
private static final Logger LOG = LoggerFactory.getLogger(PlanQueue.class);
|
|
|
|
|
|
- private int maxAppsForReservation;
|
|
|
|
- private int maxAppsPerUserForReservation;
|
|
|
|
- private int userLimit;
|
|
|
|
- private float userLimitFactor;
|
|
|
|
- protected CapacitySchedulerContext schedulerContext;
|
|
|
|
private boolean showReservationsAsQueues;
|
|
private boolean showReservationsAsQueues;
|
|
|
|
|
|
public PlanQueue(CapacitySchedulerContext cs, String queueName,
|
|
public PlanQueue(CapacitySchedulerContext cs, String queueName,
|
|
CSQueue parent, CSQueue old) throws IOException {
|
|
CSQueue parent, CSQueue old) throws IOException {
|
|
super(cs, queueName, parent, old);
|
|
super(cs, queueName, parent, old);
|
|
-
|
|
|
|
- this.schedulerContext = cs;
|
|
|
|
- // Set the reservation queue attributes for the Plan
|
|
|
|
- CapacitySchedulerConfiguration conf = cs.getConfiguration();
|
|
|
|
- String queuePath = super.getQueuePath();
|
|
|
|
- int maxAppsForReservation = conf.getMaximumApplicationsPerQueue(queuePath);
|
|
|
|
- showReservationsAsQueues = conf.getShowReservationAsQueues(queuePath);
|
|
|
|
- if (maxAppsForReservation < 0) {
|
|
|
|
- maxAppsForReservation =
|
|
|
|
- (int) (CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS * super
|
|
|
|
- .getAbsoluteCapacity());
|
|
|
|
- }
|
|
|
|
- int userLimit = conf.getUserLimit(queuePath);
|
|
|
|
- float userLimitFactor = conf.getUserLimitFactor(queuePath);
|
|
|
|
- int maxAppsPerUserForReservation =
|
|
|
|
- (int) (maxAppsForReservation * (userLimit / 100.0f) * userLimitFactor);
|
|
|
|
- updateQuotas(userLimit, userLimitFactor, maxAppsForReservation,
|
|
|
|
- maxAppsPerUserForReservation);
|
|
|
|
-
|
|
|
|
- StringBuffer queueInfo = new StringBuffer();
|
|
|
|
- queueInfo.append("Created Plan Queue: ").append(queueName)
|
|
|
|
- .append("\nwith capacity: [").append(super.getCapacity())
|
|
|
|
- .append("]\nwith max capacity: [").append(super.getMaximumCapacity())
|
|
|
|
- .append("\nwith max reservation apps: [").append(maxAppsForReservation)
|
|
|
|
- .append("]\nwith max reservation apps per user: [")
|
|
|
|
- .append(maxAppsPerUserForReservation).append("]\nwith user limit: [")
|
|
|
|
- .append(userLimit).append("]\nwith user limit factor: [")
|
|
|
|
- .append(userLimitFactor).append("].");
|
|
|
|
- LOG.info(queueInfo.toString());
|
|
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public void reinitialize(CSQueue newlyParsedQueue,
|
|
|
|
- Resource clusterResource) throws IOException {
|
|
|
|
- try {
|
|
|
|
- writeLock.lock();
|
|
|
|
- // Sanity check
|
|
|
|
- if (!(newlyParsedQueue instanceof PlanQueue) || !newlyParsedQueue
|
|
|
|
- .getQueuePath().equals(getQueuePath())) {
|
|
|
|
- throw new IOException(
|
|
|
|
- "Trying to reinitialize " + getQueuePath() + " from "
|
|
|
|
- + newlyParsedQueue.getQueuePath());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
|
|
|
|
-
|
|
|
|
- if (newlyParsedParentQueue.getChildQueues().size() != 1) {
|
|
|
|
- throw new IOException(
|
|
|
|
- "Reservable Queue should not have sub-queues in the"
|
|
|
|
- + "configuration expect the default reservation queue");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Set new configs
|
|
|
|
- setupQueueConfigs(clusterResource);
|
|
|
|
-
|
|
|
|
- updateQuotas(newlyParsedParentQueue.userLimit,
|
|
|
|
- newlyParsedParentQueue.userLimitFactor,
|
|
|
|
- newlyParsedParentQueue.maxAppsForReservation,
|
|
|
|
- newlyParsedParentQueue.maxAppsPerUserForReservation);
|
|
|
|
-
|
|
|
|
- // run reinitialize on each existing queue, to trigger absolute cap
|
|
|
|
- // recomputations
|
|
|
|
- for (CSQueue res : this.getChildQueues()) {
|
|
|
|
- res.reinitialize(res, clusterResource);
|
|
|
|
- }
|
|
|
|
- showReservationsAsQueues =
|
|
|
|
- newlyParsedParentQueue.showReservationsAsQueues;
|
|
|
|
- } finally {
|
|
|
|
- writeLock.unlock();
|
|
|
|
- }
|
|
|
|
|
|
+ public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
|
|
|
|
+ throws IOException {
|
|
|
|
+ validate(newlyParsedQueue);
|
|
|
|
+ super.reinitialize(newlyParsedQueue, clusterResource);
|
|
}
|
|
}
|
|
|
|
|
|
- void addChildQueue(CSQueue newQueue)
|
|
|
|
- throws SchedulerDynamicEditException {
|
|
|
|
- try {
|
|
|
|
- writeLock.lock();
|
|
|
|
- if (newQueue.getCapacity() > 0) {
|
|
|
|
- throw new SchedulerDynamicEditException(
|
|
|
|
- "Queue " + newQueue + " being added has non zero capacity.");
|
|
|
|
- }
|
|
|
|
- boolean added = this.childQueues.add(newQueue);
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("updateChildQueues (action: add queue): " + added + " "
|
|
|
|
- + getChildQueuesToPrint());
|
|
|
|
- }
|
|
|
|
- } finally {
|
|
|
|
- writeLock.unlock();
|
|
|
|
- }
|
|
|
|
|
|
+ @Override
|
|
|
|
+ protected void initializeLeafQueueConfigs() {
|
|
|
|
+ String queuePath = super.getQueuePath();
|
|
|
|
+ showReservationsAsQueues = csContext.getConfiguration()
|
|
|
|
+ .getShowReservationAsQueues(queuePath);
|
|
|
|
+ super.initializeLeafQueueConfigs();
|
|
}
|
|
}
|
|
|
|
|
|
- void removeChildQueue(CSQueue remQueue)
|
|
|
|
- throws SchedulerDynamicEditException {
|
|
|
|
- try {
|
|
|
|
- writeLock.lock();
|
|
|
|
- if (remQueue.getCapacity() > 0) {
|
|
|
|
- throw new SchedulerDynamicEditException(
|
|
|
|
- "Queue " + remQueue + " being removed has non zero capacity.");
|
|
|
|
- }
|
|
|
|
- Iterator<CSQueue> qiter = childQueues.iterator();
|
|
|
|
- while (qiter.hasNext()) {
|
|
|
|
- CSQueue cs = qiter.next();
|
|
|
|
- if (cs.equals(remQueue)) {
|
|
|
|
- qiter.remove();
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Removed child queue: {}", cs.getQueueName());
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- } finally {
|
|
|
|
- writeLock.unlock();
|
|
|
|
|
|
+ private void validate(final CSQueue newlyParsedQueue) throws IOException {
|
|
|
|
+ // Sanity check
|
|
|
|
+ if (!(newlyParsedQueue instanceof PlanQueue) || !newlyParsedQueue
|
|
|
|
+ .getQueuePath().equals(getQueuePath())) {
|
|
|
|
+ throw new IOException(
|
|
|
|
+ "Trying to reinitialize " + getQueuePath() + " from "
|
|
|
|
+ + newlyParsedQueue.getQueuePath());
|
|
}
|
|
}
|
|
- }
|
|
|
|
|
|
|
|
- protected float sumOfChildCapacities() {
|
|
|
|
- try {
|
|
|
|
- writeLock.lock();
|
|
|
|
- float ret = 0;
|
|
|
|
- for (CSQueue l : childQueues) {
|
|
|
|
- ret += l.getCapacity();
|
|
|
|
- }
|
|
|
|
- return ret;
|
|
|
|
- } finally {
|
|
|
|
- writeLock.unlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
|
|
|
|
|
|
- private void updateQuotas(int userLimit, float userLimitFactor,
|
|
|
|
- int maxAppsForReservation, int maxAppsPerUserForReservation) {
|
|
|
|
- this.userLimit = userLimit;
|
|
|
|
- this.userLimitFactor = userLimitFactor;
|
|
|
|
- this.maxAppsForReservation = maxAppsForReservation;
|
|
|
|
- this.maxAppsPerUserForReservation = maxAppsPerUserForReservation;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Number of maximum applications for each of the reservations in this Plan.
|
|
|
|
- *
|
|
|
|
- * @return maxAppsForreservation
|
|
|
|
- */
|
|
|
|
- public int getMaxApplicationsForReservations() {
|
|
|
|
- return maxAppsForReservation;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Number of maximum applications per user for each of the reservations in
|
|
|
|
- * this Plan.
|
|
|
|
- *
|
|
|
|
- * @return maxAppsPerUserForreservation
|
|
|
|
- */
|
|
|
|
- public int getMaxApplicationsPerUserForReservation() {
|
|
|
|
- return maxAppsPerUserForReservation;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * User limit value for each of the reservations in this Plan.
|
|
|
|
- *
|
|
|
|
- * @return userLimit
|
|
|
|
- */
|
|
|
|
- public int getUserLimitForReservation() {
|
|
|
|
- return userLimit;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * User limit factor value for each of the reservations in this Plan.
|
|
|
|
- *
|
|
|
|
- * @return userLimitFactor
|
|
|
|
- */
|
|
|
|
- public float getUserLimitFactor() {
|
|
|
|
- return userLimitFactor;
|
|
|
|
|
|
+ if (newlyParsedParentQueue.getChildQueues().size() != 1) {
|
|
|
|
+ throw new IOException(
|
|
|
|
+ "Reservable Queue should not have sub-queues in the"
|
|
|
|
+ + "configuration expect the default reservation queue");
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|