|
@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
@@ -74,7 +75,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistManager;
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistUpdates;
|
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.blacklist.DisabledBlacklistManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.blacklist.DisabledBlacklistManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
|
@@ -491,7 +491,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
ApplicationMasterService masterService,
|
|
ApplicationMasterService masterService,
|
|
ApplicationSubmissionContext submissionContext,
|
|
ApplicationSubmissionContext submissionContext,
|
|
Configuration conf, boolean maybeLastAttempt, ResourceRequest amReq,
|
|
Configuration conf, boolean maybeLastAttempt, ResourceRequest amReq,
|
|
- BlacklistManager amBlacklist) {
|
|
|
|
|
|
+ BlacklistManager amBlacklistManager) {
|
|
this.conf = conf;
|
|
this.conf = conf;
|
|
this.applicationAttemptId = appAttemptId;
|
|
this.applicationAttemptId = appAttemptId;
|
|
this.rmContext = rmContext;
|
|
this.rmContext = rmContext;
|
|
@@ -510,9 +510,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
|
|
|
this.attemptMetrics =
|
|
this.attemptMetrics =
|
|
new RMAppAttemptMetrics(applicationAttemptId, rmContext);
|
|
new RMAppAttemptMetrics(applicationAttemptId, rmContext);
|
|
-
|
|
|
|
|
|
+
|
|
this.amReq = amReq;
|
|
this.amReq = amReq;
|
|
- this.blacklistedNodesForAM = amBlacklist;
|
|
|
|
|
|
+ this.blacklistedNodesForAM = amBlacklistManager;
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1033,15 +1033,15 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
appAttempt.amReq.setResourceName(ResourceRequest.ANY);
|
|
appAttempt.amReq.setResourceName(ResourceRequest.ANY);
|
|
appAttempt.amReq.setRelaxLocality(true);
|
|
appAttempt.amReq.setRelaxLocality(true);
|
|
|
|
|
|
- appAttempt.getAMBlacklist().refreshNodeHostCount(
|
|
|
|
|
|
+ appAttempt.getAMBlacklistManager().refreshNodeHostCount(
|
|
appAttempt.scheduler.getNumClusterNodes());
|
|
appAttempt.scheduler.getNumClusterNodes());
|
|
|
|
|
|
- BlacklistUpdates amBlacklist = appAttempt.getAMBlacklist()
|
|
|
|
- .getBlacklistUpdates();
|
|
|
|
|
|
+ ResourceBlacklistRequest amBlacklist =
|
|
|
|
+ appAttempt.getAMBlacklistManager().getBlacklistUpdates();
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Using blacklist for AM: additions(" +
|
|
LOG.debug("Using blacklist for AM: additions(" +
|
|
- amBlacklist.getAdditions() + ") and removals(" +
|
|
|
|
- amBlacklist.getRemovals() + ")");
|
|
|
|
|
|
+ amBlacklist.getBlacklistAdditions() + ") and removals(" +
|
|
|
|
+ amBlacklist.getBlacklistRemovals() + ")");
|
|
}
|
|
}
|
|
// AM resource has been checked when submission
|
|
// AM resource has been checked when submission
|
|
Allocation amContainerAllocation =
|
|
Allocation amContainerAllocation =
|
|
@@ -1049,8 +1049,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
appAttempt.applicationAttemptId,
|
|
appAttempt.applicationAttemptId,
|
|
Collections.singletonList(appAttempt.amReq),
|
|
Collections.singletonList(appAttempt.amReq),
|
|
EMPTY_CONTAINER_RELEASE_LIST,
|
|
EMPTY_CONTAINER_RELEASE_LIST,
|
|
- amBlacklist.getAdditions(),
|
|
|
|
- amBlacklist.getRemovals(), null, null);
|
|
|
|
|
|
+ amBlacklist.getBlacklistAdditions(),
|
|
|
|
+ amBlacklist.getBlacklistRemovals(), null, null);
|
|
if (amContainerAllocation != null
|
|
if (amContainerAllocation != null
|
|
&& amContainerAllocation.getContainers() != null) {
|
|
&& amContainerAllocation.getContainers() != null) {
|
|
assert (amContainerAllocation.getContainers().size() == 0);
|
|
assert (amContainerAllocation.getContainers().size() == 0);
|
|
@@ -1483,9 +1483,36 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private boolean shouldCountTowardsNodeBlacklisting(int exitStatus) {
|
|
|
|
- return !(exitStatus == ContainerExitStatus.SUCCESS
|
|
|
|
- || exitStatus == ContainerExitStatus.PREEMPTED);
|
|
|
|
|
|
+ private static boolean shouldCountTowardsNodeBlacklisting(int exitStatus) {
|
|
|
|
+ switch (exitStatus) {
|
|
|
|
+ case ContainerExitStatus.PREEMPTED:
|
|
|
|
+ case ContainerExitStatus.KILLED_BY_RESOURCEMANAGER:
|
|
|
|
+ case ContainerExitStatus.KILLED_BY_APPMASTER:
|
|
|
|
+ case ContainerExitStatus.KILLED_AFTER_APP_COMPLETION:
|
|
|
|
+ case ContainerExitStatus.ABORTED:
|
|
|
|
+ // Neither the app's fault nor the system's fault. This happens by design,
|
|
|
|
+ // so no need for skipping nodes
|
|
|
|
+ return false;
|
|
|
|
+ case ContainerExitStatus.DISKS_FAILED:
|
|
|
|
+ // This container is marked with this exit-status means that the node is
|
|
|
|
+ // already marked as unhealthy given that most of the disks failed. So, no
|
|
|
|
+ // need for any explicit skipping of nodes.
|
|
|
|
+ return false;
|
|
|
|
+ case ContainerExitStatus.KILLED_EXCEEDED_VMEM:
|
|
|
|
+ case ContainerExitStatus.KILLED_EXCEEDED_PMEM:
|
|
|
|
+ // No point in skipping the node as it's not the system's fault
|
|
|
|
+ return false;
|
|
|
|
+ case ContainerExitStatus.SUCCESS:
|
|
|
|
+ return false;
|
|
|
|
+ case ContainerExitStatus.INVALID:
|
|
|
|
+ // Ideally, this shouldn't be considered for skipping a node. But in
|
|
|
|
+ // reality, it seems like there are cases where we are not setting
|
|
|
|
+ // exit-code correctly and so it's better to be conservative. See
|
|
|
|
+ // YARN-4284.
|
|
|
|
+ return true;
|
|
|
|
+ default:
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
private static final class UnmanagedAMAttemptSavedTransition
|
|
private static final class UnmanagedAMAttemptSavedTransition
|
|
@@ -1805,7 +1832,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
if (appAttempt.masterContainer != null
|
|
if (appAttempt.masterContainer != null
|
|
&& appAttempt.masterContainer.getId().equals(
|
|
&& appAttempt.masterContainer.getId().equals(
|
|
containerStatus.getContainerId())) {
|
|
containerStatus.getContainerId())) {
|
|
- appAttempt.sendAMContainerToNM(appAttempt, containerFinishedEvent);
|
|
|
|
|
|
+ appAttempt.amContainerFinished(appAttempt, containerFinishedEvent);
|
|
|
|
|
|
// Remember the follow up transition and save the final attempt state.
|
|
// Remember the follow up transition and save the final attempt state.
|
|
appAttempt.rememberTargetTransitionsAndStoreState(event,
|
|
appAttempt.rememberTargetTransitionsAndStoreState(event,
|
|
@@ -1851,13 +1878,17 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
|
|
|
// Add am container to the list so that am container instance will be
|
|
// Add am container to the list so that am container instance will be
|
|
// removed from NMContext.
|
|
// removed from NMContext.
|
|
- private void sendAMContainerToNM(RMAppAttemptImpl appAttempt,
|
|
|
|
|
|
+ private static void amContainerFinished(RMAppAttemptImpl appAttempt,
|
|
RMAppAttemptContainerFinishedEvent containerFinishedEvent) {
|
|
RMAppAttemptContainerFinishedEvent containerFinishedEvent) {
|
|
|
|
+
|
|
NodeId nodeId = containerFinishedEvent.getNodeId();
|
|
NodeId nodeId = containerFinishedEvent.getNodeId();
|
|
- if (containerFinishedEvent.getContainerStatus() != null) {
|
|
|
|
- if (shouldCountTowardsNodeBlacklisting(containerFinishedEvent
|
|
|
|
- .getContainerStatus().getExitStatus())) {
|
|
|
|
- appAttempt.addAMNodeToBlackList(containerFinishedEvent.getNodeId());
|
|
|
|
|
|
+
|
|
|
|
+ ContainerStatus containerStatus =
|
|
|
|
+ containerFinishedEvent.getContainerStatus();
|
|
|
|
+ if (containerStatus != null) {
|
|
|
|
+ int exitStatus = containerStatus.getExitStatus();
|
|
|
|
+ if (shouldCountTowardsNodeBlacklisting(exitStatus)) {
|
|
|
|
+ appAttempt.addAMNodeToBlackList(nodeId);
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
LOG.warn("No ContainerStatus in containerFinishedEvent");
|
|
LOG.warn("No ContainerStatus in containerFinishedEvent");
|
|
@@ -1865,14 +1896,13 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
|
|
|
if (!appAttempt.getSubmissionContext()
|
|
if (!appAttempt.getSubmissionContext()
|
|
.getKeepContainersAcrossApplicationAttempts()) {
|
|
.getKeepContainersAcrossApplicationAttempts()) {
|
|
- finishedContainersSentToAM.putIfAbsent(nodeId,
|
|
|
|
|
|
+ appAttempt.finishedContainersSentToAM.putIfAbsent(nodeId,
|
|
new ArrayList<ContainerStatus>());
|
|
new ArrayList<ContainerStatus>());
|
|
- appAttempt.finishedContainersSentToAM.get(nodeId).add(
|
|
|
|
- containerFinishedEvent.getContainerStatus());
|
|
|
|
|
|
+ appAttempt.finishedContainersSentToAM.get(nodeId).add(containerStatus);
|
|
appAttempt.sendFinishedContainersToNM();
|
|
appAttempt.sendFinishedContainersToNM();
|
|
} else {
|
|
} else {
|
|
appAttempt.sendFinishedAMContainerToNM(nodeId,
|
|
appAttempt.sendFinishedAMContainerToNM(nodeId,
|
|
- containerFinishedEvent.getContainerStatus().getContainerId());
|
|
|
|
|
|
+ containerStatus.getContainerId());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1887,7 +1917,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public BlacklistManager getAMBlacklist() {
|
|
|
|
|
|
+ public BlacklistManager getAMBlacklistManager() {
|
|
return blacklistedNodesForAM;
|
|
return blacklistedNodesForAM;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1946,7 +1976,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
containerStatus.getContainerId())) {
|
|
containerStatus.getContainerId())) {
|
|
new FinalTransition(RMAppAttemptState.FINISHED).transition(
|
|
new FinalTransition(RMAppAttemptState.FINISHED).transition(
|
|
appAttempt, containerFinishedEvent);
|
|
appAttempt, containerFinishedEvent);
|
|
- appAttempt.sendAMContainerToNM(appAttempt, containerFinishedEvent);
|
|
|
|
|
|
+ appAttempt.amContainerFinished(appAttempt, containerFinishedEvent);
|
|
return RMAppAttemptState.FINISHED;
|
|
return RMAppAttemptState.FINISHED;
|
|
}
|
|
}
|
|
// Add all finished containers so that they can be acked to NM.
|
|
// Add all finished containers so that they can be acked to NM.
|
|
@@ -1971,7 +2001,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
// Thus, we still return FINAL_SAVING state here.
|
|
// Thus, we still return FINAL_SAVING state here.
|
|
if (appAttempt.masterContainer.getId().equals(
|
|
if (appAttempt.masterContainer.getId().equals(
|
|
containerStatus.getContainerId())) {
|
|
containerStatus.getContainerId())) {
|
|
- appAttempt.sendAMContainerToNM(appAttempt, containerFinishedEvent);
|
|
|
|
|
|
+
|
|
|
|
+ appAttempt.amContainerFinished(appAttempt, containerFinishedEvent);
|
|
|
|
|
|
if (appAttempt.targetedFinalState.equals(RMAppAttemptState.FAILED)
|
|
if (appAttempt.targetedFinalState.equals(RMAppAttemptState.FAILED)
|
|
|| appAttempt.targetedFinalState.equals(RMAppAttemptState.KILLED)) {
|
|
|| appAttempt.targetedFinalState.equals(RMAppAttemptState.KILLED)) {
|