|
@@ -32,10 +32,10 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
-import org.apache.hadoop.util.ExitUtil;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
|
@@ -81,8 +81,12 @@ import org.apache.hadoop.yarn.state.MultipleArcTransition;
|
|
|
import org.apache.hadoop.yarn.state.SingleArcTransition;
|
|
|
import org.apache.hadoop.yarn.state.StateMachine;
|
|
|
import org.apache.hadoop.yarn.state.StateMachineFactory;
|
|
|
+import org.apache.hadoop.yarn.util.Clock;
|
|
|
+import org.apache.hadoop.yarn.util.SystemClock;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+
|
|
|
@SuppressWarnings({ "rawtypes", "unchecked" })
|
|
|
public class RMAppImpl implements RMApp, Recoverable {
|
|
|
|
|
@@ -110,6 +114,12 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
private final String applicationType;
|
|
|
private final Set<String> applicationTags;
|
|
|
|
|
|
+ private final long attemptFailuresValidityInterval;
|
|
|
+
|
|
|
+ private Clock systemClock;
|
|
|
+
|
|
|
+ private boolean isNumAttemptsBeyondThreshold = false;
|
|
|
+
|
|
|
// Mutable fields
|
|
|
private long startTime;
|
|
|
private long finishTime = 0;
|
|
@@ -328,6 +338,8 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
ApplicationMasterService masterService, long submitTime,
|
|
|
String applicationType, Set<String> applicationTags) {
|
|
|
|
|
|
+ this.systemClock = new SystemClock();
|
|
|
+
|
|
|
this.applicationId = applicationId;
|
|
|
this.name = name;
|
|
|
this.rmContext = rmContext;
|
|
@@ -340,7 +352,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
this.scheduler = scheduler;
|
|
|
this.masterService = masterService;
|
|
|
this.submitTime = submitTime;
|
|
|
- this.startTime = System.currentTimeMillis();
|
|
|
+ this.startTime = this.systemClock.getTime();
|
|
|
this.applicationType = applicationType;
|
|
|
this.applicationTags = applicationTags;
|
|
|
|
|
@@ -358,6 +370,9 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
this.maxAppAttempts = individualMaxAppAttempts;
|
|
|
}
|
|
|
|
|
|
+ this.attemptFailuresValidityInterval =
|
|
|
+ submissionContext.getAttemptFailuresValidityInterval();
|
|
|
+
|
|
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
|
this.readLock = lock.readLock();
|
|
|
this.writeLock = lock.writeLock();
|
|
@@ -897,7 +912,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
msg = "Unmanaged application " + this.getApplicationId()
|
|
|
+ " failed due to " + failedEvent.getDiagnostics()
|
|
|
+ ". Failing the application.";
|
|
|
- } else if (getNumFailedAppAttempts() >= this.maxAppAttempts) {
|
|
|
+ } else if (this.isNumAttemptsBeyondThreshold) {
|
|
|
msg = "Application " + this.getApplicationId() + " failed "
|
|
|
+ this.maxAppAttempts + " times due to "
|
|
|
+ failedEvent.getDiagnostics() + ". Failing the application.";
|
|
@@ -930,7 +945,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
RMAppState stateToBeStored) {
|
|
|
rememberTargetTransitions(event, transitionToDo, targetFinalState);
|
|
|
this.stateBeforeFinalSaving = getState();
|
|
|
- this.storedFinishTime = System.currentTimeMillis();
|
|
|
+ this.storedFinishTime = this.systemClock.getTime();
|
|
|
|
|
|
LOG.info("Updating application " + this.applicationId
|
|
|
+ " with final state: " + this.targetedFinalState);
|
|
@@ -1097,7 +1112,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
}
|
|
|
app.finishTime = app.storedFinishTime;
|
|
|
if (app.finishTime == 0 ) {
|
|
|
- app.finishTime = System.currentTimeMillis();
|
|
|
+ app.finishTime = app.systemClock.getTime();
|
|
|
}
|
|
|
// Recovered apps that are completed were not added to scheduler, so no
|
|
|
// need to remove them from scheduler.
|
|
@@ -1118,11 +1133,16 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
|
|
|
private int getNumFailedAppAttempts() {
|
|
|
int completedAttempts = 0;
|
|
|
+ long endTime = this.systemClock.getTime();
|
|
|
// Do not count AM preemption, hardware failures or NM resync
|
|
|
// as attempt failure.
|
|
|
for (RMAppAttempt attempt : attempts.values()) {
|
|
|
if (attempt.shouldCountTowardsMaxAttemptRetry()) {
|
|
|
- completedAttempts++;
|
|
|
+ if (this.attemptFailuresValidityInterval <= 0
|
|
|
+ || (attempt.getFinishTime() > endTime
|
|
|
+ - this.attemptFailuresValidityInterval)) {
|
|
|
+ completedAttempts++;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
return completedAttempts;
|
|
@@ -1139,8 +1159,9 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
|
|
|
@Override
|
|
|
public RMAppState transition(RMAppImpl app, RMAppEvent event) {
|
|
|
+ int numberOfFailure = app.getNumFailedAppAttempts();
|
|
|
if (!app.submissionContext.getUnmanagedAM()
|
|
|
- && app.getNumFailedAppAttempts() < app.maxAppAttempts) {
|
|
|
+ && numberOfFailure < app.maxAppAttempts) {
|
|
|
boolean transferStateFromPreviousAttempt = false;
|
|
|
RMAppFailedAttemptEvent failedEvent = (RMAppFailedAttemptEvent) event;
|
|
|
transferStateFromPreviousAttempt =
|
|
@@ -1158,6 +1179,9 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
}
|
|
|
return initialState;
|
|
|
} else {
|
|
|
+ if (numberOfFailure >= app.maxAppAttempts) {
|
|
|
+ app.isNumAttemptsBeyondThreshold = true;
|
|
|
+ }
|
|
|
app.rememberTargetTransitionsAndStoreState(event,
|
|
|
new AttemptFailedFinalStateSavedTransition(), RMAppState.FAILED,
|
|
|
RMAppState.FAILED);
|
|
@@ -1244,4 +1268,10 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|
|
numNonAMContainerPreempted, numAMContainerPreempted,
|
|
|
memorySeconds, vcoreSeconds);
|
|
|
}
|
|
|
+
|
|
|
+ @Private
|
|
|
+ @VisibleForTesting
|
|
|
+ public void setSystemClock(Clock clock) {
|
|
|
+ this.systemClock = clock;
|
|
|
+ }
|
|
|
}
|