|
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
|
|
|
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.SchedulingMode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
|
@@ -154,7 +155,6 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
ContainerAllocation preAllocation(Resource clusterResource,
|
|
|
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
|
|
ResourceLimits resourceLimits, Priority priority,
|
|
@@ -295,14 +295,14 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
schedulingMode, currentResoureLimits);
|
|
|
}
|
|
|
|
|
|
- return ContainerAllocation.QUEUE_SKIPPED;
|
|
|
+ return ContainerAllocation.APP_SKIPPED;
|
|
|
}
|
|
|
|
|
|
private ContainerAllocation assignContainersOnNode(Resource clusterResource,
|
|
|
FiCaSchedulerNode node, Priority priority, RMContainer reservedContainer,
|
|
|
SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
|
|
|
|
|
|
- ContainerAllocation assigned;
|
|
|
+ ContainerAllocation allocation;
|
|
|
|
|
|
NodeType requestType = null;
|
|
|
// Data-local
|
|
@@ -310,14 +310,14 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
application.getResourceRequest(priority, node.getNodeName());
|
|
|
if (nodeLocalResourceRequest != null) {
|
|
|
requestType = NodeType.NODE_LOCAL;
|
|
|
- assigned =
|
|
|
+ allocation =
|
|
|
assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest,
|
|
|
node, priority, reservedContainer, schedulingMode,
|
|
|
currentResoureLimits);
|
|
|
if (Resources.greaterThan(rc, clusterResource,
|
|
|
- assigned.getResourceToBeAllocated(), Resources.none())) {
|
|
|
- assigned.requestNodeType = requestType;
|
|
|
- return assigned;
|
|
|
+ allocation.getResourceToBeAllocated(), Resources.none())) {
|
|
|
+ allocation.requestNodeType = requestType;
|
|
|
+ return allocation;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -333,14 +333,14 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
requestType = NodeType.RACK_LOCAL;
|
|
|
}
|
|
|
|
|
|
- assigned =
|
|
|
+ allocation =
|
|
|
assignRackLocalContainers(clusterResource, rackLocalResourceRequest,
|
|
|
node, priority, reservedContainer, schedulingMode,
|
|
|
currentResoureLimits);
|
|
|
if (Resources.greaterThan(rc, clusterResource,
|
|
|
- assigned.getResourceToBeAllocated(), Resources.none())) {
|
|
|
- assigned.requestNodeType = requestType;
|
|
|
- return assigned;
|
|
|
+ allocation.getResourceToBeAllocated(), Resources.none())) {
|
|
|
+ allocation.requestNodeType = requestType;
|
|
|
+ return allocation;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -356,13 +356,19 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
requestType = NodeType.OFF_SWITCH;
|
|
|
}
|
|
|
|
|
|
- assigned =
|
|
|
+ allocation =
|
|
|
assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
|
|
|
node, priority, reservedContainer, schedulingMode,
|
|
|
currentResoureLimits);
|
|
|
- assigned.requestNodeType = requestType;
|
|
|
+ allocation.requestNodeType = requestType;
|
|
|
+
|
|
|
+ // When a returned allocation is LOCALITY_SKIPPED, since we're in
|
|
|
+ // off-switch request now, we will skip this app w.r.t priorities
|
|
|
+ if (allocation.state == AllocationState.LOCALITY_SKIPPED) {
|
|
|
+ allocation.state = AllocationState.APP_SKIPPED;
|
|
|
+ }
|
|
|
|
|
|
- return assigned;
|
|
|
+ return allocation;
|
|
|
}
|
|
|
|
|
|
return ContainerAllocation.PRIORITY_SKIPPED;
|
|
@@ -388,7 +394,7 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
// to label not match. This can be caused by node label changed
|
|
|
// We should un-reserve this container.
|
|
|
return new ContainerAllocation(rmContainer, null,
|
|
|
- AllocationState.QUEUE_SKIPPED);
|
|
|
+ AllocationState.LOCALITY_SKIPPED);
|
|
|
}
|
|
|
|
|
|
Resource capability = request.getCapability();
|
|
@@ -400,7 +406,8 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
LOG.warn("Node : " + node.getNodeID()
|
|
|
+ " does not have sufficient resource for request : " + request
|
|
|
+ " node total capability : " + node.getTotalResource());
|
|
|
- return ContainerAllocation.QUEUE_SKIPPED;
|
|
|
+ // Skip this locality request
|
|
|
+ return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
|
|
|
|
assert Resources.greaterThan(
|
|
@@ -457,7 +464,8 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
// continue)). If we failed to unreserve some resource, we can't
|
|
|
// continue.
|
|
|
if (null == unreservedContainer) {
|
|
|
- return ContainerAllocation.QUEUE_SKIPPED;
|
|
|
+ // Skip the locality request
|
|
|
+ return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -468,19 +476,20 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
result.containerNodeType = type;
|
|
|
return result;
|
|
|
} 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 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.
|
|
|
+ // 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 ContainerAllocation.QUEUE_SKIPPED;
|
|
|
+ // Skip the locality request
|
|
|
+ return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -490,7 +499,8 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
result.containerNodeType = type;
|
|
|
return result;
|
|
|
}
|
|
|
- return ContainerAllocation.QUEUE_SKIPPED;
|
|
|
+ // Skip the locality request
|
|
|
+ return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -563,8 +573,7 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
// Skip this app if we failed to allocate.
|
|
|
ContainerAllocation ret =
|
|
|
new ContainerAllocation(allocationResult.containerToBeUnreserved,
|
|
|
- null, AllocationState.QUEUE_SKIPPED);
|
|
|
- ret.state = AllocationState.APP_SKIPPED;
|
|
|
+ null, AllocationState.APP_SKIPPED);
|
|
|
return ret;
|
|
|
}
|
|
|
|
|
@@ -578,7 +587,6 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
return allocationResult;
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
ContainerAllocation doAllocation(ContainerAllocation allocationResult,
|
|
|
Resource clusterResource, FiCaSchedulerNode node,
|
|
|
SchedulingMode schedulingMode, Priority priority,
|
|
@@ -591,7 +599,7 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
// something went wrong getting/creating the container
|
|
|
if (container == null) {
|
|
|
LOG.warn("Couldn't get container for allocation!");
|
|
|
- return ContainerAllocation.QUEUE_SKIPPED;
|
|
|
+ return ContainerAllocation.APP_SKIPPED;
|
|
|
}
|
|
|
|
|
|
if (allocationResult.getAllocationState() == AllocationState.ALLOCATED) {
|
|
@@ -626,4 +634,65 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|
|
|
|
|
return allocationResult;
|
|
|
}
|
|
|
+
|
|
|
+ private ContainerAllocation allocate(Resource clusterResource,
|
|
|
+ FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
|
|
+ ResourceLimits resourceLimits, Priority priority,
|
|
|
+ RMContainer reservedContainer) {
|
|
|
+ ContainerAllocation result =
|
|
|
+ preAllocation(clusterResource, node, schedulingMode, resourceLimits,
|
|
|
+ priority, reservedContainer);
|
|
|
+
|
|
|
+ if (AllocationState.ALLOCATED == result.state
|
|
|
+ || AllocationState.RESERVED == result.state) {
|
|
|
+ result =
|
|
|
+ doAllocation(result, clusterResource, node, schedulingMode, priority,
|
|
|
+ reservedContainer);
|
|
|
+ }
|
|
|
+
|
|
|
+ return result;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public CSAssignment assignContainers(Resource clusterResource,
|
|
|
+ FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
|
|
+ ResourceLimits resourceLimits,
|
|
|
+ RMContainer reservedContainer) {
|
|
|
+ if (reservedContainer == null) {
|
|
|
+ // Check if application needs more resource, skip if it doesn't need more.
|
|
|
+ if (!application.hasPendingResourceRequest(rc,
|
|
|
+ node.getPartition(), clusterResource, schedulingMode)) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Skip app_attempt=" + application.getApplicationAttemptId()
|
|
|
+ + ", because it doesn't need more resource, schedulingMode="
|
|
|
+ + schedulingMode.name() + " node-label=" + node.getPartition());
|
|
|
+ }
|
|
|
+ return CSAssignment.SKIP_ASSIGNMENT;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Schedule in priority order
|
|
|
+ for (Priority priority : application.getPriorities()) {
|
|
|
+ ContainerAllocation result =
|
|
|
+ allocate(clusterResource, node, schedulingMode, resourceLimits,
|
|
|
+ priority, null);
|
|
|
+
|
|
|
+ AllocationState allocationState = result.getAllocationState();
|
|
|
+ if (allocationState == AllocationState.PRIORITY_SKIPPED) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ return getCSAssignmentFromAllocateResult(clusterResource, result,
|
|
|
+ null);
|
|
|
+ }
|
|
|
+
|
|
|
+ // We will reach here if we skipped all priorities of the app, so we will
|
|
|
+ // skip the app.
|
|
|
+ return CSAssignment.SKIP_ASSIGNMENT;
|
|
|
+ } else {
|
|
|
+ ContainerAllocation result =
|
|
|
+ allocate(clusterResource, node, schedulingMode, resourceLimits,
|
|
|
+ reservedContainer.getReservedPriority(), reservedContainer);
|
|
|
+ return getCSAssignmentFromAllocateResult(clusterResource, result,
|
|
|
+ reservedContainer);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|