|
@@ -48,11 +48,11 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
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.api.records.YarnApplicationAttemptState;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
@@ -146,9 +146,14 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
// if an RMAppAttemptUnregistrationEvent occurs
|
|
|
private FinalApplicationStatus finalStatus = null;
|
|
|
private final StringBuilder diagnostics = new StringBuilder();
|
|
|
+ private int amContainerExitStatus = ContainerExitStatus.INVALID;
|
|
|
|
|
|
private Configuration conf;
|
|
|
- private final boolean isLastAttempt;
|
|
|
+ // Since AM preemption is not counted towards AM failure count,
|
|
|
+ // even if this flag is true, a new attempt can still be re-created if this
|
|
|
+ // attempt is eventually preempted. So this flag indicates that this may be
|
|
|
+ // last attempt.
|
|
|
+ private final boolean maybeLastAttempt;
|
|
|
private static final ExpiredTransition EXPIRED_TRANSITION =
|
|
|
new ExpiredTransition();
|
|
|
|
|
@@ -389,7 +394,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
RMContext rmContext, YarnScheduler scheduler,
|
|
|
ApplicationMasterService masterService,
|
|
|
ApplicationSubmissionContext submissionContext,
|
|
|
- Configuration conf, boolean isLastAttempt) {
|
|
|
+ Configuration conf, boolean maybeLastAttempt) {
|
|
|
this.conf = conf;
|
|
|
this.applicationAttemptId = appAttemptId;
|
|
|
this.rmContext = rmContext;
|
|
@@ -403,7 +408,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
this.writeLock = lock.writeLock();
|
|
|
|
|
|
this.proxiedTrackingUrl = generateProxyUriWithScheme(null);
|
|
|
- this.isLastAttempt = isLastAttempt;
|
|
|
+ this.maybeLastAttempt = maybeLastAttempt;
|
|
|
this.stateMachine = stateMachineFactory.make(this);
|
|
|
}
|
|
|
|
|
@@ -565,6 +570,15 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ public int getAMContainerExitStatus() {
|
|
|
+ this.readLock.lock();
|
|
|
+ try {
|
|
|
+ return this.amContainerExitStatus;
|
|
|
+ } finally {
|
|
|
+ this.readLock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public float getProgress() {
|
|
|
this.readLock.lock();
|
|
@@ -671,6 +685,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
+ attemptState.getState());
|
|
|
diagnostics.append("Attempt recovered after RM restart");
|
|
|
diagnostics.append(attemptState.getDiagnostics());
|
|
|
+ this.amContainerExitStatus = attemptState.getAMContainerExitStatus();
|
|
|
setMasterContainer(attemptState.getMasterContainer());
|
|
|
recoverAppAttemptCredentials(attemptState.getAppAttemptCredentials());
|
|
|
this.recoveredFinalState = attemptState.getState();
|
|
@@ -931,7 +946,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
String diags = null;
|
|
|
String finalTrackingUrl = null;
|
|
|
FinalApplicationStatus finalStatus = null;
|
|
|
-
|
|
|
+ int exitStatus = ContainerExitStatus.INVALID;
|
|
|
switch (event.getType()) {
|
|
|
case LAUNCH_FAILED:
|
|
|
RMAppAttemptLaunchFailedEvent launchFaileEvent =
|
|
@@ -952,6 +967,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
RMAppAttemptContainerFinishedEvent finishEvent =
|
|
|
(RMAppAttemptContainerFinishedEvent) event;
|
|
|
diags = getAMContainerCrashedDiagnostics(finishEvent);
|
|
|
+ exitStatus = finishEvent.getContainerStatus().getExitStatus();
|
|
|
break;
|
|
|
case KILL:
|
|
|
break;
|
|
@@ -966,9 +982,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
ApplicationAttemptState attemptState =
|
|
|
new ApplicationAttemptState(applicationAttemptId, getMasterContainer(),
|
|
|
rmStore.getCredentialsFromAppAttempt(this), startTime,
|
|
|
- stateToBeStored, finalTrackingUrl, diags, finalStatus);
|
|
|
+ stateToBeStored, finalTrackingUrl, diags, finalStatus, exitStatus);
|
|
|
LOG.info("Updating application attempt " + applicationAttemptId
|
|
|
- + " with final state: " + targetedFinalState);
|
|
|
+ + " with final state: " + targetedFinalState + ", and exit status: "
|
|
|
+ + exitStatus);
|
|
|
rmStore.updateApplicationAttemptState(attemptState);
|
|
|
}
|
|
|
|
|
@@ -1061,11 +1078,19 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
// don't leave the tracking URL pointing to a non-existent AM
|
|
|
appAttempt.setTrackingUrlToRMAppPage();
|
|
|
appAttempt.invalidateAMHostAndPort();
|
|
|
+
|
|
|
if (appAttempt.submissionContext
|
|
|
.getKeepContainersAcrossApplicationAttempts()
|
|
|
- && !appAttempt.isLastAttempt
|
|
|
&& !appAttempt.submissionContext.getUnmanagedAM()) {
|
|
|
- keepContainersAcrossAppAttempts = true;
|
|
|
+ // See if we should retain containers for non-unmanaged applications
|
|
|
+ if (appAttempt.isPreempted()) {
|
|
|
+ // Premption doesn't count towards app-failures and so we should
|
|
|
+ // retain containers.
|
|
|
+ keepContainersAcrossAppAttempts = true;
|
|
|
+ } else if (!appAttempt.maybeLastAttempt) {
|
|
|
+ // Not preemption. Not last-attempt too - keep containers.
|
|
|
+ keepContainersAcrossAppAttempts = true;
|
|
|
+ }
|
|
|
}
|
|
|
appEvent =
|
|
|
new RMAppFailedAttemptEvent(applicationId,
|
|
@@ -1105,7 +1130,12 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
appAttempt.getClientTokenMasterKey());
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean isPreempted() {
|
|
|
+ return getAMContainerExitStatus() == ContainerExitStatus.PREEMPTED;
|
|
|
+ }
|
|
|
+
|
|
|
private static final class UnmanagedAMAttemptSavedTransition
|
|
|
extends AMLaunchedTransition {
|
|
|
@Override
|
|
@@ -1208,14 +1238,22 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
appAttempt.rmContext.getAMLivelinessMonitor().unregister(
|
|
|
appAttempt.getAppAttemptId());
|
|
|
|
|
|
- // Setup diagnostic message
|
|
|
- appAttempt.diagnostics
|
|
|
- .append(getAMContainerCrashedDiagnostics(finishEvent));
|
|
|
+ // Setup diagnostic message and exit status
|
|
|
+ appAttempt.setAMContainerCrashedDiagnosticsAndExitStatus(finishEvent);
|
|
|
+
|
|
|
// Tell the app, scheduler
|
|
|
super.transition(appAttempt, finishEvent);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void setAMContainerCrashedDiagnosticsAndExitStatus(
|
|
|
+ RMAppAttemptContainerFinishedEvent finishEvent) {
|
|
|
+ ContainerStatus status = finishEvent.getContainerStatus();
|
|
|
+ String diagnostics = getAMContainerCrashedDiagnostics(finishEvent);
|
|
|
+ this.diagnostics.append(diagnostics);
|
|
|
+ this.amContainerExitStatus = status.getExitStatus();
|
|
|
+ }
|
|
|
+
|
|
|
private static String getAMContainerCrashedDiagnostics(
|
|
|
RMAppAttemptContainerFinishedEvent finishEvent) {
|
|
|
ContainerStatus status = finishEvent.getContainerStatus();
|
|
@@ -1437,13 +1475,12 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
@Override
|
|
|
public void
|
|
|
transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
|
|
|
- RMAppAttemptContainerFinishedEvent containerFinishedEvent =
|
|
|
+ RMAppAttemptContainerFinishedEvent finishEvent =
|
|
|
(RMAppAttemptContainerFinishedEvent) event;
|
|
|
// container associated with AM. must not be unmanaged
|
|
|
assert appAttempt.submissionContext.getUnmanagedAM() == false;
|
|
|
- // Setup diagnostic message
|
|
|
- appAttempt.diagnostics
|
|
|
- .append(getAMContainerCrashedDiagnostics(containerFinishedEvent));
|
|
|
+ // Setup diagnostic message and exit status
|
|
|
+ appAttempt.setAMContainerCrashedDiagnosticsAndExitStatus(finishEvent);
|
|
|
new FinalTransition(RMAppAttemptState.FAILED).transition(appAttempt,
|
|
|
event);
|
|
|
}
|
|
@@ -1644,4 +1681,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|
|
}
|
|
|
return attemptReport;
|
|
|
}
|
|
|
+
|
|
|
+ // for testing
|
|
|
+ public boolean mayBeLastAttempt() {
|
|
|
+ return maybeLastAttempt;
|
|
|
+ }
|
|
|
}
|