|
@@ -21,13 +21,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocat
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
+import java.util.Optional;
|
|
|
|
|
|
import org.apache.commons.lang3.StringUtils;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.DiagnosticsCollector;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
-import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
@@ -100,14 +101,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
Resource clusterResource, FiCaSchedulerNode node,
|
|
|
SchedulingMode schedulingMode, ResourceLimits resourceLimits,
|
|
|
SchedulerRequestKey schedulerKey) {
|
|
|
- Priority priority = schedulerKey.getPriority();
|
|
|
-
|
|
|
PendingAsk offswitchPendingAsk = application.getPendingAsk(schedulerKey,
|
|
|
ResourceRequest.ANY);
|
|
|
|
|
|
if (offswitchPendingAsk.getCount() <= 0) {
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.PRIORITY_SKIPPED_BECAUSE_NULL_ANY_REQUEST);
|
|
|
return ContainerAllocation.PRIORITY_SKIPPED;
|
|
|
}
|
|
@@ -118,7 +117,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
// Do we need containers at this 'priority'?
|
|
|
if (application.getOutstandingAsksCount(schedulerKey) <= 0) {
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.APPLICATION_PRIORITY_DO_NOT_NEED_RESOURCE);
|
|
|
return ContainerAllocation.PRIORITY_SKIPPED;
|
|
|
}
|
|
@@ -133,7 +132,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
application.updateAppSkipNodeDiagnostics(
|
|
|
"Skipping assigning to Node in Ignore Exclusivity mode. ");
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.SKIP_IN_IGNORE_EXCLUSIVITY_MODE);
|
|
|
return ContainerAllocation.APP_SKIPPED;
|
|
|
}
|
|
@@ -141,11 +140,15 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
|
|
|
// Is the nodePartition of pending request matches the node's partition
|
|
|
// If not match, jump to next priority.
|
|
|
- if (!appInfo.precheckNode(schedulerKey, node, schedulingMode)) {
|
|
|
+ Optional<DiagnosticsCollector> dcOpt = activitiesManager == null ?
|
|
|
+ Optional.empty() :
|
|
|
+ activitiesManager.getOptionalDiagnosticsCollector();
|
|
|
+ if (!appInfo.precheckNode(schedulerKey, node, schedulingMode, dcOpt)) {
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.
|
|
|
- PRIORITY_SKIPPED_BECAUSE_NODE_PARTITION_DOES_NOT_MATCH_REQUEST);
|
|
|
+ NODE_DO_NOT_MATCH_PARTITION_OR_PLACEMENT_CONSTRAINTS
|
|
|
+ + ActivitiesManager.getDiagnostics(dcOpt));
|
|
|
return ContainerAllocation.PRIORITY_SKIPPED;
|
|
|
}
|
|
|
|
|
@@ -153,7 +156,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
if (!shouldAllocOrReserveNewContainer(schedulerKey, required)) {
|
|
|
LOG.debug("doesn't need containers based on reservation algo!");
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.DO_NOT_NEED_ALLOCATIONATTEMPTINFOS);
|
|
|
return ContainerAllocation.PRIORITY_SKIPPED;
|
|
|
}
|
|
@@ -164,7 +167,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
LOG.debug("cannot allocate required resource={} because of headroom",
|
|
|
required);
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.QUEUE_SKIPPED_HEADROOM);
|
|
|
return ContainerAllocation.QUEUE_SKIPPED;
|
|
|
}
|
|
@@ -179,7 +182,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
// This is possible when #pending resource decreased by a different
|
|
|
// thread.
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.PRIORITY_SKIPPED_BECAUSE_NULL_ANY_REQUEST);
|
|
|
return ContainerAllocation.PRIORITY_SKIPPED;
|
|
|
}
|
|
@@ -209,7 +212,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
+ rmContext.getScheduler().getNumClusterNodes());
|
|
|
}
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.NON_PARTITIONED_PARTITION_FIRST);
|
|
|
return ContainerAllocation.APP_SKIPPED;
|
|
|
}
|
|
@@ -220,13 +223,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
|
|
|
private ContainerAllocation checkIfNodeBlackListed(FiCaSchedulerNode node,
|
|
|
SchedulerRequestKey schedulerKey) {
|
|
|
- Priority priority = schedulerKey.getPriority();
|
|
|
-
|
|
|
if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) {
|
|
|
application.updateAppSkipNodeDiagnostics(
|
|
|
CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE);
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.SKIP_BLACK_LISTED_NODE);
|
|
|
return ContainerAllocation.APP_SKIPPED;
|
|
|
}
|
|
@@ -366,7 +367,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
|
|
|
// Skip node-local request, go to rack-local request
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, schedulerKey.getPriority(),
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.SKIP_NODE_LOCAL_REQUEST);
|
|
|
return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
@@ -384,7 +385,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
|
|
|
// Skip rack-local request, go to off-switch request
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, schedulerKey.getPriority(),
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.SKIP_RACK_LOCAL_REQUEST);
|
|
|
return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
@@ -403,7 +404,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
application.updateAppSkipNodeDiagnostics(
|
|
|
CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_DUE_TO_LOCALITY);
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, schedulerKey.getPriority(),
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.SKIP_OFF_SWITCH_REQUEST);
|
|
|
return ContainerAllocation.APP_SKIPPED;
|
|
|
}
|
|
@@ -412,8 +413,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
|
|
RMContainer reservedContainer, SchedulingMode schedulingMode,
|
|
|
ResourceLimits currentResoureLimits) {
|
|
|
- Priority priority = schedulerKey.getPriority();
|
|
|
-
|
|
|
ContainerAllocation allocation;
|
|
|
NodeType requestLocalityType = null;
|
|
|
|
|
@@ -439,7 +438,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
if (rackLocalAsk.getCount() > 0) {
|
|
|
if (!appInfo.canDelayTo(schedulerKey, node.getRackName())) {
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY);
|
|
|
return ContainerAllocation.PRIORITY_SKIPPED;
|
|
|
}
|
|
@@ -465,7 +464,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
if (offSwitchAsk.getCount() > 0) {
|
|
|
if (!appInfo.canDelayTo(schedulerKey, ResourceRequest.ANY)) {
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY);
|
|
|
return ContainerAllocation.PRIORITY_SKIPPED;
|
|
|
}
|
|
@@ -489,7 +488,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
return allocation;
|
|
|
}
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.PRIORITY_SKIPPED);
|
|
|
return ContainerAllocation.PRIORITY_SKIPPED;
|
|
|
}
|
|
@@ -498,7 +497,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
|
|
PendingAsk pendingAsk, NodeType type, RMContainer rmContainer,
|
|
|
SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
|
|
|
- Priority priority = schedulerKey.getPriority();
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("assignContainers: node=" + node.getNodeName()
|
|
@@ -511,15 +509,15 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
Resource available = node.getUnallocatedResource();
|
|
|
Resource totalResource = node.getTotalResource();
|
|
|
|
|
|
- if (!Resources.lessThanOrEqual(rc, clusterResource,
|
|
|
- capability, totalResource)) {
|
|
|
+ if (!Resources.fitsIn(rc, capability, totalResource)) {
|
|
|
LOG.warn("Node : " + node.getNodeID()
|
|
|
+ " does not have sufficient resource for ask : " + pendingAsk
|
|
|
+ " node total capability : " + node.getTotalResource());
|
|
|
// Skip this locality request
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
- ActivityDiagnosticConstant.NOT_SUFFICIENT_RESOURCE);
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
+ ActivityDiagnosticConstant.NOT_SUFFICIENT_RESOURCE
|
|
|
+ + getResourceDiagnostics(capability, totalResource));
|
|
|
return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
|
|
|
@@ -529,6 +527,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
// Can we allocate a container on this node?
|
|
|
long availableContainers =
|
|
|
rc.computeAvailableContainers(available, capability);
|
|
|
+ // available resource for diagnostics collector
|
|
|
+ Resource availableForDC = available;
|
|
|
|
|
|
// How much need to unreserve equals to:
|
|
|
// max(required - headroom, amountNeedUnreserve)
|
|
@@ -562,6 +562,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
+ availableForDC = availableAndKillable;
|
|
|
}
|
|
|
|
|
|
if (availableContainers > 0) {
|
|
@@ -594,8 +595,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
if (null == unreservedContainer) {
|
|
|
// Skip the locality request
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
- ActivityDiagnosticConstant.LOCALITY_SKIPPED);
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
+ ActivityDiagnosticConstant.
|
|
|
+ NODE_CAN_NOT_FIND_CONTAINER_TO_BE_UNRESERVED_WHEN_NEEDED);
|
|
|
return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
|
}
|
|
@@ -619,8 +621,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
|
|
|
// Skip the locality request
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
- ActivityDiagnosticConstant.LOCALITY_SKIPPED);
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
+ ActivityDiagnosticConstant.NOT_SUFFICIENT_RESOURCE
|
|
|
+ + getResourceDiagnostics(capability, availableForDC));
|
|
|
return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
|
}
|
|
@@ -633,8 +636,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
}
|
|
|
// Skip the locality request
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, priority,
|
|
|
- ActivityDiagnosticConstant.LOCALITY_SKIPPED);
|
|
|
+ activitiesManager, node, application, schedulerKey,
|
|
|
+ ActivityDiagnosticConstant.NOT_SUFFICIENT_RESOURCE
|
|
|
+ + getResourceDiagnostics(capability, availableForDC));
|
|
|
return ContainerAllocation.LOCALITY_SKIPPED;
|
|
|
}
|
|
|
}
|
|
@@ -708,7 +712,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
new ContainerAllocation(allocationResult.containerToBeUnreserved,
|
|
|
null, AllocationState.APP_SKIPPED);
|
|
|
ActivitiesLogger.APP.recordAppActivityWithoutAllocation(activitiesManager,
|
|
|
- node, application, schedulerKey.getPriority(),
|
|
|
+ node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.FAIL_TO_ALLOCATE, ActivityState.REJECTED);
|
|
|
return ret;
|
|
|
}
|
|
@@ -730,7 +734,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
.updateAppSkipNodeDiagnostics("Scheduling of container failed. ");
|
|
|
LOG.warn("Couldn't get container for allocation!");
|
|
|
ActivitiesLogger.APP.recordAppActivityWithoutAllocation(activitiesManager,
|
|
|
- node, application, schedulerKey.getPriority(),
|
|
|
+ node, application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.COULD_NOT_GET_CONTAINER,
|
|
|
ActivityState.REJECTED);
|
|
|
return ContainerAllocation.APP_SKIPPED;
|
|
@@ -753,7 +757,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
+ " schedulerRequestKey=" + schedulerKey);
|
|
|
ActivitiesLogger.APP
|
|
|
.recordAppActivityWithoutAllocation(activitiesManager, node,
|
|
|
- application, schedulerKey.getPriority(),
|
|
|
+ application, schedulerKey,
|
|
|
ActivityDiagnosticConstant.
|
|
|
PRIORITY_SKIPPED_BECAUSE_NULL_ANY_REQUEST,
|
|
|
ActivityState.REJECTED);
|
|
@@ -815,6 +819,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
|
|
|
// This could be null when #pending request decreased by another thread.
|
|
|
if (schedulingPS == null) {
|
|
|
+ ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
+ activitiesManager, null, application, schedulerKey,
|
|
|
+ ActivityDiagnosticConstant.
|
|
|
+ APPLICATION_PRIORITY_DO_NOT_NEED_RESOURCE);
|
|
|
return new ContainerAllocation(reservedContainer, null,
|
|
|
AllocationState.QUEUE_SKIPPED);
|
|
|
}
|
|
@@ -873,7 +881,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
.getPartition());
|
|
|
}
|
|
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, application.getPriority(),
|
|
|
+ activitiesManager, node, application, null,
|
|
|
ActivityDiagnosticConstant.APPLICATION_DO_NOT_NEED_RESOURCE);
|
|
|
return CSAssignment.SKIP_ASSIGNMENT;
|
|
|
}
|
|
@@ -893,9 +901,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
|
|
|
// We will reach here if we skipped all priorities of the app, so we will
|
|
|
// skip the app.
|
|
|
- ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
|
|
- activitiesManager, node, application, application.getPriority(),
|
|
|
- ActivityDiagnosticConstant.SKIPPED_ALL_PRIORITIES);
|
|
|
return CSAssignment.SKIP_ASSIGNMENT;
|
|
|
} else {
|
|
|
ContainerAllocation result =
|
|
@@ -905,4 +910,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|
|
reservedContainer, node);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private String getResourceDiagnostics(Resource required, Resource available) {
|
|
|
+ if (activitiesManager == null) {
|
|
|
+ return ActivitiesManager.EMPTY_DIAGNOSTICS;
|
|
|
+ }
|
|
|
+ return activitiesManager.getResourceDiagnostics(rc, required, available);
|
|
|
+ }
|
|
|
}
|