|
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
|
|
import org.apache.hadoop.yarn.api.records.QueueACL;
|
|
import org.apache.hadoop.yarn.api.records.QueueACL;
|
|
import org.apache.hadoop.yarn.api.records.QueueInfo;
|
|
import org.apache.hadoop.yarn.api.records.QueueInfo;
|
|
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
|
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ReservationId;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.ResourceOption;
|
|
import org.apache.hadoop.yarn.api.records.ResourceOption;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
@@ -47,6 +48,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
|
@@ -68,6 +70,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.ContainersAndNMTokensAllocation;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.ContainersAndNMTokensAllocation;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
|
@@ -1163,9 +1166,15 @@ public class FairScheduler extends
|
|
throw new RuntimeException("Unexpected event type: " + event);
|
|
throw new RuntimeException("Unexpected event type: " + event);
|
|
}
|
|
}
|
|
AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
|
|
AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
|
|
- addApplication(appAddedEvent.getApplicationId(),
|
|
|
|
- appAddedEvent.getQueue(), appAddedEvent.getUser(),
|
|
|
|
- appAddedEvent.getIsAppRecovering());
|
|
|
|
|
|
+ String queueName =
|
|
|
|
+ resolveReservationQueueName(appAddedEvent.getQueue(),
|
|
|
|
+ appAddedEvent.getApplicationId(),
|
|
|
|
+ appAddedEvent.getReservationID());
|
|
|
|
+ if (queueName != null) {
|
|
|
|
+ addApplication(appAddedEvent.getApplicationId(),
|
|
|
|
+ queueName, appAddedEvent.getUser(),
|
|
|
|
+ appAddedEvent.getIsAppRecovering());
|
|
|
|
+ }
|
|
break;
|
|
break;
|
|
case APP_REMOVED:
|
|
case APP_REMOVED:
|
|
if (!(event instanceof AppRemovedSchedulerEvent)) {
|
|
if (!(event instanceof AppRemovedSchedulerEvent)) {
|
|
@@ -1223,6 +1232,51 @@ public class FairScheduler extends
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private String resolveReservationQueueName(String queueName,
|
|
|
|
+ ApplicationId applicationId, ReservationId reservationID) {
|
|
|
|
+ FSQueue queue = queueMgr.getQueue(queueName);
|
|
|
|
+ if ((queue == null) || !allocConf.isReservable(queue.getQueueName())) {
|
|
|
|
+ return queueName;
|
|
|
|
+ }
|
|
|
|
+ // Use fully specified name from now on (including root. prefix)
|
|
|
|
+ queueName = queue.getQueueName();
|
|
|
|
+ if (reservationID != null) {
|
|
|
|
+ String resQName = queueName + "." + reservationID.toString();
|
|
|
|
+ queue = queueMgr.getQueue(resQName);
|
|
|
|
+ if (queue == null) {
|
|
|
|
+ String message =
|
|
|
|
+ "Application "
|
|
|
|
+ + applicationId
|
|
|
|
+ + " submitted to a reservation which is not yet currently active: "
|
|
|
|
+ + resQName;
|
|
|
|
+ this.rmContext.getDispatcher().getEventHandler()
|
|
|
|
+ .handle(new RMAppRejectedEvent(applicationId, message));
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ if (!queue.getParent().getQueueName().equals(queueName)) {
|
|
|
|
+ String message =
|
|
|
|
+ "Application: " + applicationId + " submitted to a reservation "
|
|
|
|
+ + resQName + " which does not belong to the specified queue: "
|
|
|
|
+ + queueName;
|
|
|
|
+ this.rmContext.getDispatcher().getEventHandler()
|
|
|
|
+ .handle(new RMAppRejectedEvent(applicationId, message));
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ // use the reservation queue to run the app
|
|
|
|
+ queueName = resQName;
|
|
|
|
+ } else {
|
|
|
|
+ // use the default child queue of the plan for unreserved apps
|
|
|
|
+ queueName = getDefaultQueueForPlanQueue(queueName);
|
|
|
|
+ }
|
|
|
|
+ return queueName;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private String getDefaultQueueForPlanQueue(String queueName) {
|
|
|
|
+ String planName = queueName.substring(queueName.lastIndexOf(".") + 1);
|
|
|
|
+ queueName = queueName + "." + planName + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
|
|
|
|
+ return queueName;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void recover(RMState state) throws Exception {
|
|
public void recover(RMState state) throws Exception {
|
|
// NOT IMPLEMENTED
|
|
// NOT IMPLEMENTED
|
|
@@ -1441,7 +1495,8 @@ public class FairScheduler extends
|
|
// To serialize with FairScheduler#allocate, synchronize on app attempt
|
|
// To serialize with FairScheduler#allocate, synchronize on app attempt
|
|
synchronized (attempt) {
|
|
synchronized (attempt) {
|
|
FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
|
|
FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
|
|
- FSLeafQueue targetQueue = queueMgr.getLeafQueue(queueName, false);
|
|
|
|
|
|
+ String destQueueName = handleMoveToPlanQueue(queueName);
|
|
|
|
+ FSLeafQueue targetQueue = queueMgr.getLeafQueue(destQueueName, false);
|
|
if (targetQueue == null) {
|
|
if (targetQueue == null) {
|
|
throw new YarnException("Target queue " + queueName
|
|
throw new YarnException("Target queue " + queueName
|
|
+ " not found or is not a leaf queue.");
|
|
+ " not found or is not a leaf queue.");
|
|
@@ -1577,4 +1632,45 @@ public class FairScheduler extends
|
|
}
|
|
}
|
|
return planQueues;
|
|
return planQueues;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void setEntitlement(String queueName,
|
|
|
|
+ QueueEntitlement entitlement) throws YarnException {
|
|
|
|
+
|
|
|
|
+ FSLeafQueue reservationQueue = queueMgr.getLeafQueue(queueName, false);
|
|
|
|
+ if (reservationQueue == null) {
|
|
|
|
+ throw new YarnException("Target queue " + queueName
|
|
|
|
+ + " not found or is not a leaf queue.");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ reservationQueue.setWeights(entitlement.getCapacity());
|
|
|
|
+
|
|
|
|
+ // TODO Does MaxCapacity need to be set for fairScheduler ?
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Only supports removing empty leaf queues
|
|
|
|
+ * @param queueName name of queue to remove
|
|
|
|
+ * @throws YarnException if queue to remove is either not a leaf or if its
|
|
|
|
+ * not empty
|
|
|
|
+ */
|
|
|
|
+ @Override
|
|
|
|
+ public void removeQueue(String queueName) throws YarnException {
|
|
|
|
+ FSLeafQueue reservationQueue = queueMgr.getLeafQueue(queueName, false);
|
|
|
|
+ if (reservationQueue != null) {
|
|
|
|
+ if (!queueMgr.removeLeafQueue(queueName)) {
|
|
|
|
+ throw new YarnException("Could not remove queue " + queueName + " as " +
|
|
|
|
+ "its either not a leaf queue or its not empty");
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private String handleMoveToPlanQueue(String targetQueueName) {
|
|
|
|
+ FSQueue dest = queueMgr.getQueue(targetQueueName);
|
|
|
|
+ if (dest != null && allocConf.isReservable(dest.getQueueName())) {
|
|
|
|
+ // use the default child reservation queue of the plan
|
|
|
|
+ targetQueueName = getDefaultQueueForPlanQueue(targetQueueName);
|
|
|
|
+ }
|
|
|
|
+ return targetQueueName;
|
|
|
|
+ }
|
|
}
|
|
}
|