|
@@ -24,7 +24,6 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
|
|
|
-import org.apache.commons.lang.mutable.MutableObject;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
@@ -40,9 +39,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
|
@@ -54,15 +51,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
|
|
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AllocationState;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocator;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.RegularContainerAllocator;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocation;
|
|
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
@@ -78,11 +76,6 @@ import com.google.common.annotations.VisibleForTesting;
|
|
|
public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|
|
private static final Log LOG = LogFactory.getLog(FiCaSchedulerApp.class);
|
|
|
|
|
|
- static final CSAssignment NULL_ASSIGNMENT =
|
|
|
- new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
|
|
|
-
|
|
|
- static final CSAssignment SKIP_ASSIGNMENT = new CSAssignment(true);
|
|
|
-
|
|
|
private final Set<ContainerId> containersToPreempt =
|
|
|
new HashSet<ContainerId>();
|
|
|
|
|
@@ -91,6 +84,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|
|
private ResourceCalculator rc = new DefaultResourceCalculator();
|
|
|
|
|
|
private ResourceScheduler scheduler;
|
|
|
+
|
|
|
+ private ContainerAllocator containerAllocator;
|
|
|
|
|
|
public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId,
|
|
|
String user, Queue queue, ActiveUsersManager activeUsersManager,
|
|
@@ -124,6 +119,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|
|
if (scheduler.getResourceCalculator() != null) {
|
|
|
rc = scheduler.getResourceCalculator();
|
|
|
}
|
|
|
+
|
|
|
+ containerAllocator = new RegularContainerAllocator(this, rc, rmContext);
|
|
|
}
|
|
|
|
|
|
synchronized public boolean containerCompleted(RMContainer rmContainer,
|
|
@@ -386,223 +383,6 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|
|
((FiCaSchedulerApp) appAttempt).getHeadroomProvider();
|
|
|
}
|
|
|
|
|
|
- private int getActualNodeLocalityDelay() {
|
|
|
- return Math.min(scheduler.getNumClusterNodes(), getCSLeafQueue()
|
|
|
- .getNodeLocalityDelay());
|
|
|
- }
|
|
|
-
|
|
|
- private boolean canAssign(Priority priority, FiCaSchedulerNode node,
|
|
|
- NodeType type, RMContainer reservedContainer) {
|
|
|
-
|
|
|
- // Clearly we need containers for this application...
|
|
|
- if (type == NodeType.OFF_SWITCH) {
|
|
|
- if (reservedContainer != null) {
|
|
|
- return true;
|
|
|
- }
|
|
|
-
|
|
|
- // 'Delay' off-switch
|
|
|
- ResourceRequest offSwitchRequest =
|
|
|
- getResourceRequest(priority, ResourceRequest.ANY);
|
|
|
- long missedOpportunities = getSchedulingOpportunities(priority);
|
|
|
- long requiredContainers = offSwitchRequest.getNumContainers();
|
|
|
-
|
|
|
- float localityWaitFactor =
|
|
|
- getLocalityWaitFactor(priority, scheduler.getNumClusterNodes());
|
|
|
-
|
|
|
- return ((requiredContainers * localityWaitFactor) < missedOpportunities);
|
|
|
- }
|
|
|
-
|
|
|
- // Check if we need containers on this rack
|
|
|
- ResourceRequest rackLocalRequest =
|
|
|
- getResourceRequest(priority, node.getRackName());
|
|
|
- if (rackLocalRequest == null || rackLocalRequest.getNumContainers() <= 0) {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- // If we are here, we do need containers on this rack for RACK_LOCAL req
|
|
|
- if (type == NodeType.RACK_LOCAL) {
|
|
|
- // 'Delay' rack-local just a little bit...
|
|
|
- long missedOpportunities = getSchedulingOpportunities(priority);
|
|
|
- return getActualNodeLocalityDelay() < missedOpportunities;
|
|
|
- }
|
|
|
-
|
|
|
- // Check if we need containers on this host
|
|
|
- if (type == NodeType.NODE_LOCAL) {
|
|
|
- // Now check if we need containers on this host...
|
|
|
- ResourceRequest nodeLocalRequest =
|
|
|
- getResourceRequest(priority, node.getNodeName());
|
|
|
- if (nodeLocalRequest != null) {
|
|
|
- return nodeLocalRequest.getNumContainers() > 0;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- boolean
|
|
|
- shouldAllocOrReserveNewContainer(Priority priority, Resource required) {
|
|
|
- int requiredContainers = getTotalRequiredResources(priority);
|
|
|
- int reservedContainers = getNumReservedContainers(priority);
|
|
|
- int starvation = 0;
|
|
|
- if (reservedContainers > 0) {
|
|
|
- float nodeFactor =
|
|
|
- Resources.ratio(
|
|
|
- rc, required, getCSLeafQueue().getMaximumAllocation()
|
|
|
- );
|
|
|
-
|
|
|
- // Use percentage of node required to bias against large containers...
|
|
|
- // Protect against corner case where you need the whole node with
|
|
|
- // Math.min(nodeFactor, minimumAllocationFactor)
|
|
|
- starvation =
|
|
|
- (int)((getReReservations(priority) / (float)reservedContainers) *
|
|
|
- (1.0f - (Math.min(nodeFactor, getCSLeafQueue().getMinimumAllocationFactor())))
|
|
|
- );
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("needsContainers:" +
|
|
|
- " app.#re-reserve=" + getReReservations(priority) +
|
|
|
- " reserved=" + reservedContainers +
|
|
|
- " nodeFactor=" + nodeFactor +
|
|
|
- " minAllocFactor=" + getCSLeafQueue().getMinimumAllocationFactor() +
|
|
|
- " starvation=" + starvation);
|
|
|
- }
|
|
|
- }
|
|
|
- return (((starvation + requiredContainers) - reservedContainers) > 0);
|
|
|
- }
|
|
|
-
|
|
|
- private CSAssignment assignNodeLocalContainers(Resource clusterResource,
|
|
|
- ResourceRequest nodeLocalResourceRequest, FiCaSchedulerNode node,
|
|
|
- Priority priority,
|
|
|
- RMContainer reservedContainer, MutableObject allocatedContainer,
|
|
|
- SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
|
|
|
- if (canAssign(priority, node, NodeType.NODE_LOCAL,
|
|
|
- reservedContainer)) {
|
|
|
- return assignContainer(clusterResource, node, priority,
|
|
|
- nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer,
|
|
|
- allocatedContainer, schedulingMode, currentResoureLimits);
|
|
|
- }
|
|
|
-
|
|
|
- return new CSAssignment(Resources.none(), NodeType.NODE_LOCAL);
|
|
|
- }
|
|
|
-
|
|
|
- private CSAssignment assignRackLocalContainers(Resource clusterResource,
|
|
|
- ResourceRequest rackLocalResourceRequest, FiCaSchedulerNode node,
|
|
|
- Priority priority,
|
|
|
- RMContainer reservedContainer, MutableObject allocatedContainer,
|
|
|
- SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
|
|
|
- if (canAssign(priority, node, NodeType.RACK_LOCAL,
|
|
|
- reservedContainer)) {
|
|
|
- return assignContainer(clusterResource, node, priority,
|
|
|
- rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer,
|
|
|
- allocatedContainer, schedulingMode, currentResoureLimits);
|
|
|
- }
|
|
|
-
|
|
|
- return new CSAssignment(Resources.none(), NodeType.RACK_LOCAL);
|
|
|
- }
|
|
|
-
|
|
|
- private CSAssignment assignOffSwitchContainers(Resource clusterResource,
|
|
|
- ResourceRequest offSwitchResourceRequest, FiCaSchedulerNode node,
|
|
|
- Priority priority,
|
|
|
- RMContainer reservedContainer, MutableObject allocatedContainer,
|
|
|
- SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
|
|
|
- if (canAssign(priority, node, NodeType.OFF_SWITCH,
|
|
|
- reservedContainer)) {
|
|
|
- return assignContainer(clusterResource, node, priority,
|
|
|
- offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer,
|
|
|
- allocatedContainer, schedulingMode, currentResoureLimits);
|
|
|
- }
|
|
|
-
|
|
|
- return new CSAssignment(Resources.none(), NodeType.OFF_SWITCH);
|
|
|
- }
|
|
|
-
|
|
|
- private CSAssignment assignContainersOnNode(Resource clusterResource,
|
|
|
- FiCaSchedulerNode node, Priority priority,
|
|
|
- RMContainer reservedContainer, SchedulingMode schedulingMode,
|
|
|
- ResourceLimits currentResoureLimits) {
|
|
|
-
|
|
|
- CSAssignment assigned;
|
|
|
-
|
|
|
- NodeType requestType = null;
|
|
|
- MutableObject allocatedContainer = new MutableObject();
|
|
|
- // Data-local
|
|
|
- ResourceRequest nodeLocalResourceRequest =
|
|
|
- getResourceRequest(priority, node.getNodeName());
|
|
|
- if (nodeLocalResourceRequest != null) {
|
|
|
- requestType = NodeType.NODE_LOCAL;
|
|
|
- assigned =
|
|
|
- assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest,
|
|
|
- node, priority, reservedContainer,
|
|
|
- allocatedContainer, schedulingMode, currentResoureLimits);
|
|
|
- if (Resources.greaterThan(rc, clusterResource,
|
|
|
- assigned.getResource(), Resources.none())) {
|
|
|
-
|
|
|
- //update locality statistics
|
|
|
- if (allocatedContainer.getValue() != null) {
|
|
|
- incNumAllocatedContainers(NodeType.NODE_LOCAL,
|
|
|
- requestType);
|
|
|
- }
|
|
|
- assigned.setType(NodeType.NODE_LOCAL);
|
|
|
- return assigned;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Rack-local
|
|
|
- ResourceRequest rackLocalResourceRequest =
|
|
|
- getResourceRequest(priority, node.getRackName());
|
|
|
- if (rackLocalResourceRequest != null) {
|
|
|
- if (!rackLocalResourceRequest.getRelaxLocality()) {
|
|
|
- return SKIP_ASSIGNMENT;
|
|
|
- }
|
|
|
-
|
|
|
- if (requestType != NodeType.NODE_LOCAL) {
|
|
|
- requestType = NodeType.RACK_LOCAL;
|
|
|
- }
|
|
|
-
|
|
|
- assigned =
|
|
|
- assignRackLocalContainers(clusterResource, rackLocalResourceRequest,
|
|
|
- node, priority, reservedContainer,
|
|
|
- allocatedContainer, schedulingMode, currentResoureLimits);
|
|
|
- if (Resources.greaterThan(rc, clusterResource,
|
|
|
- assigned.getResource(), Resources.none())) {
|
|
|
-
|
|
|
- //update locality statistics
|
|
|
- if (allocatedContainer.getValue() != null) {
|
|
|
- incNumAllocatedContainers(NodeType.RACK_LOCAL,
|
|
|
- requestType);
|
|
|
- }
|
|
|
- assigned.setType(NodeType.RACK_LOCAL);
|
|
|
- return assigned;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Off-switch
|
|
|
- ResourceRequest offSwitchResourceRequest =
|
|
|
- getResourceRequest(priority, ResourceRequest.ANY);
|
|
|
- if (offSwitchResourceRequest != null) {
|
|
|
- if (!offSwitchResourceRequest.getRelaxLocality()) {
|
|
|
- return SKIP_ASSIGNMENT;
|
|
|
- }
|
|
|
- if (requestType != NodeType.NODE_LOCAL
|
|
|
- && requestType != NodeType.RACK_LOCAL) {
|
|
|
- requestType = NodeType.OFF_SWITCH;
|
|
|
- }
|
|
|
-
|
|
|
- assigned =
|
|
|
- assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
|
|
|
- node, priority, reservedContainer,
|
|
|
- allocatedContainer, schedulingMode, currentResoureLimits);
|
|
|
-
|
|
|
- // update locality statistics
|
|
|
- if (allocatedContainer.getValue() != null) {
|
|
|
- incNumAllocatedContainers(NodeType.OFF_SWITCH, requestType);
|
|
|
- }
|
|
|
- assigned.setType(NodeType.OFF_SWITCH);
|
|
|
- return assigned;
|
|
|
- }
|
|
|
-
|
|
|
- return SKIP_ASSIGNMENT;
|
|
|
- }
|
|
|
-
|
|
|
public void reserve(Priority priority,
|
|
|
FiCaSchedulerNode node, RMContainer rmContainer, Container container) {
|
|
|
// Update reserved metrics if this is the first reservation
|
|
@@ -618,25 +398,6 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|
|
node.reserveResource(this, priority, rmContainer);
|
|
|
}
|
|
|
|
|
|
- private Container getContainer(RMContainer rmContainer,
|
|
|
- FiCaSchedulerNode node, Resource capability, Priority priority) {
|
|
|
- return (rmContainer != null) ? rmContainer.getContainer()
|
|
|
- : createContainer(node, capability, priority);
|
|
|
- }
|
|
|
-
|
|
|
- Container createContainer(FiCaSchedulerNode node, Resource capability,
|
|
|
- Priority priority) {
|
|
|
-
|
|
|
- NodeId nodeId = node.getRMNode().getNodeID();
|
|
|
- ContainerId containerId =
|
|
|
- BuilderUtils.newContainerId(getApplicationAttemptId(),
|
|
|
- getNewContainerId());
|
|
|
-
|
|
|
- // Create the container
|
|
|
- return BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
|
|
|
- .getHttpAddress(), capability, priority, null);
|
|
|
- }
|
|
|
-
|
|
|
@VisibleForTesting
|
|
|
public RMContainer findNodeToUnreserve(Resource clusterResource,
|
|
|
FiCaSchedulerNode node, Priority priority,
|
|
@@ -672,203 +433,63 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|
|
return nodeToUnreserve.getReservedContainer();
|
|
|
}
|
|
|
|
|
|
- private LeafQueue getCSLeafQueue() {
|
|
|
+ public LeafQueue getCSLeafQueue() {
|
|
|
return (LeafQueue)queue;
|
|
|
}
|
|
|
|
|
|
- private CSAssignment assignContainer(Resource clusterResource, FiCaSchedulerNode node,
|
|
|
- Priority priority,
|
|
|
- ResourceRequest request, NodeType type, RMContainer rmContainer,
|
|
|
- MutableObject createdContainer, SchedulingMode schedulingMode,
|
|
|
- ResourceLimits currentResoureLimits) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("assignContainers: node=" + node.getNodeName()
|
|
|
- + " application=" + getApplicationId()
|
|
|
- + " priority=" + priority.getPriority()
|
|
|
- + " request=" + request + " type=" + type);
|
|
|
- }
|
|
|
-
|
|
|
- // check if the resource request can access the label
|
|
|
- if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(request,
|
|
|
- node.getPartition(), schedulingMode)) {
|
|
|
- // this is a reserved container, but we cannot allocate it now according
|
|
|
- // to label not match. This can be caused by node label changed
|
|
|
- // We should un-reserve this container.
|
|
|
- if (rmContainer != null) {
|
|
|
- unreserve(priority, node, rmContainer);
|
|
|
- }
|
|
|
- return new CSAssignment(Resources.none(), type);
|
|
|
- }
|
|
|
-
|
|
|
- Resource capability = request.getCapability();
|
|
|
- Resource available = node.getAvailableResource();
|
|
|
- Resource totalResource = node.getTotalResource();
|
|
|
-
|
|
|
- if (!Resources.lessThanOrEqual(rc, clusterResource,
|
|
|
- capability, totalResource)) {
|
|
|
- LOG.warn("Node : " + node.getNodeID()
|
|
|
- + " does not have sufficient resource for request : " + request
|
|
|
- + " node total capability : " + node.getTotalResource());
|
|
|
- return new CSAssignment(Resources.none(), type);
|
|
|
- }
|
|
|
-
|
|
|
- assert Resources.greaterThan(
|
|
|
- rc, clusterResource, available, Resources.none());
|
|
|
-
|
|
|
- // Create the container if necessary
|
|
|
- Container container =
|
|
|
- getContainer(rmContainer, node, capability, priority);
|
|
|
-
|
|
|
- // something went wrong getting/creating the container
|
|
|
- if (container == null) {
|
|
|
- LOG.warn("Couldn't get container for allocation!");
|
|
|
- return new CSAssignment(Resources.none(), type);
|
|
|
- }
|
|
|
-
|
|
|
- boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
|
|
|
- priority, capability);
|
|
|
-
|
|
|
- // Can we allocate a container on this node?
|
|
|
- int availableContainers =
|
|
|
- rc.computeAvailableContainers(available, capability);
|
|
|
-
|
|
|
- // How much need to unreserve equals to:
|
|
|
- // max(required - headroom, amountNeedUnreserve)
|
|
|
- Resource resourceNeedToUnReserve =
|
|
|
- Resources.max(rc, clusterResource,
|
|
|
- Resources.subtract(capability, currentResoureLimits.getHeadroom()),
|
|
|
- currentResoureLimits.getAmountNeededUnreserve());
|
|
|
-
|
|
|
- boolean needToUnreserve =
|
|
|
- Resources.greaterThan(rc, clusterResource,
|
|
|
- resourceNeedToUnReserve, Resources.none());
|
|
|
-
|
|
|
- RMContainer unreservedContainer = null;
|
|
|
- boolean reservationsContinueLooking =
|
|
|
- getCSLeafQueue().getReservationContinueLooking();
|
|
|
-
|
|
|
- if (availableContainers > 0) {
|
|
|
- // Allocate...
|
|
|
-
|
|
|
- // Did we previously reserve containers at this 'priority'?
|
|
|
- if (rmContainer != null) {
|
|
|
- unreserve(priority, node, rmContainer);
|
|
|
- } else if (reservationsContinueLooking && node.getLabels().isEmpty()) {
|
|
|
- // when reservationsContinueLooking is set, we may need to unreserve
|
|
|
- // some containers to meet this queue, its parents', or the users' resource limits.
|
|
|
- // TODO, need change here when we want to support continuous reservation
|
|
|
- // looking for labeled partitions.
|
|
|
- if (!shouldAllocOrReserveNewContainer || needToUnreserve) {
|
|
|
- if (!needToUnreserve) {
|
|
|
- // If we shouldn't allocate/reserve new container then we should
|
|
|
- // unreserve one the same size we are asking for since the
|
|
|
- // currentResoureLimits.getAmountNeededUnreserve could be zero. If
|
|
|
- // the limit was hit then use the amount we need to unreserve to be
|
|
|
- // under the limit.
|
|
|
- resourceNeedToUnReserve = capability;
|
|
|
- }
|
|
|
- unreservedContainer =
|
|
|
- findNodeToUnreserve(clusterResource, node, priority,
|
|
|
- resourceNeedToUnReserve);
|
|
|
- // When (minimum-unreserved-resource > 0 OR we cannot allocate new/reserved
|
|
|
- // container (That means we *have to* unreserve some resource to
|
|
|
- // continue)). If we failed to unreserve some resource, we can't continue.
|
|
|
- if (null == unreservedContainer) {
|
|
|
- return new CSAssignment(Resources.none(), type);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Inform the application
|
|
|
- RMContainer allocatedContainer =
|
|
|
- allocate(type, node, priority, request, container);
|
|
|
-
|
|
|
- // Does the application need this resource?
|
|
|
- if (allocatedContainer == null) {
|
|
|
- CSAssignment csAssignment = new CSAssignment(Resources.none(), type);
|
|
|
- csAssignment.setApplication(this);
|
|
|
- csAssignment.setExcessReservation(unreservedContainer);
|
|
|
- return csAssignment;
|
|
|
- }
|
|
|
-
|
|
|
- // Inform the node
|
|
|
- node.allocateContainer(allocatedContainer);
|
|
|
-
|
|
|
- // Inform the ordering policy
|
|
|
- getCSLeafQueue().getOrderingPolicy().containerAllocated(this,
|
|
|
- allocatedContainer);
|
|
|
-
|
|
|
- LOG.info("assignedContainer" +
|
|
|
- " application attempt=" + getApplicationAttemptId() +
|
|
|
- " container=" + container +
|
|
|
- " queue=" + this +
|
|
|
- " clusterResource=" + clusterResource);
|
|
|
- createdContainer.setValue(allocatedContainer);
|
|
|
- CSAssignment assignment = new CSAssignment(container.getResource(), type);
|
|
|
- assignment.getAssignmentInformation().addAllocationDetails(
|
|
|
- container.getId(), getCSLeafQueue().getQueuePath());
|
|
|
- assignment.getAssignmentInformation().incrAllocations();
|
|
|
- assignment.setApplication(this);
|
|
|
- Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
|
|
|
- container.getResource());
|
|
|
-
|
|
|
- assignment.setExcessReservation(unreservedContainer);
|
|
|
- return assignment;
|
|
|
- } else {
|
|
|
- // if we are allowed to allocate but this node doesn't have space, reserve it or
|
|
|
- // if this was an already a reserved container, reserve it again
|
|
|
- if (shouldAllocOrReserveNewContainer || rmContainer != null) {
|
|
|
-
|
|
|
- if (reservationsContinueLooking && rmContainer == null) {
|
|
|
- // we could possibly ignoring queue capacity or user limits when
|
|
|
- // reservationsContinueLooking is set. Make sure we didn't need to unreserve
|
|
|
- // one.
|
|
|
- if (needToUnreserve) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("we needed to unreserve to be able to allocate");
|
|
|
- }
|
|
|
- return new CSAssignment(Resources.none(), type);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Reserve by 'charging' in advance...
|
|
|
- reserve(priority, node, rmContainer, container);
|
|
|
-
|
|
|
- LOG.info("Reserved container " +
|
|
|
- " application=" + getApplicationId() +
|
|
|
- " resource=" + request.getCapability() +
|
|
|
- " queue=" + this.toString() +
|
|
|
- " cluster=" + clusterResource);
|
|
|
- CSAssignment assignment =
|
|
|
- new CSAssignment(request.getCapability(), type);
|
|
|
+ private CSAssignment getCSAssignmentFromAllocateResult(
|
|
|
+ Resource clusterResource, ContainerAllocation result) {
|
|
|
+ // Handle skipped
|
|
|
+ boolean skipped =
|
|
|
+ (result.getAllocationState() == AllocationState.APP_SKIPPED);
|
|
|
+ CSAssignment assignment = new CSAssignment(skipped);
|
|
|
+ assignment.setApplication(this);
|
|
|
+
|
|
|
+ // Handle excess reservation
|
|
|
+ assignment.setExcessReservation(result.getContainerToBeUnreserved());
|
|
|
+
|
|
|
+ // If we allocated something
|
|
|
+ if (Resources.greaterThan(rc, clusterResource,
|
|
|
+ result.getResourceToBeAllocated(), Resources.none())) {
|
|
|
+ Resource allocatedResource = result.getResourceToBeAllocated();
|
|
|
+ Container updatedContainer = result.getUpdatedContainer();
|
|
|
+
|
|
|
+ assignment.setResource(allocatedResource);
|
|
|
+ assignment.setType(result.getContainerNodeType());
|
|
|
+
|
|
|
+ if (result.getAllocationState() == AllocationState.RESERVED) {
|
|
|
+ // This is a reserved container
|
|
|
+ LOG.info("Reserved container " + " application=" + getApplicationId()
|
|
|
+ + " resource=" + allocatedResource + " queue="
|
|
|
+ + this.toString() + " cluster=" + clusterResource);
|
|
|
assignment.getAssignmentInformation().addReservationDetails(
|
|
|
- container.getId(), getCSLeafQueue().getQueuePath());
|
|
|
+ updatedContainer.getId(), getCSLeafQueue().getQueuePath());
|
|
|
assignment.getAssignmentInformation().incrReservations();
|
|
|
Resources.addTo(assignment.getAssignmentInformation().getReserved(),
|
|
|
- request.getCapability());
|
|
|
- return assignment;
|
|
|
+ allocatedResource);
|
|
|
+ assignment.setFulfilledReservation(true);
|
|
|
+ } else {
|
|
|
+ // This is a new container
|
|
|
+ // Inform the ordering policy
|
|
|
+ LOG.info("assignedContainer" + " application attempt="
|
|
|
+ + getApplicationAttemptId() + " container="
|
|
|
+ + updatedContainer.getId() + " queue=" + this + " clusterResource="
|
|
|
+ + clusterResource);
|
|
|
+
|
|
|
+ getCSLeafQueue().getOrderingPolicy().containerAllocated(this,
|
|
|
+ getRMContainer(updatedContainer.getId()));
|
|
|
+
|
|
|
+ assignment.getAssignmentInformation().addAllocationDetails(
|
|
|
+ updatedContainer.getId(), getCSLeafQueue().getQueuePath());
|
|
|
+ assignment.getAssignmentInformation().incrAllocations();
|
|
|
+ Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
|
|
|
+ allocatedResource);
|
|
|
}
|
|
|
- return new CSAssignment(Resources.none(), type);
|
|
|
}
|
|
|
+
|
|
|
+ return assignment;
|
|
|
}
|
|
|
-
|
|
|
- private boolean checkHeadroom(Resource clusterResource,
|
|
|
- ResourceLimits currentResourceLimits, Resource required, FiCaSchedulerNode node) {
|
|
|
- // If headroom + currentReservation < required, we cannot allocate this
|
|
|
- // require
|
|
|
- Resource resourceCouldBeUnReserved = getCurrentReservation();
|
|
|
- if (!getCSLeafQueue().getReservationContinueLooking() || !node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) {
|
|
|
- // If we don't allow reservation continuous looking, OR we're looking at
|
|
|
- // non-default node partition, we won't allow to unreserve before
|
|
|
- // allocation.
|
|
|
- resourceCouldBeUnReserved = Resources.none();
|
|
|
- }
|
|
|
- return Resources
|
|
|
- .greaterThanOrEqual(rc, clusterResource, Resources.add(
|
|
|
- currentResourceLimits.getHeadroom(), resourceCouldBeUnReserved),
|
|
|
- required);
|
|
|
- }
|
|
|
-
|
|
|
+
|
|
|
public CSAssignment assignContainers(Resource clusterResource,
|
|
|
FiCaSchedulerNode node, ResourceLimits currentResourceLimits,
|
|
|
SchedulingMode schedulingMode) {
|
|
@@ -886,174 +507,41 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|
|
+ ", because it doesn't need more resource, schedulingMode="
|
|
|
+ schedulingMode.name() + " node-label=" + node.getPartition());
|
|
|
}
|
|
|
- return SKIP_ASSIGNMENT;
|
|
|
+ return CSAssignment.SKIP_ASSIGNMENT;
|
|
|
}
|
|
|
|
|
|
synchronized (this) {
|
|
|
- // Check if this resource is on the blacklist
|
|
|
- if (SchedulerAppUtils.isBlacklisted(this, node, LOG)) {
|
|
|
- return SKIP_ASSIGNMENT;
|
|
|
- }
|
|
|
-
|
|
|
// Schedule in priority order
|
|
|
for (Priority priority : getPriorities()) {
|
|
|
- ResourceRequest anyRequest =
|
|
|
- getResourceRequest(priority, ResourceRequest.ANY);
|
|
|
- if (null == anyRequest) {
|
|
|
- continue;
|
|
|
- }
|
|
|
-
|
|
|
- // Required resource
|
|
|
- Resource required = anyRequest.getCapability();
|
|
|
-
|
|
|
- // Do we need containers at this 'priority'?
|
|
|
- if (getTotalRequiredResources(priority) <= 0) {
|
|
|
- continue;
|
|
|
- }
|
|
|
-
|
|
|
- // AM container allocation doesn't support non-exclusive allocation to
|
|
|
- // avoid painful of preempt an AM container
|
|
|
- if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
|
|
|
+ ContainerAllocation allocationResult =
|
|
|
+ containerAllocator.allocate(clusterResource, node,
|
|
|
+ schedulingMode, currentResourceLimits, priority, null);
|
|
|
|
|
|
- RMAppAttempt rmAppAttempt =
|
|
|
- rmContext.getRMApps()
|
|
|
- .get(getApplicationId()).getCurrentAppAttempt();
|
|
|
- if (rmAppAttempt.getSubmissionContext().getUnmanagedAM() == false
|
|
|
- && null == rmAppAttempt.getMasterContainer()) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Skip allocating AM container to app_attempt="
|
|
|
- + getApplicationAttemptId()
|
|
|
- + ", don't allow to allocate AM container in non-exclusive mode");
|
|
|
- }
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
+ // If it's a skipped allocation
|
|
|
+ AllocationState allocationState = allocationResult.getAllocationState();
|
|
|
|
|
|
- // Is the node-label-expression of this offswitch resource request
|
|
|
- // matches the node's label?
|
|
|
- // If not match, jump to next priority.
|
|
|
- if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
|
|
|
- anyRequest, node.getPartition(), schedulingMode)) {
|
|
|
+ if (allocationState == AllocationState.PRIORITY_SKIPPED) {
|
|
|
continue;
|
|
|
}
|
|
|
-
|
|
|
- if (!getCSLeafQueue().getReservationContinueLooking()) {
|
|
|
- if (!shouldAllocOrReserveNewContainer(priority, required)) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("doesn't need containers based on reservation algo!");
|
|
|
- }
|
|
|
- continue;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- if (!checkHeadroom(clusterResource, currentResourceLimits, required,
|
|
|
- node)) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("cannot allocate required resource=" + required
|
|
|
- + " because of headroom");
|
|
|
- }
|
|
|
- return NULL_ASSIGNMENT;
|
|
|
- }
|
|
|
-
|
|
|
- // Inform the application it is about to get a scheduling opportunity
|
|
|
- addSchedulingOpportunity(priority);
|
|
|
-
|
|
|
- // Increase missed-non-partitioned-resource-request-opportunity.
|
|
|
- // This is to make sure non-partitioned-resource-request will prefer
|
|
|
- // to be allocated to non-partitioned nodes
|
|
|
- int missedNonPartitionedRequestSchedulingOpportunity = 0;
|
|
|
- if (anyRequest.getNodeLabelExpression().equals(
|
|
|
- RMNodeLabelsManager.NO_LABEL)) {
|
|
|
- missedNonPartitionedRequestSchedulingOpportunity =
|
|
|
- addMissedNonPartitionedRequestSchedulingOpportunity(priority);
|
|
|
- }
|
|
|
-
|
|
|
- if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
|
|
|
- // Before doing allocation, we need to check scheduling opportunity to
|
|
|
- // make sure : non-partitioned resource request should be scheduled to
|
|
|
- // non-partitioned partition first.
|
|
|
- if (missedNonPartitionedRequestSchedulingOpportunity < rmContext
|
|
|
- .getScheduler().getNumClusterNodes()) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Skip app_attempt="
|
|
|
- + getApplicationAttemptId() + " priority="
|
|
|
- + priority
|
|
|
- + " because missed-non-partitioned-resource-request"
|
|
|
- + " opportunity under requred:" + " Now="
|
|
|
- + missedNonPartitionedRequestSchedulingOpportunity
|
|
|
- + " required="
|
|
|
- + rmContext.getScheduler().getNumClusterNodes());
|
|
|
- }
|
|
|
-
|
|
|
- return SKIP_ASSIGNMENT;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Try to schedule
|
|
|
- CSAssignment assignment =
|
|
|
- assignContainersOnNode(clusterResource, node,
|
|
|
- priority, null, schedulingMode, currentResourceLimits);
|
|
|
-
|
|
|
- // Did the application skip this node?
|
|
|
- if (assignment.getSkipped()) {
|
|
|
- // Don't count 'skipped nodes' as a scheduling opportunity!
|
|
|
- subtractSchedulingOpportunity(priority);
|
|
|
- continue;
|
|
|
- }
|
|
|
-
|
|
|
- // Did we schedule or reserve a container?
|
|
|
- Resource assigned = assignment.getResource();
|
|
|
- if (Resources.greaterThan(rc, clusterResource,
|
|
|
- assigned, Resources.none())) {
|
|
|
- // Don't reset scheduling opportunities for offswitch assignments
|
|
|
- // otherwise the app will be delayed for each non-local assignment.
|
|
|
- // This helps apps with many off-cluster requests schedule faster.
|
|
|
- if (assignment.getType() != NodeType.OFF_SWITCH) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Resetting scheduling opportunities");
|
|
|
- }
|
|
|
- resetSchedulingOpportunities(priority);
|
|
|
- }
|
|
|
- // Non-exclusive scheduling opportunity is different: we need reset
|
|
|
- // it every time to make sure non-labeled resource request will be
|
|
|
- // most likely allocated on non-labeled nodes first.
|
|
|
- resetMissedNonPartitionedRequestSchedulingOpportunity(priority);
|
|
|
-
|
|
|
- // Done
|
|
|
- return assignment;
|
|
|
- } else {
|
|
|
- // Do not assign out of order w.r.t priorities
|
|
|
- return SKIP_ASSIGNMENT;
|
|
|
- }
|
|
|
+ return getCSAssignmentFromAllocateResult(clusterResource,
|
|
|
+ allocationResult);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- return SKIP_ASSIGNMENT;
|
|
|
+ // We will reach here if we skipped all priorities of the app, so we will
|
|
|
+ // skip the app.
|
|
|
+ return CSAssignment.SKIP_ASSIGNMENT;
|
|
|
}
|
|
|
|
|
|
|
|
|
public synchronized CSAssignment assignReservedContainer(
|
|
|
FiCaSchedulerNode node, RMContainer rmContainer,
|
|
|
Resource clusterResource, SchedulingMode schedulingMode) {
|
|
|
- // Do we still need this reservation?
|
|
|
- Priority priority = rmContainer.getReservedPriority();
|
|
|
- if (getTotalRequiredResources(priority) == 0) {
|
|
|
- // Release
|
|
|
- return new CSAssignment(this, rmContainer);
|
|
|
- }
|
|
|
+ ContainerAllocation result =
|
|
|
+ containerAllocator.allocate(clusterResource, node,
|
|
|
+ schedulingMode, new ResourceLimits(Resources.none()),
|
|
|
+ rmContainer.getReservedPriority(), rmContainer);
|
|
|
|
|
|
- // Try to assign if we have sufficient resources
|
|
|
- CSAssignment tmp =
|
|
|
- assignContainersOnNode(clusterResource, node, priority,
|
|
|
- rmContainer, schedulingMode, new ResourceLimits(Resources.none()));
|
|
|
-
|
|
|
- // Doesn't matter... since it's already charged for at time of reservation
|
|
|
- // "re-reservation" is *free*
|
|
|
- CSAssignment ret = new CSAssignment(Resources.none(), NodeType.NODE_LOCAL);
|
|
|
- if (tmp.getAssignmentInformation().getNumAllocations() > 0) {
|
|
|
- ret.setFulfilledReservation(true);
|
|
|
- }
|
|
|
- return ret;
|
|
|
+ return getCSAssignmentFromAllocateResult(clusterResource, result);
|
|
|
}
|
|
|
-
|
|
|
}
|