|
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
|
|
+import com.google.common.collect.Lists;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
|
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
|
@@ -42,6 +43,8 @@ import org.apache.hadoop.yarn.api.records.ResourceOption;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
|
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
|
+import org.apache.hadoop.yarn.exceptions
|
|
|
|
+ .SchedulerInvalidResoureRequestException;
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
|
@@ -73,6 +76,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
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.SchedulerUtils;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils.MaxResourceValidationResult;
|
|
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.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;
|
|
@@ -449,10 +453,7 @@ public class FairScheduler extends
|
|
String message =
|
|
String message =
|
|
"Reject application " + applicationId + " submitted by user " + user
|
|
"Reject application " + applicationId + " submitted by user " + user
|
|
+ " with an empty queue name.";
|
|
+ " with an empty queue name.";
|
|
- LOG.info(message);
|
|
|
|
- rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
- new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
|
|
|
|
- message));
|
|
|
|
|
|
+ rejectApplicationWithMessage(applicationId, message);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -461,10 +462,7 @@ public class FairScheduler extends
|
|
"Reject application " + applicationId + " submitted by user " + user
|
|
"Reject application " + applicationId + " submitted by user " + user
|
|
+ " with an illegal queue name " + queueName + ". "
|
|
+ " with an illegal queue name " + queueName + ". "
|
|
+ "The queue name cannot start/end with period.";
|
|
+ "The queue name cannot start/end with period.";
|
|
- LOG.info(message);
|
|
|
|
- rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
- new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
|
|
|
|
- message));
|
|
|
|
|
|
+ rejectApplicationWithMessage(applicationId, message);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -476,6 +474,31 @@ public class FairScheduler extends
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ if (rmApp != null && rmApp.getAMResourceRequests() != null) {
|
|
|
|
+ // Resources.fitsIn would always return false when queueMaxShare is 0
|
|
|
|
+ // for any resource, but only using Resources.fitsIn is not enough
|
|
|
|
+ // is it would return false for such cases when the requested
|
|
|
|
+ // resource is smaller than the max resource but that max resource is
|
|
|
|
+ // not zero, e.g. requested vCores = 2, max vCores = 1.
|
|
|
|
+ // With this check, we only reject those applications where resource
|
|
|
|
+ // requested is greater than 0 and we have 0
|
|
|
|
+ // of that resource on the queue.
|
|
|
|
+ List<MaxResourceValidationResult> invalidAMResourceRequests =
|
|
|
|
+ validateResourceRequests(rmApp.getAMResourceRequests(), queue);
|
|
|
|
+
|
|
|
|
+ if (!invalidAMResourceRequests.isEmpty()) {
|
|
|
|
+ String msg = String.format(
|
|
|
|
+ "Cannot submit application %s to queue %s because "
|
|
|
|
+ + "it has zero amount of resource for a requested "
|
|
|
|
+ + "resource! Invalid requested AM resources: %s, "
|
|
|
|
+ + "maximum queue resources: %s",
|
|
|
|
+ applicationId, queue.getName(),
|
|
|
|
+ invalidAMResourceRequests, queue.getMaxShare());
|
|
|
|
+ rejectApplicationWithMessage(applicationId, msg);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
// Enforce ACLs
|
|
// Enforce ACLs
|
|
UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(
|
|
UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(
|
|
user);
|
|
user);
|
|
@@ -485,9 +508,7 @@ public class FairScheduler extends
|
|
String msg = "User " + userUgi.getUserName()
|
|
String msg = "User " + userUgi.getUserName()
|
|
+ " cannot submit applications to queue " + queue.getName()
|
|
+ " cannot submit applications to queue " + queue.getName()
|
|
+ "(requested queuename is " + queueName + ")";
|
|
+ "(requested queuename is " + queueName + ")";
|
|
- LOG.info(msg);
|
|
|
|
- rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
- new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED, msg));
|
|
|
|
|
|
+ rejectApplicationWithMessage(applicationId, msg);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -604,10 +625,7 @@ public class FairScheduler extends
|
|
}
|
|
}
|
|
|
|
|
|
if (appRejectMsg != null && rmApp != null) {
|
|
if (appRejectMsg != null && rmApp != null) {
|
|
- LOG.error(appRejectMsg);
|
|
|
|
- rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
- new RMAppEvent(rmApp.getApplicationId(),
|
|
|
|
- RMAppEventType.APP_REJECTED, appRejectMsg));
|
|
|
|
|
|
+ rejectApplicationWithMessage(rmApp.getApplicationId(), appRejectMsg);
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -834,7 +852,6 @@ public class FairScheduler extends
|
|
List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
|
|
List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
|
|
List<ContainerId> release, List<String> blacklistAdditions,
|
|
List<ContainerId> release, List<String> blacklistAdditions,
|
|
List<String> blacklistRemovals, ContainerUpdates updateRequests) {
|
|
List<String> blacklistRemovals, ContainerUpdates updateRequests) {
|
|
-
|
|
|
|
// Make sure this application exists
|
|
// Make sure this application exists
|
|
FSAppAttempt application = getSchedulerApp(appAttemptId);
|
|
FSAppAttempt application = getSchedulerApp(appAttemptId);
|
|
if (application == null) {
|
|
if (application == null) {
|
|
@@ -854,6 +871,24 @@ public class FairScheduler extends
|
|
return EMPTY_ALLOCATION;
|
|
return EMPTY_ALLOCATION;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ ApplicationId applicationId = application.getApplicationId();
|
|
|
|
+ FSLeafQueue queue = application.getQueue();
|
|
|
|
+ List<MaxResourceValidationResult> invalidAsks =
|
|
|
|
+ validateResourceRequests(ask, queue);
|
|
|
|
+
|
|
|
|
+ // We need to be fail-fast here if any invalid ask is detected.
|
|
|
|
+ // If we would have thrown exception later, this could be problematic as
|
|
|
|
+ // tokens and promoted / demoted containers would have been lost because
|
|
|
|
+ // scheduler would clear them right away and AM
|
|
|
|
+ // would not get this information.
|
|
|
|
+ if (!invalidAsks.isEmpty()) {
|
|
|
|
+ throw new SchedulerInvalidResoureRequestException(String.format(
|
|
|
|
+ "Resource request is invalid for application %s because queue %s "
|
|
|
|
+ + "has 0 amount of resource for a resource type! "
|
|
|
|
+ + "Validation result: %s",
|
|
|
|
+ applicationId, queue.getName(), invalidAsks));
|
|
|
|
+ }
|
|
|
|
+
|
|
// Handle promotions and demotions
|
|
// Handle promotions and demotions
|
|
handleContainerUpdates(application, updateRequests);
|
|
handleContainerUpdates(application, updateRequests);
|
|
|
|
|
|
@@ -912,6 +947,7 @@ public class FairScheduler extends
|
|
|
|
|
|
Resource headroom = application.getHeadroom();
|
|
Resource headroom = application.getHeadroom();
|
|
application.setApplicationHeadroomForMetrics(headroom);
|
|
application.setApplicationHeadroomForMetrics(headroom);
|
|
|
|
+
|
|
return new Allocation(newlyAllocatedContainers, headroom,
|
|
return new Allocation(newlyAllocatedContainers, headroom,
|
|
preemptionContainerIds, null, null,
|
|
preemptionContainerIds, null, null,
|
|
application.pullUpdatedNMTokens(), null, null,
|
|
application.pullUpdatedNMTokens(), null, null,
|
|
@@ -920,6 +956,34 @@ public class FairScheduler extends
|
|
application.pullPreviousAttemptContainers());
|
|
application.pullPreviousAttemptContainers());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private List<MaxResourceValidationResult> validateResourceRequests(
|
|
|
|
+ List<ResourceRequest> requests, FSLeafQueue queue) {
|
|
|
|
+ List<MaxResourceValidationResult> validationResults = Lists.newArrayList();
|
|
|
|
+
|
|
|
|
+ for (ResourceRequest resourceRequest : requests) {
|
|
|
|
+ if (LOG.isTraceEnabled()) {
|
|
|
|
+ LOG.trace("Validating resource request: " + resourceRequest);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ MaxResourceValidationResult validationResult =
|
|
|
|
+ SchedulerUtils.validateResourceRequestsAgainstQueueMaxResource(
|
|
|
|
+ resourceRequest, queue.getMaxShare());
|
|
|
|
+ if (!validationResult.isValid()) {
|
|
|
|
+ validationResults.add(validationResult);
|
|
|
|
+ LOG.warn(String.format("Queue %s cannot handle resource request" +
|
|
|
|
+ "because it has zero available amount of resource " +
|
|
|
|
+ "for a requested resource type, " +
|
|
|
|
+ "so the resource request is ignored!"
|
|
|
|
+ + " Requested resources: %s, " +
|
|
|
|
+ "maximum queue resources: %s",
|
|
|
|
+ queue.getName(), resourceRequest.getCapability(),
|
|
|
|
+ queue.getMaxShare()));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return validationResults;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
protected void nodeUpdate(RMNode nm) {
|
|
protected void nodeUpdate(RMNode nm) {
|
|
try {
|
|
try {
|
|
@@ -1060,9 +1124,14 @@ public class FairScheduler extends
|
|
Resource assignedResource = Resources.clone(Resources.none());
|
|
Resource assignedResource = Resources.clone(Resources.none());
|
|
Resource maxResourcesToAssign = Resources.multiply(
|
|
Resource maxResourcesToAssign = Resources.multiply(
|
|
node.getUnallocatedResource(), 0.5f);
|
|
node.getUnallocatedResource(), 0.5f);
|
|
|
|
+
|
|
while (node.getReservedContainer() == null) {
|
|
while (node.getReservedContainer() == null) {
|
|
Resource assignment = queueMgr.getRootQueue().assignContainer(node);
|
|
Resource assignment = queueMgr.getRootQueue().assignContainer(node);
|
|
|
|
+
|
|
if (assignment.equals(Resources.none())) {
|
|
if (assignment.equals(Resources.none())) {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("No container is allocated on node " + node);
|
|
|
|
+ }
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1254,9 +1323,7 @@ public class FairScheduler extends
|
|
String message = "Application " + applicationId
|
|
String message = "Application " + applicationId
|
|
+ " submitted to a reservation which is not yet "
|
|
+ " submitted to a reservation which is not yet "
|
|
+ "currently active: " + resQName;
|
|
+ "currently active: " + resQName;
|
|
- this.rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
- new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
|
|
|
|
- message));
|
|
|
|
|
|
+ rejectApplicationWithMessage(applicationId, message);
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
if (!queue.getParent().getQueueName().equals(queueName)) {
|
|
if (!queue.getParent().getQueueName().equals(queueName)) {
|
|
@@ -1264,9 +1331,7 @@ public class FairScheduler extends
|
|
"Application: " + applicationId + " submitted to a reservation "
|
|
"Application: " + applicationId + " submitted to a reservation "
|
|
+ resQName + " which does not belong to the specified queue: "
|
|
+ resQName + " which does not belong to the specified queue: "
|
|
+ queueName;
|
|
+ queueName;
|
|
- this.rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
- new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
|
|
|
|
- message));
|
|
|
|
|
|
+ rejectApplicationWithMessage(applicationId, message);
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
// use the reservation queue to run the app
|
|
// use the reservation queue to run the app
|
|
@@ -1279,7 +1344,13 @@ public class FairScheduler extends
|
|
} finally {
|
|
} finally {
|
|
readLock.unlock();
|
|
readLock.unlock();
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ private void rejectApplicationWithMessage(ApplicationId applicationId,
|
|
|
|
+ String msg) {
|
|
|
|
+ LOG.info(msg);
|
|
|
|
+ rmContext.getDispatcher().getEventHandler().handle(new RMAppEvent(
|
|
|
|
+ applicationId, RMAppEventType.APP_REJECTED, msg));
|
|
}
|
|
}
|
|
|
|
|
|
private String getDefaultQueueForPlanQueue(String queueName) {
|
|
private String getDefaultQueueForPlanQueue(String queueName) {
|