YARN-891. Modified ResourceManager state-store to remember completed applications so that clients can get information about them post RM-restart. Contributed by Jian He.

svn merge --ignore-ancestry -c 1537560 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1537561 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2013-10-31 17:25:37 +00:00
parent 5ac97bbe25
commit ad46e707f4
32 changed files with 2303 additions and 597 deletions

View File

@ -55,6 +55,10 @@ Release 2.3.0 - UNRELEASED
YARN-1306. Clean up hadoop-sls sample-conf according to YARN-1228 (Wei Yan
via Sandy Ryza)
YARN-891. Modified ResourceManager state-store to remember completed
applications so that clients can get information about them post RM-restart.
(Jian He via vinodkv)
OPTIMIZATIONS
BUG FIXES

View File

@ -71,14 +71,53 @@ message GetGroupsForUserResponseProto {
////////////////////////////////////////////////////////////////////////
////// RM recovery related records /////////////////////////////////////
////////////////////////////////////////////////////////////////////////
enum RMAppAttemptStateProto {
RMATTEMPT_NEW = 1;
RMATTEMPT_SUBMITTED = 2;
RMATTEMPT_SCHEDULED = 3;
RMATTEMPT_ALLOCATED = 4;
RMATTEMPT_LAUNCHED = 5;
RMATTEMPT_FAILED = 6;
RMATTEMPT_RUNNING = 7;
RMATTEMPT_FINISHING = 8;
RMATTEMPT_FINISHED = 9;
RMATTEMPT_KILLED = 10;
RMATTEMPT_ALLOCATED_SAVING = 11;
RMATTEMPT_LAUNCHED_UNMANAGED_SAVING = 12;
RMATTEMPT_RECOVERED = 13;
RMATTEMPT_FINAL_SAVING = 14;
}
enum RMAppStateProto {
RMAPP_NEW = 1;
RMAPP_NEW_SAVING = 2;
RMAPP_SUBMITTED = 3;
RMAPP_ACCEPTED = 4;
RMAPP_RUNNING = 5;
RMAPP_FINAL_SAVING = 6;
RMAPP_FINISHING = 7;
RMAPP_FINISHED = 8;
RMAPP_FAILED = 9;
RMAPP_KILLED = 10;
}
message ApplicationStateDataProto {
optional int64 submit_time = 1;
optional ApplicationSubmissionContextProto application_submission_context = 2;
optional string user = 3;
optional int64 start_time = 2;
optional ApplicationSubmissionContextProto application_submission_context = 3;
optional string user = 4;
optional RMAppStateProto application_state = 5;
optional string diagnostics = 6 [default = "N/A"];
optional int64 finish_time = 7;
}
message ApplicationAttemptStateDataProto {
optional ApplicationAttemptIdProto attemptId = 1;
optional ContainerProto master_container = 2;
optional bytes app_attempt_tokens = 3;
optional RMAppAttemptStateProto app_attempt_state = 4;
optional string final_tracking_url = 5;
optional string diagnostics = 6 [default = "N/A"];
optional int64 start_time = 7;
optional FinalApplicationStatusProto final_application_status = 8;
}

View File

@ -63,7 +63,6 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
private static final Log LOG = LogFactory.getLog(RMAppManager.class);
private int completedAppsMax = YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS;
private int globalMaxAppAttempts;
private LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
private final RMContext rmContext;
@ -83,8 +82,6 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
setCompletedAppsMax(conf.getInt(
YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS,
YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS));
globalMaxAppAttempts = conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
}
/**
@ -302,10 +299,11 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
throw RPCUtil.getRemoteException(ie);
}
if (!isRecovered) {
// All done, start the RMApp
this.rmContext.getDispatcher().getEventHandler().handle(
new RMAppEvent(applicationId, isRecovered ? RMAppEventType.RECOVER:
RMAppEventType.START));
this.rmContext.getDispatcher().getEventHandler()
.handle(new RMAppEvent(applicationId, RMAppEventType.START));
}
}
private Credentials parseCredentials(ApplicationSubmissionContext application)
@ -328,53 +326,19 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
// recover applications
Map<ApplicationId, ApplicationState> appStates = state.getApplicationState();
LOG.info("Recovering " + appStates.size() + " applications");
for(ApplicationState appState : appStates.values()) {
boolean shouldRecover = true;
if(appState.getApplicationSubmissionContext().getUnmanagedAM()) {
// do not recover unmanaged applications since current recovery
// mechanism of restarting attempts does not work for them.
// This will need to be changed in work preserving recovery in which
// RM will re-connect with the running AM's instead of restarting them
LOG.info("Not recovering unmanaged application " + appState.getAppId());
shouldRecover = false;
}
int individualMaxAppAttempts = appState.getApplicationSubmissionContext()
.getMaxAppAttempts();
int maxAppAttempts;
if (individualMaxAppAttempts <= 0 ||
individualMaxAppAttempts > globalMaxAppAttempts) {
maxAppAttempts = globalMaxAppAttempts;
LOG.warn("The specific max attempts: " + individualMaxAppAttempts
+ " for application: " + appState.getAppId()
+ " is invalid, because it is out of the range [1, "
+ globalMaxAppAttempts + "]. Use the global max attempts instead.");
} else {
maxAppAttempts = individualMaxAppAttempts;
}
// In work-preserve restart, if attemptCount == maxAttempts, the job still
// needs to be recovered because the last attempt may still be running.
if(appState.getAttemptCount() >= maxAppAttempts) {
LOG.info("Not recovering application " + appState.getAppId() +
" due to recovering attempt is beyond maxAppAttempt limit");
shouldRecover = false;
}
// re-submit the application
// this is going to send an app start event but since the async dispatcher
// has not started that event will be queued until we have completed re
// populating the state
if(shouldRecover) {
for (ApplicationState appState : appStates.values()) {
LOG.info("Recovering application " + appState.getAppId());
submitApplication(appState.getApplicationSubmissionContext(),
appState.getSubmitTime(), true, appState.getUser());
// re-populate attempt information in application
RMAppImpl appImpl = (RMAppImpl) rmContext.getRMApps().get(
appState.getAppId());
RMAppImpl appImpl =
(RMAppImpl) rmContext.getRMApps().get(appState.getAppId());
appImpl.recover(state);
}
else {
store.removeApplication(appState);
}
// Recover the app synchronously, as otherwise client is possible to see
// the application not recovered before it is actually recovered because
// ClientRMService is already started at this point of time.
appImpl.handle(new RMAppEvent(appImpl.getApplicationId(),
RMAppEventType.RECOVER));
}
}

View File

@ -133,8 +133,11 @@ public class FileSystemRMStateStore extends RMStateStore {
ApplicationStateDataProto.parseFrom(childData));
ApplicationState appState =
new ApplicationState(appStateData.getSubmitTime(),
appStateData.getStartTime(),
appStateData.getApplicationSubmissionContext(),
appStateData.getUser());
appStateData.getUser(),
appStateData.getState(),
appStateData.getDiagnostics(), appStateData.getFinishTime());
// assert child node name is same as actual applicationId
assert appId.equals(appState.context.getApplicationId());
rmState.appState.put(appId, appState);
@ -156,7 +159,12 @@ public class FileSystemRMStateStore extends RMStateStore {
}
ApplicationAttemptState attemptState =
new ApplicationAttemptState(attemptId,
attemptStateData.getMasterContainer(), credentials);
attemptStateData.getMasterContainer(), credentials,
attemptStateData.getStartTime(),
attemptStateData.getState(),
attemptStateData.getFinalTrackingUrl(),
attemptStateData.getDiagnostics(),
attemptStateData.getFinalApplicationStatus());
// assert child node name is same as application attempt id
assert attemptId.equals(attemptState.getAttemptId());
@ -232,7 +240,7 @@ public class FileSystemRMStateStore extends RMStateStore {
}
@Override
public synchronized void storeApplicationState(String appId,
public synchronized void storeApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateDataPB) throws Exception {
Path appDirPath = getAppDir(rmAppRoot, appId);
fs.mkdirs(appDirPath);
@ -251,15 +259,34 @@ public class FileSystemRMStateStore extends RMStateStore {
}
@Override
public synchronized void storeApplicationAttemptState(String attemptId,
ApplicationAttemptStateDataPBImpl attemptStateDataPB) throws Exception {
public synchronized void updateApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateDataPB) throws Exception {
Path appDirPath = getAppDir(rmAppRoot, appId);
Path nodeCreatePath = getNodePath(appDirPath, appId);
LOG.info("Updating info for app: " + appId + " at: " + nodeCreatePath);
byte[] appStateData = appStateDataPB.getProto().toByteArray();
try {
// currently throw all exceptions. May need to respond differently for HA
// based on whether we have lost the right to write to FS
updateFile(nodeCreatePath, appStateData);
} catch (Exception e) {
LOG.info("Error updating info for app: " + appId, e);
throw e;
}
}
@Override
public synchronized void storeApplicationAttemptStateInternal(
String attemptId, ApplicationAttemptStateDataPBImpl attemptStateDataPB)
throws Exception {
ApplicationAttemptId appAttemptId =
ConverterUtils.toApplicationAttemptId(attemptId);
Path appDirPath =
getAppDir(rmAppRoot, appAttemptId.getApplicationId().toString());
Path nodeCreatePath = getNodePath(appDirPath, attemptId);
LOG.info("Storing info for attempt: " + attemptId
+ " at: " + nodeCreatePath);
LOG.info("Storing info for attempt: " + attemptId + " at: "
+ nodeCreatePath);
byte[] attemptStateData = attemptStateDataPB.getProto().toByteArray();
try {
// currently throw all exceptions. May need to respond differently for HA
@ -271,6 +298,28 @@ public class FileSystemRMStateStore extends RMStateStore {
}
}
@Override
public synchronized void updateApplicationAttemptStateInternal(
String attemptId, ApplicationAttemptStateDataPBImpl attemptStateDataPB)
throws Exception {
ApplicationAttemptId appAttemptId =
ConverterUtils.toApplicationAttemptId(attemptId);
Path appDirPath =
getAppDir(rmAppRoot, appAttemptId.getApplicationId().toString());
Path nodeCreatePath = getNodePath(appDirPath, attemptId);
LOG.info("Updating info for attempt: " + attemptId + " at: "
+ nodeCreatePath);
byte[] attemptStateData = attemptStateDataPB.getProto().toByteArray();
try {
// currently throw all exceptions. May need to respond differently for HA
// based on whether we have lost the right to write to FS
updateFile(nodeCreatePath, attemptStateData);
} catch (Exception e) {
LOG.info("Error updating info for attempt: " + attemptId, e);
throw e;
}
}
@Override
public synchronized void removeApplicationState(ApplicationState appState)
throws Exception {
@ -373,12 +422,21 @@ public class FileSystemRMStateStore extends RMStateStore {
Path tempPath =
new Path(outputPath.getParent(), outputPath.getName() + ".tmp");
FSDataOutputStream fsOut = null;
fsOut = fs.create(tempPath, false);
// This file will be overwritten when app/attempt finishes for saving the
// final status.
fsOut = fs.create(tempPath, true);
fsOut.write(data);
fsOut.close();
fs.rename(tempPath, outputPath);
}
private void updateFile(Path outputPath, byte[] data) throws Exception {
if (fs.exists(outputPath)) {
deleteFile(outputPath);
}
writeFile(outputPath, data);
}
private boolean renameFile(Path src, Path dst) throws Exception {
return fs.rename(src, dst);
}

View File

@ -76,22 +76,39 @@ public class MemoryRMStateStore extends RMStateStore {
}
@Override
public void storeApplicationState(String appId,
public void storeApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateData)
throws Exception {
ApplicationState appState = new ApplicationState(
appStateData.getSubmitTime(),
appStateData.getApplicationSubmissionContext(), appStateData.getUser());
if (state.appState.containsKey(appState.getAppId())) {
Exception e = new IOException("App: " + appId + " is already stored.");
LOG.info("Error storing info for app: " + appId, e);
throw e;
}
ApplicationState appState =
new ApplicationState(appStateData.getSubmitTime(),
appStateData.getStartTime(),
appStateData.getApplicationSubmissionContext(),
appStateData.getUser());
state.appState.put(appState.getAppId(), appState);
}
@Override
public synchronized void storeApplicationAttemptState(String attemptIdStr,
public void updateApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateData) throws Exception {
ApplicationState updatedAppState =
new ApplicationState(appStateData.getSubmitTime(),
appStateData.getStartTime(),
appStateData.getApplicationSubmissionContext(),
appStateData.getUser(), appStateData.getState(),
appStateData.getDiagnostics(), appStateData.getFinishTime());
LOG.info("Updating final state " + appStateData.getState() + " for app: "
+ appId);
ApplicationId applicationId = updatedAppState.getAppId();
if (state.appState.get(applicationId) != null) {
// add the earlier attempts back
updatedAppState.attempts
.putAll(state.appState.get(applicationId).attempts);
}
state.appState.put(applicationId, updatedAppState);
}
@Override
public synchronized void storeApplicationAttemptStateInternal(String attemptIdStr,
ApplicationAttemptStateDataPBImpl attemptStateData)
throws Exception {
ApplicationAttemptId attemptId = ConverterUtils
@ -105,24 +122,50 @@ public class MemoryRMStateStore extends RMStateStore {
}
ApplicationAttemptState attemptState =
new ApplicationAttemptState(attemptId,
attemptStateData.getMasterContainer(), credentials);
attemptStateData.getMasterContainer(), credentials,
attemptStateData.getStartTime());
ApplicationState appState = state.getApplicationState().get(
attemptState.getAttemptId().getApplicationId());
if (appState == null) {
throw new YarnRuntimeException("Application doesn't exist");
}
if (appState.attempts.containsKey(attemptState.getAttemptId())) {
Exception e = new IOException("Attempt: " +
attemptState.getAttemptId() + " is already stored.");
LOG.info("Error storing info for attempt: " +
attemptState.getAttemptId(), e);
throw e;
}
appState.attempts.put(attemptState.getAttemptId(), attemptState);
}
@Override
public synchronized void updateApplicationAttemptStateInternal(
String attemptIdStr, ApplicationAttemptStateDataPBImpl attemptStateData)
throws Exception {
ApplicationAttemptId attemptId =
ConverterUtils.toApplicationAttemptId(attemptIdStr);
Credentials credentials = null;
if (attemptStateData.getAppAttemptTokens() != null) {
DataInputByteBuffer dibb = new DataInputByteBuffer();
credentials = new Credentials();
dibb.reset(attemptStateData.getAppAttemptTokens());
credentials.readTokenStorageStream(dibb);
}
ApplicationAttemptState updatedAttemptState =
new ApplicationAttemptState(attemptId,
attemptStateData.getMasterContainer(), credentials,
attemptStateData.getStartTime(), attemptStateData.getState(),
attemptStateData.getFinalTrackingUrl(),
attemptStateData.getDiagnostics(),
attemptStateData.getFinalApplicationStatus());
ApplicationState appState =
state.getApplicationState().get(
updatedAttemptState.getAttemptId().getApplicationId());
if (appState == null) {
throw new YarnRuntimeException("Application doesn't exist");
}
LOG.info("Updating final state " + updatedAttemptState.getState()
+ " for attempt: " + updatedAttemptState.getAttemptId());
appState.attempts.put(updatedAttemptState.getAttemptId(),
updatedAttemptState);
}
@Override
public synchronized void removeApplicationState(ApplicationState appState)
throws Exception {

View File

@ -50,13 +50,13 @@ public class NullRMStateStore extends RMStateStore {
}
@Override
protected void storeApplicationState(String appId,
protected void storeApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateData) throws Exception {
// Do nothing
}
@Override
protected void storeApplicationAttemptState(String attemptId,
protected void storeApplicationAttemptStateInternal(String attemptId,
ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception {
// Do nothing
}
@ -89,4 +89,16 @@ public class NullRMStateStore extends RMStateStore {
public void removeRMDTMasterKeyState(DelegationKey delegationKey) throws Exception {
// Do nothing
}
@Override
protected void updateApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateData) throws Exception {
// Do nothing
}
@Override
protected void updateApplicationAttemptStateInternal(String attemptId,
ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception {
// Do nothing
}
}

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
@ -50,10 +51,14 @@ import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppStoredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRemovedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppUpdateSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStoredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
@Private
@Unstable
@ -86,13 +91,32 @@ public abstract class RMStateStore extends AbstractService {
final ApplicationAttemptId attemptId;
final Container masterContainer;
final Credentials appAttemptCredentials;
long startTime = 0;
// fields set when attempt completes
RMAppAttemptState state;
String finalTrackingUrl = "N/A";
String diagnostics;
FinalApplicationStatus amUnregisteredFinalStatus;
public ApplicationAttemptState(ApplicationAttemptId attemptId,
Container masterContainer,
Credentials appAttemptCredentials) {
Container masterContainer, Credentials appAttemptCredentials,
long startTime) {
this(attemptId, masterContainer, appAttemptCredentials, startTime, null,
null, "", null);
}
public ApplicationAttemptState(ApplicationAttemptId attemptId,
Container masterContainer, Credentials appAttemptCredentials,
long startTime, RMAppAttemptState state, String finalTrackingUrl,
String diagnostics, FinalApplicationStatus amUnregisteredFinalStatus) {
this.attemptId = attemptId;
this.masterContainer = masterContainer;
this.appAttemptCredentials = appAttemptCredentials;
this.startTime = startTime;
this.state = state;
this.finalTrackingUrl = finalTrackingUrl;
this.diagnostics = diagnostics == null ? "" : diagnostics;
this.amUnregisteredFinalStatus = amUnregisteredFinalStatus;
}
public Container getMasterContainer() {
@ -104,6 +128,21 @@ public abstract class RMStateStore extends AbstractService {
public Credentials getAppAttemptCredentials() {
return appAttemptCredentials;
}
public RMAppAttemptState getState(){
return state;
}
public String getFinalTrackingUrl() {
return finalTrackingUrl;
}
public String getDiagnostics() {
return diagnostics;
}
public long getStartTime() {
return startTime;
}
public FinalApplicationStatus getFinalApplicationStatus() {
return amUnregisteredFinalStatus;
}
}
/**
@ -112,15 +151,30 @@ public abstract class RMStateStore extends AbstractService {
public static class ApplicationState {
final ApplicationSubmissionContext context;
final long submitTime;
final long startTime;
final String user;
Map<ApplicationAttemptId, ApplicationAttemptState> attempts =
new HashMap<ApplicationAttemptId, ApplicationAttemptState>();
// fields set when application completes.
RMAppState state;
String diagnostics;
long finishTime;
ApplicationState(long submitTime, ApplicationSubmissionContext context,
String user) {
public ApplicationState(long submitTime,
long startTime, ApplicationSubmissionContext context, String user) {
this(submitTime, startTime, context, user, null, "", 0);
}
public ApplicationState(long submitTime,
long startTime,ApplicationSubmissionContext context,
String user, RMAppState state, String diagnostics, long finishTime) {
this.submitTime = submitTime;
this.startTime = startTime;
this.context = context;
this.user = user;
this.state = state;
this.diagnostics = diagnostics == null ? "" : diagnostics;
this.finishTime = finishTime;
}
public ApplicationId getAppId() {
@ -129,6 +183,9 @@ public abstract class RMStateStore extends AbstractService {
public long getSubmitTime() {
return submitTime;
}
public long getStartTime() {
return startTime;
}
public int getAttemptCount() {
return attempts.size();
}
@ -141,6 +198,15 @@ public abstract class RMStateStore extends AbstractService {
public String getUser() {
return user;
}
public RMAppState getState() {
return state;
}
public String getDiagnostics() {
return diagnostics;
}
public long getFinishTime() {
return finishTime;
}
}
public static class RMDTSecretManagerState {
@ -249,23 +315,31 @@ public abstract class RMStateStore extends AbstractService {
* RMAppStoredEvent will be sent on completion to notify the RMApp
*/
@SuppressWarnings("unchecked")
public synchronized void storeApplication(RMApp app) {
public synchronized void storeNewApplication(RMApp app) {
ApplicationSubmissionContext context = app
.getApplicationSubmissionContext();
assert context instanceof ApplicationSubmissionContextPBImpl;
ApplicationState appState = new ApplicationState(
app.getSubmitTime(), context, app.getUser());
ApplicationState appState =
new ApplicationState(app.getSubmitTime(), app.getStartTime(), context,
app.getUser());
dispatcher.getEventHandler().handle(new RMStateStoreAppEvent(appState));
}
@SuppressWarnings("unchecked")
public synchronized void updateApplicationState(ApplicationState appState) {
dispatcher.getEventHandler().handle(new RMStateUpdateAppEvent(appState));
}
/**
* Blocking API
* Derived classes must implement this method to store the state of an
* application.
*/
protected abstract void storeApplicationState(String appId,
ApplicationStateDataPBImpl appStateData)
throws Exception;
protected abstract void storeApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateData) throws Exception;
protected abstract void updateApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateData) throws Exception;
@SuppressWarnings("unchecked")
/**
@ -274,26 +348,35 @@ public abstract class RMStateStore extends AbstractService {
* This does not block the dispatcher threads
* RMAppAttemptStoredEvent will be sent on completion to notify the RMAppAttempt
*/
public synchronized void storeApplicationAttempt(RMAppAttempt appAttempt) {
public synchronized void storeNewApplicationAttempt(RMAppAttempt appAttempt) {
Credentials credentials = getCredentialsFromAppAttempt(appAttempt);
ApplicationAttemptState attemptState =
new ApplicationAttemptState(appAttempt.getAppAttemptId(),
appAttempt.getMasterContainer(), credentials);
appAttempt.getMasterContainer(), credentials,
appAttempt.getStartTime());
dispatcher.getEventHandler().handle(
new RMStateStoreAppAttemptEvent(attemptState));
}
@SuppressWarnings("unchecked")
public synchronized void updateApplicationAttemptState(
ApplicationAttemptState attemptState) {
dispatcher.getEventHandler().handle(
new RMStateUpdateAppAttemptEvent(attemptState));
}
/**
* Blocking API
* Derived classes must implement this method to store the state of an
* application attempt
*/
protected abstract void storeApplicationAttemptState(String attemptId,
ApplicationAttemptStateDataPBImpl attemptStateData)
throws Exception;
protected abstract void storeApplicationAttemptStateInternal(String attemptId,
ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception;
protected abstract void updateApplicationAttemptStateInternal(String attemptId,
ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception;
/**
* RMDTSecretManager call this to store the state of a delegation token
@ -372,13 +455,14 @@ public abstract class RMStateStore extends AbstractService {
*/
public synchronized void removeApplication(RMApp app) {
ApplicationState appState = new ApplicationState(
app.getSubmitTime(), app.getApplicationSubmissionContext(),
app.getUser());
app.getSubmitTime(), app.getStartTime(),
app.getApplicationSubmissionContext(), app.getUser());
for(RMAppAttempt appAttempt : app.getAppAttempts().values()) {
Credentials credentials = getCredentialsFromAppAttempt(appAttempt);
ApplicationAttemptState attemptState =
new ApplicationAttemptState(appAttempt.getAppAttemptId(),
appAttempt.getMasterContainer(), credentials);
appAttempt.getMasterContainer(), credentials,
appAttempt.getStartTime());
appState.attempts.put(attemptState.getAttemptId(), attemptState);
}
@ -409,7 +493,7 @@ public abstract class RMStateStore extends AbstractService {
public static final Text AM_CLIENT_TOKEN_MASTER_KEY_NAME =
new Text("YARN_CLIENT_TOKEN_MASTER_KEY");
private Credentials getCredentialsFromAppAttempt(RMAppAttempt appAttempt) {
public Credentials getCredentialsFromAppAttempt(RMAppAttempt appAttempt) {
Credentials credentials = new Credentials();
Token<AMRMTokenIdentifier> appToken = appAttempt.getAMRMToken();
if(appToken != null){
@ -427,68 +511,101 @@ public abstract class RMStateStore extends AbstractService {
// Dispatcher related code
private synchronized void handleStoreEvent(RMStateStoreEvent event) {
switch(event.getType()) {
case STORE_APP:
{
ApplicationState apptState =
((RMStateStoreAppEvent) event).getAppState();
if (event.getType().equals(RMStateStoreEventType.STORE_APP)
|| event.getType().equals(RMStateStoreEventType.UPDATE_APP)) {
ApplicationState appState = null;
if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
appState = ((RMStateStoreAppEvent) event).getAppState();
} else {
assert event.getType().equals(RMStateStoreEventType.UPDATE_APP);
appState = ((RMStateUpdateAppEvent) event).getAppState();
}
Exception storedException = null;
ApplicationStateDataPBImpl appStateData =
new ApplicationStateDataPBImpl();
appStateData.setSubmitTime(apptState.getSubmitTime());
appStateData.setApplicationSubmissionContext(
apptState.getApplicationSubmissionContext());
appStateData.setUser(apptState.getUser());
(ApplicationStateDataPBImpl) ApplicationStateDataPBImpl
.newApplicationStateData(appState.getSubmitTime(),
appState.getStartTime(), appState.getUser(),
appState.getApplicationSubmissionContext(), appState.getState(),
appState.getDiagnostics(), appState.getFinishTime());
ApplicationId appId =
apptState.getApplicationSubmissionContext().getApplicationId();
appState.getApplicationSubmissionContext().getApplicationId();
LOG.info("Storing info for app: " + appId);
try {
storeApplicationState(appId.toString(), appStateData);
if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
storeApplicationStateInternal(appId.toString(), appStateData);
} else {
assert event.getType().equals(RMStateStoreEventType.UPDATE_APP);
updateApplicationStateInternal(appId.toString(), appStateData);
}
} catch (Exception e) {
LOG.error("Error storing app: " + appId, e);
storedException = e;
} finally {
if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
notifyDoneStoringApplication(appId, storedException);
} else {
notifyDoneUpdatingApplication(appId, storedException);
}
}
break;
case STORE_APP_ATTEMPT:
{
ApplicationAttemptState attemptState =
((RMStateStoreAppAttemptEvent) event).getAppAttemptState();
Exception storedException = null;
} else if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)
|| event.getType().equals(RMStateStoreEventType.UPDATE_APP_ATTEMPT)) {
ApplicationAttemptState attemptState = null;
if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
attemptState =
((RMStateStoreAppAttemptEvent) event).getAppAttemptState();
} else {
assert event.getType().equals(RMStateStoreEventType.UPDATE_APP_ATTEMPT);
attemptState =
((RMStateUpdateAppAttemptEvent) event).getAppAttemptState();
}
Exception storedException = null;
Credentials credentials = attemptState.getAppAttemptCredentials();
ByteBuffer appAttemptTokens = null;
try {
if(credentials != null){
if (credentials != null) {
DataOutputBuffer dob = new DataOutputBuffer();
credentials.writeTokenStorageToStream(dob);
appAttemptTokens =
ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
appAttemptTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
}
ApplicationAttemptStateDataPBImpl attemptStateData =
(ApplicationAttemptStateDataPBImpl) ApplicationAttemptStateDataPBImpl
.newApplicationAttemptStateData(attemptState.getAttemptId(),
attemptState.getMasterContainer(), appAttemptTokens);
attemptState.getMasterContainer(), appAttemptTokens,
attemptState.getStartTime(), attemptState.getState(),
attemptState.getFinalTrackingUrl(),
attemptState.getDiagnostics(),
attemptState.getFinalApplicationStatus());
if (LOG.isDebugEnabled()) {
LOG.debug("Storing info for attempt: " + attemptState.getAttemptId());
}
storeApplicationAttemptState(attemptState.getAttemptId().toString(),
attemptStateData);
if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
storeApplicationAttemptStateInternal(attemptState.getAttemptId()
.toString(), attemptStateData);
} else {
assert event.getType().equals(
RMStateStoreEventType.UPDATE_APP_ATTEMPT);
updateApplicationAttemptStateInternal(attemptState.getAttemptId()
.toString(), attemptStateData);
}
} catch (Exception e) {
LOG.error("Error storing appAttempt: "
+ attemptState.getAttemptId(), e);
LOG
.error("Error storing appAttempt: " + attemptState.getAttemptId(), e);
storedException = e;
} finally {
if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
notifyDoneStoringApplicationAttempt(attemptState.getAttemptId(),
storedException);
} else {
notifyDoneUpdatingApplicationAttempt(attemptState.getAttemptId(),
storedException);
}
}
break;
case REMOVE_APP:
{
} else if (event.getType().equals(RMStateStoreEventType.REMOVE_APP)) {
ApplicationState appState =
((RMStateStoreRemoveAppEvent) event).getAppState();
ApplicationId appId = appState.getAppId();
@ -502,9 +619,7 @@ public abstract class RMStateStore extends AbstractService {
} finally {
notifyDoneRemovingApplcation(appId, removedException);
}
}
break;
default:
} else {
LOG.error("Unknown RMStateStoreEvent type: " + event.getType());
}
}
@ -512,7 +627,7 @@ public abstract class RMStateStore extends AbstractService {
@SuppressWarnings("unchecked")
/**
* In (@link handleStoreEvent}, this method is called to notify the
* application about operation completion
* application that new application is stored in state store
* @param appId id of the application that has been saved
* @param storedException the exception that is thrown when storing the
* application
@ -520,19 +635,33 @@ public abstract class RMStateStore extends AbstractService {
private void notifyDoneStoringApplication(ApplicationId appId,
Exception storedException) {
rmDispatcher.getEventHandler().handle(
new RMAppStoredEvent(appId, storedException));
new RMAppNewSavedEvent(appId, storedException));
}
@SuppressWarnings("unchecked")
private void notifyDoneUpdatingApplication(ApplicationId appId,
Exception storedException) {
rmDispatcher.getEventHandler().handle(
new RMAppUpdateSavedEvent(appId, storedException));
}
@SuppressWarnings("unchecked")
/**
* In (@link handleStoreEvent}, this method is called to notify the
* application attempt about operation completion
* application attempt that new attempt is stored in state store
* @param appAttempt attempt that has been saved
*/
private void notifyDoneStoringApplicationAttempt(ApplicationAttemptId attemptId,
Exception storedException) {
rmDispatcher.getEventHandler().handle(
new RMAppAttemptStoredEvent(attemptId, storedException));
new RMAppAttemptNewSavedEvent(attemptId, storedException));
}
@SuppressWarnings("unchecked")
private void notifyDoneUpdatingApplicationAttempt(ApplicationAttemptId attemptId,
Exception updatedException) {
rmDispatcher.getEventHandler().handle(
new RMAppAttemptUpdateSavedEvent(attemptId, updatedException));
}
@SuppressWarnings("unchecked")

View File

@ -21,5 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.recovery;
public enum RMStateStoreEventType {
STORE_APP_ATTEMPT,
STORE_APP,
UPDATE_APP,
UPDATE_APP_ATTEMPT,
REMOVE_APP
}

View File

@ -0,0 +1,35 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
public class RMStateUpdateAppAttemptEvent extends RMStateStoreEvent {
ApplicationAttemptState attemptState;
public RMStateUpdateAppAttemptEvent(ApplicationAttemptState attemptState) {
super(RMStateStoreEventType.UPDATE_APP_ATTEMPT);
this.attemptState = attemptState;
}
public ApplicationAttemptState getAppAttemptState() {
return attemptState;
}
}

View File

@ -0,0 +1,34 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
public class RMStateUpdateAppEvent extends RMStateStoreEvent {
private final ApplicationState appState;
public RMStateUpdateAppEvent(ApplicationState appState) {
super(RMStateStoreEventType.UPDATE_APP);
this.appState = appState;
}
public ApplicationState getAppState() {
return appState;
}
}

View File

@ -18,7 +18,14 @@
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
import com.google.common.annotations.VisibleForTesting;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@ -27,6 +34,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.DataInputByteBuffer;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ZKUtil;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@ -37,9 +46,6 @@ import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ZKUtil;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.Code;
@ -51,13 +57,7 @@ import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import com.google.common.annotations.VisibleForTesting;
@Private
@Unstable
@ -224,8 +224,11 @@ public class ZKRMStateStore extends RMStateStore {
ApplicationStateDataProto.parseFrom(childData));
ApplicationState appState =
new ApplicationState(appStateData.getSubmitTime(),
appStateData.getStartTime(),
appStateData.getApplicationSubmissionContext(),
appStateData.getUser());
appStateData.getUser(),
appStateData.getState(),
appStateData.getDiagnostics(), appStateData.getFinishTime());
if (!appId.equals(appState.context.getApplicationId())) {
throw new YarnRuntimeException("The child node name is different " +
"from the application id");
@ -249,7 +252,12 @@ public class ZKRMStateStore extends RMStateStore {
}
ApplicationAttemptState attemptState =
new ApplicationAttemptState(attemptId,
attemptStateData.getMasterContainer(), credentials);
attemptStateData.getMasterContainer(), credentials,
attemptStateData.getStartTime(),
attemptStateData.getState(),
attemptStateData.getFinalTrackingUrl(),
attemptStateData.getDiagnostics(),
attemptStateData.getFinalApplicationStatus());
if (!attemptId.equals(attemptState.getAttemptId())) {
throw new YarnRuntimeException("The child node name is different " +
"from the application attempt id");
@ -280,21 +288,34 @@ public class ZKRMStateStore extends RMStateStore {
}
@Override
public synchronized void storeApplicationState(
String appId, ApplicationStateDataPBImpl appStateDataPB) throws
Exception {
public synchronized void storeApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateDataPB) throws Exception {
String nodeCreatePath = getNodePath(rmAppRoot, appId);
if (LOG.isDebugEnabled()) {
LOG.debug("Storing info for app: " + appId + " at: " + nodeCreatePath);
}
byte[] appStateData = appStateDataPB.getProto().toByteArray();
createWithRetries(
nodeCreatePath, appStateData, zkAcl, CreateMode.PERSISTENT);
createWithRetries(nodeCreatePath, appStateData, zkAcl,
CreateMode.PERSISTENT);
}
@Override
public synchronized void storeApplicationAttemptState(
public synchronized void updateApplicationStateInternal(String appId,
ApplicationStateDataPBImpl appStateDataPB) throws Exception {
String nodeCreatePath = getNodePath(rmAppRoot, appId);
if (LOG.isDebugEnabled()) {
LOG.debug("Storing final state info for app: " + appId + " at: "
+ nodeCreatePath);
}
byte[] appStateData = appStateDataPB.getProto().toByteArray();
setDataWithRetries(nodeCreatePath, appStateData, 0);
}
@Override
public synchronized void storeApplicationAttemptStateInternal(
String attemptId, ApplicationAttemptStateDataPBImpl attemptStateDataPB)
throws Exception {
String nodeCreatePath = getNodePath(rmAppRoot, attemptId);
@ -307,6 +328,19 @@ public class ZKRMStateStore extends RMStateStore {
CreateMode.PERSISTENT);
}
@Override
public synchronized void updateApplicationAttemptStateInternal(
String attemptId, ApplicationAttemptStateDataPBImpl attemptStateDataPB)
throws Exception {
String nodeCreatePath = getNodePath(rmAppRoot, attemptId);
if (LOG.isDebugEnabled()) {
LOG.debug("Storing final state info for attempt: " + attemptId + " at: "
+ nodeCreatePath);
}
byte[] attemptStateData = attemptStateDataPB.getProto().toByteArray();
setDataWithRetries(nodeCreatePath, attemptStateData, 0);
}
@Override
public synchronized void removeApplicationState(ApplicationState appState)
throws Exception {

View File

@ -24,6 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
/*
* Contains the state data that needs to be persisted for an ApplicationAttempt
@ -61,4 +63,50 @@ public interface ApplicationAttemptStateData {
public ByteBuffer getAppAttemptTokens();
public void setAppAttemptTokens(ByteBuffer attemptTokens);
/**
* Get the final state of the application attempt.
* @return the final state of the application attempt.
*/
public RMAppAttemptState getState();
public void setState(RMAppAttemptState state);
/**
* Get the original not-proxied <em>final tracking url</em> for the
* application. This is intended to only be used by the proxy itself.
*
* @return the original not-proxied <em>final tracking url</em> for the
* application
*/
public String getFinalTrackingUrl();
/**
* Set the final tracking Url of the AM.
* @param url
*/
public void setFinalTrackingUrl(String url);
/**
* Get the <em>diagnositic information</em> of the attempt
* @return <em>diagnositic information</em> of the attempt
*/
public String getDiagnostics();
public void setDiagnostics(String diagnostics);
/**
* Get the <em>start time</em> of the application.
* @return <em>start time</em> of the application
*/
public long getStartTime();
public void setStartTime(long startTime);
/**
* Get the <em>final finish status</em> of the application.
* @return <em>final finish status</em> of the application
*/
public FinalApplicationStatus getFinalApplicationStatus();
public void setFinalApplicationStatus(FinalApplicationStatus finishState);
}

View File

@ -18,10 +18,13 @@
package org.apache.hadoop.yarn.server.resourcemanager.recovery.records;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
/**
* Contains all the state data that needs to be stored persistently
@ -43,6 +46,18 @@ public interface ApplicationStateData {
@Unstable
public void setSubmitTime(long submitTime);
/**
* Get the <em>start time</em> of the application.
* @return <em>start time</em> of the application
*/
@Public
@Stable
public abstract long getStartTime();
@Private
@Unstable
public abstract void setStartTime(long startTime);
/**
* The application submitter
*/
@ -68,4 +83,27 @@ public interface ApplicationStateData {
public void setApplicationSubmissionContext(
ApplicationSubmissionContext context);
/**
* Get the final state of the application.
* @return the final state of the application.
*/
public RMAppState getState();
public void setState(RMAppState state);
/**
* Get the diagnostics information for the application master.
* @return the diagnostics information for the application master.
*/
public String getDiagnostics();
public void setDiagnostics(String diagnostics);
/**
* The finish time of the application.
* @return the finish time of the application.,
*/
public long getFinishTime();
public void setFinishTime(long finishTime);
}

View File

@ -22,14 +22,19 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProto;
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProtoOrBuilder;
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RMAppAttemptStateProto;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
public class ApplicationAttemptStateDataPBImpl
extends ProtoBase<ApplicationAttemptStateDataProto>
@ -156,14 +161,125 @@ implements ApplicationAttemptStateData {
this.appAttemptTokens = attemptTokens;
}
@Override
public RMAppAttemptState getState() {
ApplicationAttemptStateDataProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasAppAttemptState()) {
return null;
}
return convertFromProtoFormat(p.getAppAttemptState());
}
@Override
public void setState(RMAppAttemptState state) {
maybeInitBuilder();
if (state == null) {
builder.clearAppAttemptState();
return;
}
builder.setAppAttemptState(convertToProtoFormat(state));
}
@Override
public String getFinalTrackingUrl() {
ApplicationAttemptStateDataProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasFinalTrackingUrl()) {
return null;
}
return p.getFinalTrackingUrl();
}
@Override
public void setFinalTrackingUrl(String url) {
maybeInitBuilder();
if (url == null) {
builder.clearFinalTrackingUrl();
return;
}
builder.setFinalTrackingUrl(url);
}
@Override
public String getDiagnostics() {
ApplicationAttemptStateDataProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasDiagnostics()) {
return null;
}
return p.getDiagnostics();
}
@Override
public void setDiagnostics(String diagnostics) {
maybeInitBuilder();
if (diagnostics == null) {
builder.clearDiagnostics();
return;
}
builder.setDiagnostics(diagnostics);
}
@Override
public long getStartTime() {
ApplicationAttemptStateDataProtoOrBuilder p = viaProto ? proto : builder;
return p.getStartTime();
}
@Override
public void setStartTime(long startTime) {
maybeInitBuilder();
builder.setStartTime(startTime);
}
@Override
public FinalApplicationStatus getFinalApplicationStatus() {
ApplicationAttemptStateDataProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasFinalApplicationStatus()) {
return null;
}
return convertFromProtoFormat(p.getFinalApplicationStatus());
}
@Override
public void setFinalApplicationStatus(FinalApplicationStatus finishState) {
maybeInitBuilder();
if (finishState == null) {
builder.clearFinalApplicationStatus();
return;
}
builder.setFinalApplicationStatus(convertToProtoFormat(finishState));
}
public static ApplicationAttemptStateData newApplicationAttemptStateData(
ApplicationAttemptId attemptId, Container container,
ByteBuffer attemptTokens) {
ByteBuffer attemptTokens, long startTime, RMAppAttemptState finalState,
String finalTrackingUrl, String diagnostics,
FinalApplicationStatus amUnregisteredFinalStatus) {
ApplicationAttemptStateData attemptStateData =
recordFactory.newRecordInstance(ApplicationAttemptStateData.class);
attemptStateData.setAttemptId(attemptId);
attemptStateData.setMasterContainer(container);
attemptStateData.setAppAttemptTokens(attemptTokens);
attemptStateData.setState(finalState);
attemptStateData.setFinalTrackingUrl(finalTrackingUrl);
attemptStateData.setDiagnostics(diagnostics);
attemptStateData.setStartTime(startTime);
attemptStateData.setFinalApplicationStatus(amUnregisteredFinalStatus);
return attemptStateData;
}
private static String RM_APP_ATTEMPT_PREFIX = "RMATTEMPT_";
public static RMAppAttemptStateProto convertToProtoFormat(RMAppAttemptState e) {
return RMAppAttemptStateProto.valueOf(RM_APP_ATTEMPT_PREFIX + e.name());
}
public static RMAppAttemptState convertFromProtoFormat(RMAppAttemptStateProto e) {
return RMAppAttemptState.valueOf(e.name().replace(RM_APP_ATTEMPT_PREFIX, ""));
}
private FinalApplicationStatusProto convertToProtoFormat(FinalApplicationStatus s) {
return ProtoUtils.convertToProtoFormat(s);
}
private FinalApplicationStatus convertFromProtoFormat(FinalApplicationStatusProto s) {
return ProtoUtils.convertFromProtoFormat(s);
}
}

View File

@ -21,13 +21,19 @@ package org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProtoOrBuilder;
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RMAppStateProto;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
public class ApplicationStateDataPBImpl
extends ProtoBase<ApplicationStateDataProto>
implements ApplicationStateData {
private static final RecordFactory recordFactory = RecordFactoryProvider
.getRecordFactory(null);
ApplicationStateDataProto proto =
ApplicationStateDataProto.getDefaultInstance();
@ -91,6 +97,18 @@ implements ApplicationStateData {
builder.setSubmitTime(submitTime);
}
@Override
public long getStartTime() {
ApplicationStateDataProtoOrBuilder p = viaProto ? proto : builder;
return p.getStartTime();
}
@Override
public void setStartTime(long startTime) {
maybeInitBuilder();
builder.setStartTime(startTime);
}
@Override
public String getUser() {
ApplicationStateDataProtoOrBuilder p = viaProto ? proto : builder;
@ -132,4 +150,78 @@ implements ApplicationStateData {
this.applicationSubmissionContext = context;
}
@Override
public RMAppState getState() {
ApplicationStateDataProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasApplicationState()) {
return null;
}
return convertFromProtoFormat(p.getApplicationState());
}
@Override
public void setState(RMAppState finalState) {
maybeInitBuilder();
if (finalState == null) {
builder.clearApplicationState();
return;
}
builder.setApplicationState(convertToProtoFormat(finalState));
}
@Override
public String getDiagnostics() {
ApplicationStateDataProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasDiagnostics()) {
return null;
}
return p.getDiagnostics();
}
@Override
public void setDiagnostics(String diagnostics) {
maybeInitBuilder();
if (diagnostics == null) {
builder.clearDiagnostics();
return;
}
builder.setDiagnostics(diagnostics);
}
@Override
public long getFinishTime() {
ApplicationStateDataProtoOrBuilder p = viaProto ? proto : builder;
return p.getFinishTime();
}
@Override
public void setFinishTime(long finishTime) {
maybeInitBuilder();
builder.setFinishTime(finishTime);
}
public static ApplicationStateData newApplicationStateData(long submitTime,
long startTime, String user,
ApplicationSubmissionContext submissionContext, RMAppState state,
String diagnostics, long finishTime) {
ApplicationStateData appState =
recordFactory.newRecordInstance(ApplicationStateData.class);
appState.setSubmitTime(submitTime);
appState.setStartTime(startTime);
appState.setUser(user);
appState.setApplicationSubmissionContext(submissionContext);
appState.setState(state);
appState.setDiagnostics(diagnostics);
appState.setFinishTime(finishTime);
return appState;
}
private static String RM_APP_PREFIX = "RMAPP_";
public static RMAppStateProto convertToProtoFormat(RMAppState e) {
return RMAppStateProto.valueOf(RM_APP_PREFIX + e.name());
}
public static RMAppState convertFromProtoFormat(RMAppStateProto e) {
return RMAppState.valueOf(e.name().replace(RM_APP_PREFIX, ""));
}
}

View File

@ -35,6 +35,7 @@ public enum RMAppEventType {
NODE_UPDATE,
// Source: RMStateStore
APP_SAVED,
APP_NEW_SAVED,
APP_UPDATE_SAVED,
APP_REMOVED
}

View File

@ -54,10 +54,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppNodeUpdateEvent.RMAppNodeUpdateType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@ -103,7 +101,8 @@ public class RMAppImpl implements RMApp, Recoverable {
// Mutable fields
private long startTime;
private long finishTime;
private long finishTime = 0;
private long storedFinishTime = 0;
private RMAppAttempt currentAttempt;
private String queue;
@SuppressWarnings("rawtypes")
@ -111,8 +110,11 @@ public class RMAppImpl implements RMApp, Recoverable {
private static final FinalTransition FINAL_TRANSITION = new FinalTransition();
private static final AppFinishedTransition FINISHED_TRANSITION =
new AppFinishedTransition();
private boolean isAppRemovalRequestSent = false;
private RMAppState previousStateAtRemoving;
private RMAppState stateBeforeFinalSaving;
private RMAppEvent eventCausingFinalSaving;
private RMAppState targetedFinalState;
private RMAppState recoveredFinalState;
Object transitionTodo;
private static final StateMachineFactory<RMAppImpl,
RMAppState,
@ -129,32 +131,45 @@ public class RMAppImpl implements RMApp, Recoverable {
RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
.addTransition(RMAppState.NEW, RMAppState.NEW_SAVING,
RMAppEventType.START, new RMAppSavingTransition())
.addTransition(RMAppState.NEW, RMAppState.SUBMITTED,
RMAppEventType.RECOVER, new StartAppAttemptTransition())
.addTransition(RMAppState.NEW, RMAppState.KILLED, RMAppEventType.KILL,
new AppKilledTransition())
.addTransition(RMAppState.NEW, RMAppState.FAILED,
RMAppEventType.APP_REJECTED, new AppRejectedTransition())
.addTransition(RMAppState.NEW, EnumSet.of(RMAppState.SUBMITTED,
RMAppState.FINISHED, RMAppState.FAILED, RMAppState.KILLED,
RMAppState.FINAL_SAVING),
RMAppEventType.RECOVER, new RMAppRecoveredTransition())
.addTransition(RMAppState.NEW, RMAppState.FINAL_SAVING, RMAppEventType.KILL,
new FinalSavingTransition(
new AppKilledTransition(), RMAppState.KILLED))
.addTransition(RMAppState.NEW, RMAppState.FINAL_SAVING,
RMAppEventType.APP_REJECTED,
new FinalSavingTransition(
new AppRejectedTransition(), RMAppState.FAILED))
// Transitions from NEW_SAVING state
.addTransition(RMAppState.NEW_SAVING, RMAppState.NEW_SAVING,
RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
.addTransition(RMAppState.NEW_SAVING, RMAppState.SUBMITTED,
RMAppEventType.APP_SAVED, new StartAppAttemptTransition())
.addTransition(RMAppState.NEW_SAVING, RMAppState.KILLED,
RMAppEventType.KILL, new AppKilledTransition())
.addTransition(RMAppState.NEW_SAVING, RMAppState.FAILED,
RMAppEventType.APP_REJECTED, new AppRejectedTransition())
RMAppEventType.APP_NEW_SAVED, new StartAppAttemptTransition())
.addTransition(RMAppState.NEW_SAVING, RMAppState.FINAL_SAVING,
RMAppEventType.KILL,
new FinalSavingTransition(
new AppKilledTransition(), RMAppState.KILLED))
.addTransition(RMAppState.NEW_SAVING, RMAppState.FINAL_SAVING,
RMAppEventType.APP_REJECTED,
new FinalSavingTransition(new AppRejectedTransition(),
RMAppState.FAILED))
// Transitions from SUBMITTED state
.addTransition(RMAppState.SUBMITTED, RMAppState.SUBMITTED,
RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
.addTransition(RMAppState.SUBMITTED, RMAppState.FAILED,
RMAppEventType.APP_REJECTED, new AppRejectedTransition())
.addTransition(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING,
RMAppEventType.APP_REJECTED,
new FinalSavingTransition(
new AppRejectedTransition(), RMAppState.FAILED))
.addTransition(RMAppState.SUBMITTED, RMAppState.ACCEPTED,
RMAppEventType.APP_ACCEPTED)
.addTransition(RMAppState.SUBMITTED, RMAppState.KILLED,
RMAppEventType.KILL, new KillAppAndAttemptTransition())
.addTransition(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING,
RMAppEventType.KILL,
new FinalSavingTransition(
new KillAppAndAttemptTransition(), RMAppState.KILLED))
// Transitions from ACCEPTED state
.addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
@ -162,37 +177,45 @@ public class RMAppImpl implements RMApp, Recoverable {
.addTransition(RMAppState.ACCEPTED, RMAppState.RUNNING,
RMAppEventType.ATTEMPT_REGISTERED)
.addTransition(RMAppState.ACCEPTED,
EnumSet.of(RMAppState.SUBMITTED, RMAppState.FAILED),
EnumSet.of(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING),
RMAppEventType.ATTEMPT_FAILED,
new AttemptFailedTransition(RMAppState.SUBMITTED))
.addTransition(RMAppState.ACCEPTED, RMAppState.KILLED,
RMAppEventType.KILL, new KillAppAndAttemptTransition())
.addTransition(RMAppState.ACCEPTED, RMAppState.FINAL_SAVING,
RMAppEventType.KILL,
new FinalSavingTransition(
new KillAppAndAttemptTransition(), RMAppState.KILLED))
// Transitions from RUNNING state
.addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
.addTransition(RMAppState.RUNNING, RMAppState.REMOVING,
.addTransition(RMAppState.RUNNING, RMAppState.FINAL_SAVING,
RMAppEventType.ATTEMPT_UNREGISTERED,
new RMAppRemovingTransition())
new FinalSavingTransition(
new AttemptUnregisteredTransition(),
RMAppState.FINISHING, RMAppState.FINISHED))
.addTransition(RMAppState.RUNNING, RMAppState.FINISHED,
// UnManagedAM directly jumps to finished
RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION)
.addTransition(RMAppState.RUNNING,
EnumSet.of(RMAppState.SUBMITTED, RMAppState.FAILED),
EnumSet.of(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING),
RMAppEventType.ATTEMPT_FAILED,
new AttemptFailedTransition(RMAppState.SUBMITTED))
.addTransition(RMAppState.RUNNING, RMAppState.KILLED,
RMAppEventType.KILL, new KillAppAndAttemptTransition())
.addTransition(RMAppState.RUNNING, RMAppState.FINAL_SAVING,
RMAppEventType.KILL,
new FinalSavingTransition(
new KillAppAndAttemptTransition(), RMAppState.KILLED))
// Transitions from REMOVING state
.addTransition(RMAppState.REMOVING, RMAppState.FINISHING,
RMAppEventType.APP_REMOVED, new RMAppFinishingTransition())
.addTransition(RMAppState.REMOVING, RMAppState.FINISHED,
RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION)
.addTransition(RMAppState.REMOVING, RMAppState.KILLED,
RMAppEventType.KILL, new KillAppAndAttemptTransition())
// Transitions from FINAL_SAVING state
.addTransition(RMAppState.FINAL_SAVING,
EnumSet.of(RMAppState.FINISHING, RMAppState.FAILED,
RMAppState.KILLED, RMAppState.FINISHED), RMAppEventType.APP_UPDATE_SAVED,
new FinalStateSavedTransition())
.addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
RMAppEventType.ATTEMPT_FINISHED,
new AttemptFinishedAtFinalSavingTransition())
// ignorable transitions
.addTransition(RMAppState.REMOVING, RMAppState.REMOVING,
RMAppEventType.NODE_UPDATE)
.addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
EnumSet.of(RMAppEventType.NODE_UPDATE, RMAppEventType.KILL))
// Transitions from FINISHING state
.addTransition(RMAppState.FINISHING, RMAppState.FINISHED,
@ -201,7 +224,7 @@ public class RMAppImpl implements RMApp, Recoverable {
RMAppEventType.KILL, new KillAppAndAttemptTransition())
// ignorable transitions
.addTransition(RMAppState.FINISHING, RMAppState.FINISHING,
EnumSet.of(RMAppEventType.NODE_UPDATE, RMAppEventType.APP_REMOVED))
EnumSet.of(RMAppEventType.NODE_UPDATE))
// Transitions from FINISHED state
// ignorable transitions
@ -210,14 +233,12 @@ public class RMAppImpl implements RMApp, Recoverable {
RMAppEventType.NODE_UPDATE,
RMAppEventType.ATTEMPT_UNREGISTERED,
RMAppEventType.ATTEMPT_FINISHED,
RMAppEventType.KILL,
RMAppEventType.APP_REMOVED))
RMAppEventType.KILL))
// Transitions from FAILED state
// ignorable transitions
.addTransition(RMAppState.FAILED, RMAppState.FAILED,
EnumSet.of(RMAppEventType.KILL, RMAppEventType.NODE_UPDATE,
RMAppEventType.APP_SAVED, RMAppEventType.APP_REMOVED))
EnumSet.of(RMAppEventType.KILL, RMAppEventType.NODE_UPDATE))
// Transitions from KILLED state
// ignorable transitions
@ -227,8 +248,7 @@ public class RMAppImpl implements RMApp, Recoverable {
EnumSet.of(RMAppEventType.APP_ACCEPTED,
RMAppEventType.APP_REJECTED, RMAppEventType.KILL,
RMAppEventType.ATTEMPT_FINISHED, RMAppEventType.ATTEMPT_FAILED,
RMAppEventType.ATTEMPT_KILLED, RMAppEventType.NODE_UPDATE,
RMAppEventType.APP_SAVED, RMAppEventType.APP_REMOVED))
RMAppEventType.ATTEMPT_KILLED, RMAppEventType.NODE_UPDATE))
.installTopology();
@ -316,7 +336,6 @@ public class RMAppImpl implements RMApp, Recoverable {
@Override
public RMAppState getState() {
this.readLock.lock();
try {
return this.stateMachine.getCurrentState();
} finally {
@ -398,7 +417,7 @@ public class RMAppImpl implements RMApp, Recoverable {
case SUBMITTED:
case ACCEPTED:
case RUNNING:
case REMOVING:
case FINAL_SAVING:
return FinalApplicationStatus.UNDEFINED;
// finished without a proper final state is the same as failed
case FINISHING:
@ -586,8 +605,12 @@ public class RMAppImpl implements RMApp, Recoverable {
@Override
public void recover(RMState state) throws Exception{
ApplicationState appState = state.getApplicationState().get(getApplicationId());
this.recoveredFinalState = appState.getState();
LOG.info("Recovering app: " + getApplicationId() + " with " +
+ appState.getAttemptCount() + " attempts");
+ appState.getAttemptCount() + " attempts and final state = " + this.recoveredFinalState );
this.diagnostics.append(appState.getDiagnostics());
this.storedFinishTime = appState.getFinishTime();
this.startTime = appState.getStartTime();
for(int i=0; i<appState.getAttemptCount(); ++i) {
// create attempt
createNewAttempt(false);
@ -633,59 +656,194 @@ public class RMAppImpl implements RMApp, Recoverable {
};
}
private static final class StartAppAttemptTransition extends RMAppTransition {
public void transition(RMAppImpl app, RMAppEvent event) {
if (event.getType().equals(RMAppEventType.APP_SAVED)) {
assert app.getState().equals(RMAppState.NEW_SAVING);
RMAppStoredEvent storeEvent = (RMAppStoredEvent) event;
if(storeEvent.getStoredException() != null) {
// For HA this exception needs to be handled by giving up
// master status if we got fenced
LOG.error("Failed to store application: "
+ storeEvent.getApplicationId(),
storeEvent.getStoredException());
ExitUtil.terminate(1, storeEvent.getStoredException());
private static final class RMAppRecoveredTransition implements
MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
@Override
public RMAppState transition(RMAppImpl app, RMAppEvent event) {
if (app.recoveredFinalState != null) {
FINAL_TRANSITION.transition(app, event);
return app.recoveredFinalState;
}
// Directly call AttemptFailedTransition, since now we deem that an
// application fails because of RM restart as a normal AM failure.
// Do not recover unmanaged applications since current recovery
// mechanism of restarting attempts does not work for them.
// This will need to be changed in work preserving recovery in which
// RM will re-connect with the running AM's instead of restarting them
// In work-preserve restart, if attemptCount == maxAttempts, the job still
// needs to be recovered because the last attempt may still be running.
// As part of YARN-1210, we may return ACCECPTED state waiting for AM to
// reregister or fail and remove the following code.
return new AttemptFailedTransition(RMAppState.SUBMITTED).transition(app,
event);
}
}
private static final class StartAppAttemptTransition extends RMAppTransition {
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
RMAppNewSavedEvent storeEvent = (RMAppNewSavedEvent) event;
if (storeEvent.getStoredException() != null) {
// For HA this exception needs to be handled by giving up
// master status if we got fenced
LOG.error(
"Failed to store application: " + storeEvent.getApplicationId(),
storeEvent.getStoredException());
ExitUtil.terminate(1, storeEvent.getStoredException());
}
app.createNewAttempt(true);
};
}
private static final class RMAppFinishingTransition extends RMAppTransition {
private static final class FinalStateSavedTransition implements
MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
@SuppressWarnings({ "rawtypes", "unchecked" })
@Override
public RMAppState transition(RMAppImpl app, RMAppEvent event) {
RMAppUpdateSavedEvent storeEvent = (RMAppUpdateSavedEvent) event;
if (storeEvent.getUpdatedException() != null) {
LOG.error("Failed to update the final state of application"
+ storeEvent.getApplicationId(), storeEvent.getUpdatedException());
ExitUtil.terminate(1, storeEvent.getUpdatedException());
}
if (app.transitionTodo instanceof SingleArcTransition) {
((SingleArcTransition) app.transitionTodo).transition(app,
app.eventCausingFinalSaving);
} else if (app.transitionTodo instanceof MultipleArcTransition) {
((MultipleArcTransition) app.transitionTodo).transition(app,
app.eventCausingFinalSaving);
}
return app.targetedFinalState;
}
}
private static class AttemptFailedFinalStateSavedTransition extends
RMAppTransition {
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
if (event.getType().equals(RMAppEventType.APP_REMOVED)) {
RMAppRemovedEvent removeEvent = (RMAppRemovedEvent) event;
if (removeEvent.getRemovedException() != null) {
LOG.error(
"Failed to remove application: " + removeEvent.getApplicationId(),
removeEvent.getRemovedException());
ExitUtil.terminate(1, removeEvent.getRemovedException());
String msg = null;
if (event instanceof RMAppFailedAttemptEvent) {
msg = app.getAppAttemptFailedDiagnostics(event);
}
LOG.info(msg);
app.diagnostics.append(msg);
// Inform the node for app-finish
FINAL_TRANSITION.transition(app, event);
}
}
app.finishTime = System.currentTimeMillis();
private String getAppAttemptFailedDiagnostics(RMAppEvent event) {
String msg = null;
RMAppFailedAttemptEvent failedEvent = (RMAppFailedAttemptEvent) event;
if (this.submissionContext.getUnmanagedAM()) {
// RM does not manage the AM. Do not retry
msg = "Unmanaged application " + this.getApplicationId()
+ " failed due to " + failedEvent.getDiagnostics()
+ ". Failing the application.";
} else if (this.attempts.size() >= this.maxAppAttempts) {
msg = "Application " + this.getApplicationId() + " failed "
+ this.maxAppAttempts + " times due to "
+ failedEvent.getDiagnostics() + ". Failing the application.";
}
return msg;
}
private static final class RMAppSavingTransition extends RMAppTransition {
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
// If recovery is enabled then store the application information in a
// non-blocking call so make sure that RM has stored the information
// needed to restart the AM after RM restart without further client
// communication
LOG.info("Storing application with id " + app.applicationId);
app.rmContext.getStateStore().storeApplication(app);
app.rmContext.getStateStore().storeNewApplication(app);
}
}
private static final class RMAppRemovingTransition extends RMAppTransition {
private void rememberTargetTransitions(RMAppEvent event,
Object transitionToDo, RMAppState targetFinalState) {
transitionTodo = transitionToDo;
targetedFinalState = targetFinalState;
eventCausingFinalSaving = event;
}
private void rememberTargetTransitionsAndStoreState(RMAppEvent event,
Object transitionToDo, RMAppState targetFinalState,
RMAppState stateToBeStored) {
rememberTargetTransitions(event, transitionToDo, targetFinalState);
this.stateBeforeFinalSaving = getState();
this.storedFinishTime = System.currentTimeMillis();
LOG.info("Updating application " + this.applicationId
+ " with final state: " + this.targetedFinalState);
// we lost attempt_finished diagnostics in app, because attempt_finished
// diagnostics is sent after app final state is saved. Later on, we will
// create GetApplicationAttemptReport specifically for getting per attempt
// info.
String diags = null;
switch (event.getType()) {
case APP_REJECTED:
RMAppRejectedEvent rejectedEvent = (RMAppRejectedEvent) event;
diags = rejectedEvent.getMessage();
break;
case ATTEMPT_FINISHED:
RMAppFinishedAttemptEvent finishedEvent =
(RMAppFinishedAttemptEvent) event;
diags = finishedEvent.getDiagnostics();
break;
case ATTEMPT_FAILED:
RMAppFailedAttemptEvent failedEvent = (RMAppFailedAttemptEvent) event;
diags = getAppAttemptFailedDiagnostics(failedEvent);
break;
case KILL:
diags = getAppKilledDiagnostics();
break;
default:
break;
}
ApplicationState appState =
new ApplicationState(this.submitTime, this.startTime,
this.submissionContext, this.user, stateToBeStored, diags,
this.storedFinishTime);
this.rmContext.getStateStore().updateApplicationState(appState);
}
private static final class FinalSavingTransition extends RMAppTransition {
Object transitionToDo;
RMAppState targetedFinalState;
RMAppState stateToBeStored;
public FinalSavingTransition(Object transitionToDo,
RMAppState targetedFinalState) {
this(transitionToDo, targetedFinalState, targetedFinalState);
}
public FinalSavingTransition(Object transitionToDo,
RMAppState targetedFinalState, RMAppState stateToBeStored) {
this.transitionToDo = transitionToDo;
this.targetedFinalState = targetedFinalState;
this.stateToBeStored = stateToBeStored;
}
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
LOG.info("Removing application with id " + app.applicationId);
app.removeApplicationState();
app.previousStateAtRemoving = app.getState();
app.rememberTargetTransitionsAndStoreState(event, transitionToDo,
targetedFinalState, stateToBeStored);
}
}
private static class AttemptUnregisteredTransition extends RMAppTransition {
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
app.finishTime = app.storedFinishTime;
}
}
@ -698,6 +856,40 @@ public class RMAppImpl implements RMApp, Recoverable {
};
}
private static class AttemptFinishedAtFinalSavingTransition extends
RMAppTransition {
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
if (app.targetedFinalState.equals(RMAppState.FAILED)
|| app.targetedFinalState.equals(RMAppState.KILLED)) {
// Ignore Attempt_Finished event if we were supposed to reach FAILED
// FINISHED state
return;
}
// pass in the earlier attempt_unregistered event, as it is needed in
// AppFinishedFinalStateSavedTransition later on
app.rememberTargetTransitions(event,
new AppFinishedFinalStateSavedTransition(app.eventCausingFinalSaving),
RMAppState.FINISHED);
};
}
private static class AppFinishedFinalStateSavedTransition extends
RMAppTransition {
RMAppEvent attemptUnregistered;
public AppFinishedFinalStateSavedTransition(RMAppEvent attemptUnregistered) {
this.attemptUnregistered = attemptUnregistered;
}
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
new AttemptUnregisteredTransition().transition(app, attemptUnregistered);
FINISHED_TRANSITION.transition(app, event);
};
}
private static class AppKilledTransition extends FinalTransition {
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
@ -706,6 +898,10 @@ public class RMAppImpl implements RMApp, Recoverable {
};
}
private static String getAppKilledDiagnostics() {
return "Application killed by user.";
}
private static class KillAppAndAttemptTransition extends AppKilledTransition {
@SuppressWarnings("unchecked")
@Override
@ -741,12 +937,10 @@ public class RMAppImpl implements RMApp, Recoverable {
app.handler.handle(
new RMNodeCleanAppEvent(nodeId, app.applicationId));
}
if (app.getState() != RMAppState.FINISHING) {
app.finishTime = app.storedFinishTime;
if (app.finishTime == 0 ) {
app.finishTime = System.currentTimeMillis();
}
// application completely done and remove from state store.
app.removeApplicationState();
app.handler.handle(
new RMAppManagerEvent(app.applicationId,
RMAppManagerEventType.APP_COMPLETED));
@ -764,32 +958,15 @@ public class RMAppImpl implements RMApp, Recoverable {
@Override
public RMAppState transition(RMAppImpl app, RMAppEvent event) {
RMAppFailedAttemptEvent failedEvent = ((RMAppFailedAttemptEvent) event);
boolean retryApp = true;
String msg = null;
if (app.submissionContext.getUnmanagedAM()) {
// RM does not manage the AM. Do not retry
retryApp = false;
msg = "Unmanaged application " + app.getApplicationId()
+ " failed due to " + failedEvent.getDiagnostics()
+ ". Failing the application.";
} else if (app.attempts.size() >= app.maxAppAttempts) {
retryApp = false;
msg = "Application " + app.getApplicationId() + " failed "
+ app.maxAppAttempts + " times due to " + failedEvent.getDiagnostics()
+ ". Failing the application.";
}
if (retryApp) {
if (!app.submissionContext.getUnmanagedAM()
&& app.attempts.size() < app.maxAppAttempts) {
app.createNewAttempt(true);
return initialState;
} else {
LOG.info(msg);
app.diagnostics.append(msg);
// Inform the node for app-finish
FINAL_TRANSITION.transition(app, event);
return RMAppState.FAILED;
app.rememberTargetTransitionsAndStoreState(event,
new AttemptFailedFinalStateSavedTransition(), RMAppState.FAILED,
RMAppState.FAILED);
return RMAppState.FINAL_SAVING;
}
}
@ -814,9 +991,9 @@ public class RMAppImpl implements RMApp, Recoverable {
@Override
public YarnApplicationState createApplicationState() {
RMAppState rmAppState = getState();
// If App is in REMOVING state, return its previous state.
if (rmAppState.equals(RMAppState.REMOVING)) {
rmAppState = previousStateAtRemoving;
// If App is in FINAL_SAVING state, return its previous state.
if (rmAppState.equals(RMAppState.FINAL_SAVING)) {
rmAppState = stateBeforeFinalSaving;
}
switch (rmAppState) {
case NEW:
@ -840,11 +1017,4 @@ public class RMAppImpl implements RMApp, Recoverable {
throw new YarnRuntimeException("Unknown state passed!");
}
}
private void removeApplicationState(){
if (!isAppRemovalRequestSent) {
rmContext.getStateStore().removeApplication(this);
isAppRemovalRequestSent = true;
}
}
}

View File

@ -20,12 +20,12 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
import org.apache.hadoop.yarn.api.records.ApplicationId;
public class RMAppStoredEvent extends RMAppEvent {
public class RMAppNewSavedEvent extends RMAppEvent {
private final Exception storedException;
public RMAppStoredEvent(ApplicationId appId, Exception storedException) {
super(appId, RMAppEventType.APP_SAVED);
public RMAppNewSavedEvent(ApplicationId appId, Exception storedException) {
super(appId, RMAppEventType.APP_NEW_SAVED);
this.storedException = storedException;
}

View File

@ -24,7 +24,7 @@ public enum RMAppState {
SUBMITTED,
ACCEPTED,
RUNNING,
REMOVING,
FINAL_SAVING,
FINISHING,
FINISHED,
FAILED,

View File

@ -0,0 +1,36 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
import org.apache.hadoop.yarn.api.records.ApplicationId;
public class RMAppUpdateSavedEvent extends RMAppEvent {
private final Exception updatedException;
public RMAppUpdateSavedEvent(ApplicationId appId, Exception updatedException) {
super(appId, RMAppEventType.APP_UPDATE_SAVED);
this.updatedException = updatedException;
}
public Exception getUpdatedException() {
return updatedException;
}
}

View File

@ -41,7 +41,8 @@ public enum RMAppAttemptEventType {
CONTAINER_FINISHED,
// Source: RMStateStore
ATTEMPT_SAVED,
ATTEMPT_NEW_SAVED,
ATTEMPT_UPDATE_SAVED,
// Source: Scheduler
APP_REJECTED,

View File

@ -40,7 +40,6 @@ import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.http.HttpConfig;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
@ -82,8 +81,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStoredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
@ -142,7 +142,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private float progress = 0;
private String host = "N/A";
private int rpcPort;
private String origTrackingUrl = "N/A";
private String originalTrackingUrl = "N/A";
private String proxiedTrackingUrl = "N/A";
private long startTime = 0;
@ -157,6 +157,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private static final ExpiredTransition EXPIRED_TRANSITION =
new ExpiredTransition();
private RMAppAttemptEvent eventCausingFinalSaving;
private RMAppAttemptState targetedFinalState;
private RMAppAttemptState recoveredFinalState;
private Object transitionTodo;
private static final StateMachineFactory<RMAppAttemptImpl,
RMAppAttemptState,
RMAppAttemptEventType,
@ -169,68 +174,80 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
// Transitions from NEW State
.addTransition(RMAppAttemptState.NEW, RMAppAttemptState.SUBMITTED,
RMAppAttemptEventType.START, new AttemptStartedTransition())
.addTransition(RMAppAttemptState.NEW, RMAppAttemptState.KILLED,
.addTransition(RMAppAttemptState.NEW, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.KILL,
new BaseFinalTransition(RMAppAttemptState.KILLED))
.addTransition(RMAppAttemptState.NEW, RMAppAttemptState.FAILED,
new FinalSavingTransition(new BaseFinalTransition(
RMAppAttemptState.KILLED), RMAppAttemptState.KILLED))
.addTransition(RMAppAttemptState.NEW, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.REGISTERED,
new UnexpectedAMRegisteredTransition())
.addTransition(RMAppAttemptState.NEW, RMAppAttemptState.RECOVERED,
RMAppAttemptEventType.RECOVER)
new FinalSavingTransition(
new UnexpectedAMRegisteredTransition(), RMAppAttemptState.FAILED))
.addTransition( RMAppAttemptState.NEW,
EnumSet.of(RMAppAttemptState.FINISHED, RMAppAttemptState.KILLED,
RMAppAttemptState.FAILED, RMAppAttemptState.RECOVERED),
RMAppAttemptEventType.RECOVER, new AttemptRecoveredTransition())
// Transitions from SUBMITTED state
.addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.FAILED,
RMAppAttemptEventType.APP_REJECTED, new AppRejectedTransition())
.addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.APP_REJECTED,
new FinalSavingTransition(new AppRejectedTransition(),
RMAppAttemptState.FAILED))
.addTransition(RMAppAttemptState.SUBMITTED,
EnumSet.of(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING,
RMAppAttemptState.SCHEDULED),
RMAppAttemptEventType.APP_ACCEPTED,
new ScheduleTransition())
.addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.KILLED,
.addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.KILL,
new BaseFinalTransition(RMAppAttemptState.KILLED))
.addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.FAILED,
new FinalSavingTransition(new BaseFinalTransition(
RMAppAttemptState.KILLED), RMAppAttemptState.KILLED))
.addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.REGISTERED,
new UnexpectedAMRegisteredTransition())
new FinalSavingTransition(
new UnexpectedAMRegisteredTransition(), RMAppAttemptState.FAILED))
// Transitions from SCHEDULED State
.addTransition(RMAppAttemptState.SCHEDULED,
RMAppAttemptState.ALLOCATED_SAVING,
RMAppAttemptEventType.CONTAINER_ALLOCATED,
new AMContainerAllocatedTransition())
.addTransition(RMAppAttemptState.SCHEDULED, RMAppAttemptState.KILLED,
.addTransition(RMAppAttemptState.SCHEDULED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.KILL,
new BaseFinalTransition(RMAppAttemptState.KILLED))
new FinalSavingTransition(new BaseFinalTransition(
RMAppAttemptState.KILLED), RMAppAttemptState.KILLED))
// Transitions from ALLOCATED_SAVING State
.addTransition(RMAppAttemptState.ALLOCATED_SAVING,
RMAppAttemptState.ALLOCATED,
RMAppAttemptEventType.ATTEMPT_SAVED, new AttemptStoredTransition())
RMAppAttemptEventType.ATTEMPT_NEW_SAVED, new AttemptStoredTransition())
.addTransition(RMAppAttemptState.ALLOCATED_SAVING,
RMAppAttemptState.ALLOCATED_SAVING,
RMAppAttemptEventType.CONTAINER_ACQUIRED,
new ContainerAcquiredTransition())
// App could be killed by the client. So need to handle this.
.addTransition(RMAppAttemptState.ALLOCATED_SAVING,
RMAppAttemptState.KILLED,
RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.KILL,
new BaseFinalTransition(RMAppAttemptState.KILLED))
new FinalSavingTransition(new BaseFinalTransition(
RMAppAttemptState.KILLED), RMAppAttemptState.KILLED))
// Transitions from LAUNCHED_UNMANAGED_SAVING State
.addTransition(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING,
RMAppAttemptState.LAUNCHED,
RMAppAttemptEventType.ATTEMPT_SAVED,
RMAppAttemptEventType.ATTEMPT_NEW_SAVED,
new UnmanagedAMAttemptSavedTransition())
// attempt should not try to register in this state
.addTransition(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING,
RMAppAttemptState.FAILED,
RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.REGISTERED,
new UnexpectedAMRegisteredTransition())
new FinalSavingTransition(
new UnexpectedAMRegisteredTransition(), RMAppAttemptState.FAILED))
// App could be killed by the client. So need to handle this.
.addTransition(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING,
RMAppAttemptState.KILLED,
RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.KILL,
new BaseFinalTransition(RMAppAttemptState.KILLED))
new FinalSavingTransition(new BaseFinalTransition(
RMAppAttemptState.KILLED), RMAppAttemptState.KILLED))
// Transitions from ALLOCATED State
.addTransition(RMAppAttemptState.ALLOCATED,
@ -239,32 +256,40 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
new ContainerAcquiredTransition())
.addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.LAUNCHED,
RMAppAttemptEventType.LAUNCHED, new AMLaunchedTransition())
.addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.FAILED,
RMAppAttemptEventType.LAUNCH_FAILED, new LaunchFailedTransition())
.addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.KILLED,
RMAppAttemptEventType.KILL, new KillAllocatedAMTransition())
.addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.LAUNCH_FAILED,
new FinalSavingTransition(new LaunchFailedTransition(),
RMAppAttemptState.FAILED))
.addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.KILL,
new FinalSavingTransition(
new KillAllocatedAMTransition(), RMAppAttemptState.KILLED))
.addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.FAILED,
.addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.CONTAINER_FINISHED,
new AMContainerCrashedTransition())
new FinalSavingTransition(
new AMContainerCrashedTransition(), RMAppAttemptState.FAILED))
// Transitions from LAUNCHED State
.addTransition(RMAppAttemptState.LAUNCHED, RMAppAttemptState.RUNNING,
RMAppAttemptEventType.REGISTERED, new AMRegisteredTransition())
.addTransition(RMAppAttemptState.LAUNCHED, RMAppAttemptState.FAILED,
.addTransition(RMAppAttemptState.LAUNCHED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.CONTAINER_FINISHED,
new AMContainerCrashedTransition())
new FinalSavingTransition(
new AMContainerCrashedTransition(), RMAppAttemptState.FAILED))
.addTransition(
RMAppAttemptState.LAUNCHED, RMAppAttemptState.FAILED,
RMAppAttemptState.LAUNCHED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.EXPIRE,
EXPIRED_TRANSITION)
.addTransition(RMAppAttemptState.LAUNCHED, RMAppAttemptState.KILLED,
new FinalSavingTransition(EXPIRED_TRANSITION,
RMAppAttemptState.FAILED))
.addTransition(RMAppAttemptState.LAUNCHED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.KILL,
new FinalTransition(RMAppAttemptState.KILLED))
new FinalSavingTransition(new FinalTransition(
RMAppAttemptState.KILLED), RMAppAttemptState.KILLED))
// Transitions from RUNNING State
.addTransition(RMAppAttemptState.RUNNING,
EnumSet.of(RMAppAttemptState.FINISHING, RMAppAttemptState.FINISHED),
EnumSet.of(RMAppAttemptState.FINAL_SAVING, RMAppAttemptState.FINISHED),
RMAppAttemptEventType.UNREGISTERED, new AMUnregisteredTransition())
.addTransition(RMAppAttemptState.RUNNING, RMAppAttemptState.RUNNING,
RMAppAttemptEventType.STATUS_UPDATE, new StatusUpdateTransition())
@ -276,17 +301,41 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
new ContainerAcquiredTransition())
.addTransition(
RMAppAttemptState.RUNNING,
EnumSet.of(RMAppAttemptState.RUNNING, RMAppAttemptState.FAILED),
EnumSet.of(RMAppAttemptState.RUNNING, RMAppAttemptState.FINAL_SAVING),
RMAppAttemptEventType.CONTAINER_FINISHED,
new ContainerFinishedTransition())
.addTransition(
RMAppAttemptState.RUNNING, RMAppAttemptState.FAILED,
RMAppAttemptState.RUNNING, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.EXPIRE,
EXPIRED_TRANSITION)
new FinalSavingTransition(EXPIRED_TRANSITION,
RMAppAttemptState.FAILED))
.addTransition(
RMAppAttemptState.RUNNING, RMAppAttemptState.KILLED,
RMAppAttemptState.RUNNING, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.KILL,
new FinalTransition(RMAppAttemptState.KILLED))
new FinalSavingTransition(new FinalTransition(
RMAppAttemptState.KILLED), RMAppAttemptState.KILLED))
// Transitions from FINAL_SAVING State
.addTransition(RMAppAttemptState.FINAL_SAVING,
EnumSet.of(RMAppAttemptState.FINISHING, RMAppAttemptState.FAILED,
RMAppAttemptState.KILLED, RMAppAttemptState.FINISHED),
RMAppAttemptEventType.ATTEMPT_UPDATE_SAVED,
new FinalStateSavedTransition())
.addTransition(RMAppAttemptState.FINAL_SAVING, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.CONTAINER_FINISHED,
new ContainerFinishedAtFinalSavingTransition())
.addTransition(RMAppAttemptState.FINAL_SAVING, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.EXPIRE,
new AMExpiredAtFinalSavingTransition())
.addTransition(RMAppAttemptState.FINAL_SAVING, RMAppAttemptState.FINAL_SAVING,
EnumSet.of(
RMAppAttemptEventType.UNREGISTERED,
RMAppAttemptEventType.STATUS_UPDATE,
// should be fixed to reject container allocate request at Final
// Saving in scheduler
RMAppAttemptEventType.CONTAINER_ALLOCATED,
RMAppAttemptEventType.CONTAINER_ACQUIRED,
RMAppAttemptEventType.KILL))
// Transitions from FAILED State
.addTransition(
@ -338,7 +387,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
RMAppAttemptEventType.EXPIRE,
RMAppAttemptEventType.REGISTERED,
RMAppAttemptEventType.CONTAINER_ALLOCATED,
RMAppAttemptEventType.ATTEMPT_SAVED,
RMAppAttemptEventType.CONTAINER_FINISHED,
RMAppAttemptEventType.UNREGISTERED,
RMAppAttemptEventType.KILL,
@ -357,7 +405,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
RMAppAttemptEventType.REGISTERED,
RMAppAttemptEventType.CONTAINER_ALLOCATED,
RMAppAttemptEventType.CONTAINER_ACQUIRED,
RMAppAttemptEventType.ATTEMPT_SAVED,
RMAppAttemptEventType.ATTEMPT_NEW_SAVED,
RMAppAttemptEventType.CONTAINER_FINISHED,
RMAppAttemptEventType.UNREGISTERED,
RMAppAttemptEventType.KILL,
@ -444,7 +492,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
this.readLock.lock();
try {
return (getSubmissionContext().getUnmanagedAM()) ?
this.origTrackingUrl : this.proxiedTrackingUrl;
this.originalTrackingUrl : this.proxiedTrackingUrl;
} finally {
this.readLock.unlock();
}
@ -454,7 +502,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
public String getOriginalTrackingUrl() {
this.readLock.lock();
try {
return this.origTrackingUrl;
return this.originalTrackingUrl;
} finally {
this.readLock.unlock();
}
@ -490,10 +538,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
}
private void setTrackingUrlToRMAppPage() {
origTrackingUrl = pjoin(
originalTrackingUrl = pjoin(
WebAppUtils.getResolvedRMWebAppURLWithoutScheme(conf),
"cluster", "app", getAppAttemptId().getApplicationId());
proxiedTrackingUrl = origTrackingUrl;
proxiedTrackingUrl = originalTrackingUrl;
}
// This is only used for RMStateStore. Normal operation must invoke the secret
@ -539,16 +587,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
}
}
public void setDiagnostics(String message) {
this.writeLock.lock();
try {
this.diagnostics.append(message);
} finally {
this.writeLock.unlock();
}
}
@Override
public float getProgress() {
this.readLock.lock();
@ -673,17 +711,24 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
}
@Override
public void recover(RMState state) throws Exception{
public void recover(RMState state) throws Exception {
ApplicationState appState =
state.getApplicationState().get(getAppAttemptId().getApplicationId());
ApplicationAttemptState attemptState = appState.getAttempt(getAppAttemptId());
ApplicationAttemptState attemptState =
appState.getAttempt(getAppAttemptId());
assert attemptState != null;
LOG.info("Recovered attempt: AppId: "
+ getAppAttemptId().getApplicationId() + " AttemptId: "
+ getAppAttemptId() + " MasterContainer: " + masterContainer);
diagnostics.append("Attempt recovered after RM restart");
diagnostics.append(attemptState.getDiagnostics());
setMasterContainer(attemptState.getMasterContainer());
recoverAppAttemptCredentials(attemptState.getAppAttemptCredentials());
LOG.info("Recovered attempt: AppId: " + getAppAttemptId().getApplicationId()
+ " AttemptId: " + getAppAttemptId()
+ " MasterContainer: " + masterContainer);
setDiagnostics("Attempt recovered after RM restart");
this.recoveredFinalState = attemptState.getState();
this.originalTrackingUrl = attemptState.getFinalTrackingUrl();
this.proxiedTrackingUrl = generateProxyUriWithScheme(originalTrackingUrl);
this.finalStatus = attemptState.getFinalApplicationStatus();
this.startTime = attemptState.getStartTime();
handle(new RMAppAttemptEvent(getAppAttemptId(),
RMAppAttemptEventType.RECOVER));
}
@ -763,7 +808,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
// Save the diagnostic message
String message = rejectedEvent.getMessage();
appAttempt.setDiagnostics(message);
appAttempt.diagnostics.append(message);
// Send the rejection event to app
appAttempt.eventHandler.handle(
@ -810,10 +855,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
}
return RMAppAttemptState.SCHEDULED;
} else {
// RM not allocating container. AM is self launched.
RMStateStore store = appAttempt.rmContext.getStateStore();
// save state and then go to LAUNCHED state
appAttempt.storeAttempt(store);
appAttempt.storeAttempt();
return RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING;
}
}
@ -838,8 +881,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
0));
appAttempt.getSubmissionContext().setResource(
appAttempt.getMasterContainer().getResource());
RMStateStore store = appAttempt.rmContext.getStateStore();
appAttempt.storeAttempt(store);
appAttempt.storeAttempt();
}
}
@ -852,6 +894,134 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
}
}
private static class AttemptRecoveredTransition
implements
MultipleArcTransition<RMAppAttemptImpl, RMAppAttemptEvent, RMAppAttemptState> {
@Override
public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) {
if (appAttempt.recoveredFinalState != null) {
appAttempt.progress = 1.0f;
return appAttempt.recoveredFinalState;
} else {
return RMAppAttemptState.RECOVERED;
}
}
}
private void rememberTargetTransitions(RMAppAttemptEvent event,
Object transitionToDo, RMAppAttemptState targetFinalState) {
transitionTodo = transitionToDo;
targetedFinalState = targetFinalState;
eventCausingFinalSaving = event;
}
private void rememberTargetTransitionsAndStoreState(RMAppAttemptEvent event,
Object transitionToDo, RMAppAttemptState targetFinalState,
RMAppAttemptState stateToBeStored) {
rememberTargetTransitions(event, transitionToDo, targetFinalState);
// As of today, finalState, diagnostics, final-tracking-url and
// finalAppStatus are the only things that we store into the StateStore
// AFTER the initial saving on app-attempt-start
// These fields can be visible from outside only after they are saved in
// StateStore
String diags = null;
String finalTrackingUrl = null;
FinalApplicationStatus finalStatus = null;
switch (event.getType()) {
case APP_REJECTED:
RMAppAttemptRejectedEvent rejectedEvent =
(RMAppAttemptRejectedEvent) event;
diags = rejectedEvent.getMessage();
break;
case LAUNCH_FAILED:
RMAppAttemptLaunchFailedEvent launchFaileEvent =
(RMAppAttemptLaunchFailedEvent) event;
diags = launchFaileEvent.getMessage();
break;
case REGISTERED:
diags = getUnexpectedAMRegisteredDiagnostics();
break;
case UNREGISTERED:
RMAppAttemptUnregistrationEvent unregisterEvent =
(RMAppAttemptUnregistrationEvent) event;
diags = unregisterEvent.getDiagnostics();
finalTrackingUrl = sanitizeTrackingUrl(unregisterEvent.getFinalTrackingUrl());
finalStatus = unregisterEvent.getFinalApplicationStatus();
break;
case CONTAINER_FINISHED:
RMAppAttemptContainerFinishedEvent finishEvent =
(RMAppAttemptContainerFinishedEvent) event;
diags = getAMContainerCrashedDiagnostics(finishEvent);
break;
case KILL:
break;
case EXPIRE:
diags = getAMExpiredDiagnostics(event);
break;
default:
break;
}
RMStateStore rmStore = rmContext.getStateStore();
ApplicationAttemptState attemptState =
new ApplicationAttemptState(applicationAttemptId, getMasterContainer(),
rmStore.getCredentialsFromAppAttempt(this), startTime,
stateToBeStored, finalTrackingUrl, diags, finalStatus);
LOG.info("Updating application attempt " + applicationAttemptId
+ " with final state: " + targetedFinalState);
rmStore.updateApplicationAttemptState(attemptState);
}
private static class FinalSavingTransition extends BaseTransition {
Object transitionToDo;
RMAppAttemptState targetedFinalState;
public FinalSavingTransition(Object transitionToDo,
RMAppAttemptState targetedFinalState) {
this.transitionToDo = transitionToDo;
this.targetedFinalState = targetedFinalState;
}
@Override
public void transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
// For cases Killed/Failed, targetedFinalState is the same as the state to
// be stored
appAttempt.rememberTargetTransitionsAndStoreState(event, transitionToDo,
targetedFinalState, targetedFinalState);
}
}
private static class FinalStateSavedTransition implements
MultipleArcTransition<RMAppAttemptImpl, RMAppAttemptEvent, RMAppAttemptState> {
@Override
public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) {
RMAppAttemptUpdateSavedEvent storeEvent = (RMAppAttemptUpdateSavedEvent) event;
if (storeEvent.getUpdatedException() != null) {
LOG.error("Failed to update the final state of application attempt: "
+ storeEvent.getApplicationAttemptId(),
storeEvent.getUpdatedException());
ExitUtil.terminate(1, storeEvent.getUpdatedException());
}
RMAppAttemptEvent causeEvent = appAttempt.eventCausingFinalSaving;
if (appAttempt.transitionTodo instanceof SingleArcTransition) {
((SingleArcTransition) appAttempt.transitionTodo).transition(
appAttempt, causeEvent);
} else if (appAttempt.transitionTodo instanceof MultipleArcTransition) {
((MultipleArcTransition) appAttempt.transitionTodo).transition(
appAttempt, causeEvent);
}
return appAttempt.targetedFinalState;
}
}
private static class BaseFinalTransition extends BaseTransition {
private final RMAppAttemptState finalAttemptState;
@ -998,15 +1168,20 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
= (RMAppAttemptRegistrationEvent) event;
appAttempt.host = registrationEvent.getHost();
appAttempt.rpcPort = registrationEvent.getRpcport();
appAttempt.origTrackingUrl =
appAttempt.originalTrackingUrl =
sanitizeTrackingUrl(registrationEvent.getTrackingurl());
appAttempt.proxiedTrackingUrl =
appAttempt.generateProxyUriWithScheme(appAttempt.origTrackingUrl);
appAttempt.generateProxyUriWithScheme(appAttempt.originalTrackingUrl);
// Let the app know
appAttempt.eventHandler.handle(new RMAppEvent(appAttempt
.getAppAttemptId().getApplicationId(),
RMAppEventType.ATTEMPT_REGISTERED));
// TODO:FIXME: Note for future. Unfortunately we only do a state-store
// write at AM launch time, so we don't save the AM's tracking URL anywhere
// as that would mean an extra state-store write. For now, we hope that in
// work-preserving restart, AMs are forced to reregister.
}
}
@ -1029,17 +1204,24 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
appAttempt.getAppAttemptId());
// Setup diagnostic message
ContainerStatus status = finishEvent.getContainerStatus();
appAttempt.diagnostics.append("AM Container for " +
appAttempt.getAppAttemptId() + " exited with " +
" exitCode: " + status.getExitStatus() +
" due to: " + status.getDiagnostics() + "." +
"Failing this attempt.");
appAttempt.diagnostics
.append(getAMContainerCrashedDiagnostics(finishEvent));
// Tell the app, scheduler
super.transition(appAttempt, finishEvent);
}
}
private static String getAMContainerCrashedDiagnostics(
RMAppAttemptContainerFinishedEvent finishEvent) {
ContainerStatus status = finishEvent.getContainerStatus();
String diagnostics =
"AM Container for " + finishEvent.getApplicationAttemptId()
+ " exited with " + " exitCode: " + status.getExitStatus()
+ " due to: " + status.getDiagnostics() + "."
+ "Failing this attempt.";
return diagnostics;
}
private static class FinalTransition extends BaseFinalTransition {
public FinalTransition(RMAppAttemptState finalAttemptState) {
@ -1055,7 +1237,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
// Tell the app and the scheduler
super.transition(appAttempt, event);
// UnRegister from AMLivelinessMonitor
// UnRegister from AMLivelinessMonitor. Perhaps for
// FAILING/KILLED/UnManaged AMs
appAttempt.rmContext.getAMLivelinessMonitor().unregister(
appAttempt.getAppAttemptId());
appAttempt.rmContext.getAMFinishingMonitor().unregister(
@ -1078,12 +1261,18 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
@Override
public void transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) {
appAttempt.diagnostics.append("ApplicationMaster for attempt " +
appAttempt.getAppAttemptId() + " timed out");
appAttempt.diagnostics.append(getAMExpiredDiagnostics(event));
super.transition(appAttempt, event);
}
}
private static String getAMExpiredDiagnostics(RMAppAttemptEvent event) {
String diag =
"ApplicationMaster for attempt " + event.getApplicationAttemptId()
+ " timed out";
return diag;
}
private static class UnexpectedAMRegisteredTransition extends
BaseFinalTransition {
@ -1094,13 +1283,16 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
@Override
public void transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
assert appAttempt.submissionContext.getUnmanagedAM();
appAttempt
.setDiagnostics("Unmanaged AM must register after AM attempt reaches LAUNCHED state.");
appAttempt.diagnostics.append(getUnexpectedAMRegisteredDiagnostics());
super.transition(appAttempt, event);
}
}
private static String getUnexpectedAMRegisteredDiagnostics() {
return "Unmanaged AM must register after AM attempt reaches LAUNCHED state.";
}
private static final class StatusUpdateTransition extends
BaseTransition {
@Override
@ -1125,38 +1317,62 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
@Override
public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) {
ApplicationAttemptId appAttemptId = appAttempt.getAppAttemptId();
appAttempt.rmContext.getAMLivelinessMonitor().unregister(appAttemptId);
appAttempt.progress = 1.0f;
RMAppAttemptUnregistrationEvent unregisterEvent
= (RMAppAttemptUnregistrationEvent) event;
appAttempt.diagnostics.append(unregisterEvent.getDiagnostics());
appAttempt.origTrackingUrl =
sanitizeTrackingUrl(unregisterEvent.getTrackingUrl());
appAttempt.proxiedTrackingUrl =
appAttempt.generateProxyUriWithScheme(appAttempt.origTrackingUrl);
appAttempt.finalStatus = unregisterEvent.getFinalApplicationStatus();
// Tell the app
if (appAttempt.getSubmissionContext().getUnmanagedAM()) {
// Unmanaged AMs have no container to wait for, so they skip
// the FINISHING state and go straight to FINISHED.
appAttempt.updateInfoOnAMUnregister(event);
new FinalTransition(RMAppAttemptState.FINISHED).transition(
appAttempt, event);
return RMAppAttemptState.FINISHED;
}
appAttempt.rmContext.getAMFinishingMonitor().register(appAttemptId);
// Saving the attempt final state
appAttempt.rememberTargetTransitionsAndStoreState(event,
new FinalStateSavedAfterAMUnregisterTransition(),
RMAppAttemptState.FINISHING, RMAppAttemptState.FINISHED);
ApplicationId applicationId =
appAttempt.getAppAttemptId().getApplicationId();
appAttempt.eventHandler.handle(
new RMAppEvent(applicationId, RMAppEventType.ATTEMPT_UNREGISTERED));
return RMAppAttemptState.FINISHING;
// Tell the app immediately that AM is unregistering so that app itself
// can save its state as soon as possible. Whether we do it like this, or
// we wait till AppAttempt is saved, it doesn't make any difference on the
// app side w.r.t failure conditions. The only event going out of
// AppAttempt to App after this point of time is AM/AppAttempt Finished.
appAttempt.eventHandler.handle(new RMAppEvent(applicationId,
RMAppEventType.ATTEMPT_UNREGISTERED));
return RMAppAttemptState.FINAL_SAVING;
}
}
private static class FinalStateSavedAfterAMUnregisterTransition extends
BaseTransition {
@Override
public void
transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
// Unregister from the AMlivenessMonitor and register with AMFinishingMonitor
appAttempt.rmContext.getAMLivelinessMonitor().unregister(
appAttempt.applicationAttemptId);
appAttempt.rmContext.getAMFinishingMonitor().register(
appAttempt.applicationAttemptId);
// Do not make any more changes to this transition code. Make all changes
// to the following method. Unless you are absolutely sure that you have
// stuff to do that shouldn't be used by the callers of the following
// method.
appAttempt.updateInfoOnAMUnregister(event);
}
}
private void updateInfoOnAMUnregister(RMAppAttemptEvent event) {
progress = 1.0f;
RMAppAttemptUnregistrationEvent unregisterEvent =
(RMAppAttemptUnregistrationEvent) event;
diagnostics.append(unregisterEvent.getDiagnostics());
originalTrackingUrl = sanitizeTrackingUrl(unregisterEvent.getFinalTrackingUrl());
proxiedTrackingUrl = generateProxyUriWithScheme(originalTrackingUrl);
finalStatus = unregisterEvent.getFinalApplicationStatus();
}
private static final class ContainerAcquiredTransition extends
BaseTransition {
@Override
@ -1186,25 +1402,33 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
if (appAttempt.masterContainer != null
&& appAttempt.masterContainer.getId().equals(
containerStatus.getContainerId())) {
// Remember the follow up transition and save the final attempt state.
appAttempt.rememberTargetTransitionsAndStoreState(event,
new ContainerFinishedFinalStateSavedTransition(),
RMAppAttemptState.FAILED, RMAppAttemptState.FAILED);
return RMAppAttemptState.FINAL_SAVING;
}
// Normal container.Put it in completedcontainers list
appAttempt.justFinishedContainers.add(containerStatus);
return RMAppAttemptState.RUNNING;
}
}
private static class ContainerFinishedFinalStateSavedTransition extends
BaseTransition {
@Override
public void
transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
RMAppAttemptContainerFinishedEvent containerFinishedEvent =
(RMAppAttemptContainerFinishedEvent) event;
// container associated with AM. must not be unmanaged
assert appAttempt.submissionContext.getUnmanagedAM() == false;
// Setup diagnostic message
appAttempt.diagnostics.append("AM Container for " +
appAttempt.getAppAttemptId() + " exited with " +
" exitCode: " + containerStatus.getExitStatus() +
" due to: " + containerStatus.getDiagnostics() + "." +
"Failing this attempt.");
new FinalTransition(RMAppAttemptState.FAILED).transition(
appAttempt, containerFinishedEvent);
return RMAppAttemptState.FAILED;
}
// Normal container.
// Put it in completedcontainers list
appAttempt.justFinishedContainers.add(containerStatus);
return RMAppAttemptState.RUNNING;
appAttempt.diagnostics
.append(getAMContainerCrashedDiagnostics(containerFinishedEvent));
new FinalTransition(RMAppAttemptState.FAILED).transition(appAttempt,
event);
}
}
@ -1228,13 +1452,83 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
appAttempt, containerFinishedEvent);
return RMAppAttemptState.FINISHED;
}
// Normal container.
appAttempt.justFinishedContainers.add(containerStatus);
return RMAppAttemptState.FINISHING;
}
}
private static class ContainerFinishedAtFinalSavingTransition extends
BaseTransition {
@Override
public void
transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
RMAppAttemptContainerFinishedEvent containerFinishedEvent =
(RMAppAttemptContainerFinishedEvent) event;
ContainerStatus containerStatus =
containerFinishedEvent.getContainerStatus();
// If this is the AM container, it means the AM container is finished,
// but we are not yet acknowledged that the final state has been saved.
// Thus, we still return FINAL_SAVING state here.
if (appAttempt.masterContainer.getId().equals(
containerStatus.getContainerId())) {
if (appAttempt.targetedFinalState.equals(RMAppAttemptState.FAILED)
|| appAttempt.targetedFinalState.equals(RMAppAttemptState.KILLED)) {
// ignore Container_Finished Event if we were supposed to reach
// FAILED/KILLED state.
return;
}
// pass in the earlier AMUnregistered Event also, as this is needed for
// AMFinishedAfterFinalSavingTransition later on
appAttempt.rememberTargetTransitions(event,
new AMFinishedAfterFinalSavingTransition(
appAttempt.eventCausingFinalSaving), RMAppAttemptState.FINISHED);
return;
}
// Normal container.
appAttempt.justFinishedContainers.add(containerStatus);
}
}
private static class AMFinishedAfterFinalSavingTransition extends
BaseTransition {
RMAppAttemptEvent amUnregisteredEvent;
public AMFinishedAfterFinalSavingTransition(
RMAppAttemptEvent amUnregisteredEvent) {
this.amUnregisteredEvent = amUnregisteredEvent;
}
@Override
public void
transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
appAttempt.updateInfoOnAMUnregister(amUnregisteredEvent);
new FinalTransition(RMAppAttemptState.FINISHED).transition(appAttempt,
event);
}
}
private static class AMExpiredAtFinalSavingTransition extends
BaseTransition {
@Override
public void
transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
if (appAttempt.targetedFinalState.equals(RMAppAttemptState.FAILED)
|| appAttempt.targetedFinalState.equals(RMAppAttemptState.KILLED)) {
// ignore Container_Finished Event if we were supposed to reach
// FAILED/KILLED state.
return;
}
// pass in the earlier AMUnregistered Event also, as this is needed for
// AMFinishedAfterFinalSavingTransition later on
appAttempt.rememberTargetTransitions(event,
new AMFinishedAfterFinalSavingTransition(
appAttempt.eventCausingFinalSaving), RMAppAttemptState.FINISHED);
}
}
@Override
public long getStartTime() {
this.readLock.lock();
@ -1256,7 +1550,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
}
private void checkAttemptStoreError(RMAppAttemptEvent event) {
RMAppAttemptStoredEvent storeEvent = (RMAppAttemptStoredEvent) event;
RMAppAttemptNewSavedEvent storeEvent = (RMAppAttemptNewSavedEvent) event;
if(storeEvent.getStoredException() != null)
{
// This needs to be handled for HA and give up master status if we got
@ -1267,7 +1561,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
}
}
private void storeAttempt(RMStateStore store) {
private void storeAttempt() {
// store attempt data in a non-blocking manner to prevent dispatcher
// thread starvation and wait for state to be saved
LOG.info("Storing attempt: AppId: " +
@ -1275,7 +1569,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
+ " AttemptId: " +
getAppAttemptId()
+ " MasterContainer: " + masterContainer);
store.storeApplicationAttempt(this);
rmContext.getStateStore().storeNewApplicationAttempt(this);
}
private void removeCredentials(RMAppAttemptImpl appAttempt) {

View File

@ -20,5 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
public enum RMAppAttemptState {
NEW, SUBMITTED, SCHEDULED, ALLOCATED, LAUNCHED, FAILED, RUNNING, FINISHING,
FINISHED, KILLED, ALLOCATED_SAVING, LAUNCHED_UNMANAGED_SAVING, RECOVERED
FINISHED, KILLED, ALLOCATED_SAVING, LAUNCHED_UNMANAGED_SAVING, RECOVERED,
FINAL_SAVING
}

View File

@ -22,13 +22,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
public class RMAppAttemptStoredEvent extends RMAppAttemptEvent {
public class RMAppAttemptNewSavedEvent extends RMAppAttemptEvent {
final Exception storedException;
public RMAppAttemptStoredEvent(ApplicationAttemptId appAttemptId,
public RMAppAttemptNewSavedEvent(ApplicationAttemptId appAttemptId,
Exception storedException) {
super(appAttemptId, RMAppAttemptEventType.ATTEMPT_SAVED);
super(appAttemptId, RMAppAttemptEventType.ATTEMPT_NEW_SAVED);
this.storedException = storedException;
}

View File

@ -25,20 +25,20 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
public class RMAppAttemptUnregistrationEvent extends RMAppAttemptEvent {
private final String trackingUrl;
private final String finalTrackingUrl;
private final FinalApplicationStatus finalStatus;
private final String diagnostics;
public RMAppAttemptUnregistrationEvent(ApplicationAttemptId appAttemptId,
String trackingUrl, FinalApplicationStatus finalStatus, String diagnostics) {
super(appAttemptId, RMAppAttemptEventType.UNREGISTERED);
this.trackingUrl = trackingUrl;
this.finalTrackingUrl = trackingUrl;
this.finalStatus = finalStatus;
this.diagnostics = diagnostics;
}
public String getTrackingUrl() {
return this.trackingUrl;
public String getFinalTrackingUrl() {
return this.finalTrackingUrl;
}
public FinalApplicationStatus getFinalApplicationStatus() {

View File

@ -0,0 +1,38 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
public class RMAppAttemptUpdateSavedEvent extends RMAppAttemptEvent {
final Exception updatedException;
public RMAppAttemptUpdateSavedEvent(ApplicationAttemptId appAttemptId,
Exception updatedException) {
super(appAttemptId, RMAppAttemptEventType.ATTEMPT_UPDATE_SAVED);
this.updatedException = updatedException;
}
public Exception getUpdatedException() {
return updatedException;
}
}

View File

@ -202,6 +202,12 @@ public class MockAM {
final FinishApplicationMasterRequest req =
FinishApplicationMasterRequest.newInstance(
FinalApplicationStatus.SUCCEEDED, "", "");
unregisterAppAttempt(req);
}
public void unregisterAppAttempt(final FinishApplicationMasterRequest req)
throws Exception {
waitForState(RMAppAttemptState.RUNNING);
UserGroupInformation ugi =
UserGroupInformation.createRemoteUser(attemptId.toString());
Token<AMRMTokenIdentifier> token =
@ -216,4 +222,8 @@ public class MockAM {
}
});
}
public ApplicationAttemptId getApplicationAttemptId() {
return this.attemptId;
}
}

View File

@ -23,6 +23,7 @@ import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -41,16 +42,24 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
import org.apache.hadoop.yarn.api.records.AMCommand;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
@ -90,13 +99,12 @@ public class TestRMRestart {
UserGroupInformation.setConfiguration(conf);
conf.set(YarnConfiguration.RECOVERY_ENABLED, "true");
conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
rmAddr = new InetSocketAddress("localhost", 8032);
Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);
}
@Test (timeout=180000)
public void testRMRestart() throws Exception {
Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
@ -122,7 +130,7 @@ public class TestRMRestart {
nm1.registerNode();
nm2.registerNode(); // nm2 will not heartbeat with RM1
// create app that will not be saved because it will finish
// create app that will finish and the final state should be saved.
RMApp app0 = rm1.submitApp(200);
RMAppAttempt attempt0 = app0.getCurrentAppAttempt();
// spot check that app is saved
@ -130,13 +138,7 @@ public class TestRMRestart {
nm1.nodeHeartbeat(true);
MockAM am0 = rm1.sendAMLaunched(attempt0.getAppAttemptId());
am0.registerAppAttempt();
am0.unregisterAppAttempt();
nm1.nodeHeartbeat(attempt0.getAppAttemptId(), 1, ContainerState.COMPLETE);
am0.waitForState(RMAppAttemptState.FINISHED);
rm1.waitForState(app0.getApplicationId(), RMAppState.FINISHED);
// spot check that app is not saved anymore
Assert.assertEquals(0, rmAppState.size());
finishApplicationMaster(app0, rm1, nm1, am0);
// create app that gets launched and does allocate before RM restart
RMApp app1 = rm1.submitApp(200);
@ -209,7 +211,6 @@ public class TestRMRestart {
.getApplicationId(), appUnmanaged.getApplicationSubmissionContext()
.getApplicationId());
// PHASE 2: create new RM and start from old state
// create new RM to represent restart and recover state
@ -223,10 +224,16 @@ public class TestRMRestart {
nm2.setResourceTrackerService(rm2.getResourceTrackerService());
// verify load of old state
// only 2 apps are loaded since unmanaged app is not loaded back since it
// cannot be restarted by the RM this will change with work preserving RM
// restart in which AMs/NMs are not rebooted
Assert.assertEquals(2, rm2.getRMContext().getRMApps().size());
// 4 apps are loaded.
// FINISHED app and attempt is also loaded back.
// Unmanaged app state is still loaded back but it cannot be restarted by
// the RM. this will change with work preserving RM restart in which AMs/NMs
// are not rebooted.
Assert.assertEquals(4, rm2.getRMContext().getRMApps().size());
// check that earlier finished app and attempt is also loaded back and move
// to finished state.
rm2.waitForState(app0.getApplicationId(), RMAppState.FINISHED);
rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.FINISHED);
// verify correct number of attempts and other data
RMApp loadedApp1 = rm2.getRMContext().getRMApps().get(app1.getApplicationId());
@ -331,29 +338,343 @@ public class TestRMRestart {
new ArrayList<ContainerId>()).getAllocatedContainers());
Thread.sleep(500);
}
// finish the AM's
am1.unregisterAppAttempt();
rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.FINISHING);
am1Node.nodeHeartbeat(attempt1.getAppAttemptId(), 1, ContainerState.COMPLETE);
am1.waitForState(RMAppAttemptState.FINISHED);
am2.unregisterAppAttempt();
rm2.waitForState(loadedApp2.getApplicationId(), RMAppState.FINISHING);
am2Node.nodeHeartbeat(attempt2.getAppAttemptId(), 1, ContainerState.COMPLETE);
am2.waitForState(RMAppAttemptState.FINISHED);
// finish the AMs
finishApplicationMaster(loadedApp1, rm2, am1Node, am1);
finishApplicationMaster(loadedApp2, rm2, am2Node, am2);
// stop RM's
rm2.stop();
rm1.stop();
// completed apps should be removed
Assert.assertEquals(0, rmAppState.size());
// completed apps are not removed immediately after app finish
// And finished app is also loaded back.
Assert.assertEquals(4, rmAppState.size());
}
@Test
public void testRMRestartAppRunningAMFailed() throws Exception {
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
MemoryRMStateStore memStore = new MemoryRMStateStore();
memStore.init(conf);
RMState rmState = memStore.getState();
Map<ApplicationId, ApplicationState> rmAppState =
rmState.getApplicationState();
// start RM
MockRM rm1 = new MockRM(conf, memStore);
rm1.start();
MockNM nm1 =
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
nm1.registerNode();
// create app and launch the AM
RMApp app0 = rm1.submitApp(200);
MockAM am0 = launchAM(app0, rm1, nm1);
// fail the AM by sending CONTAINER_FINISHED event without registering.
nm1.nodeHeartbeat(am0.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am0.waitForState(RMAppAttemptState.FAILED);
ApplicationState appState = rmAppState.get(app0.getApplicationId());
// assert the AM failed state is saved.
Assert.assertEquals(RMAppAttemptState.FAILED,
appState.getAttempt(am0.getApplicationAttemptId()).getState());
// assert app state has not been saved.
Assert.assertNull(rmAppState.get(app0.getApplicationId()).getState());
// new AM started but not registered, app still stays at ACCECPTED state.
rm1.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED);
// start new RM
MockRM rm2 = new MockRM(conf, memStore);
rm2.start();
// assert the previous AM state is loaded back on RM recovery.
RMApp recoveredApp =
rm2.getRMContext().getRMApps().get(app0.getApplicationId());
Assert.assertEquals(RMAppAttemptState.FAILED, recoveredApp
.getAppAttempts().get(am0.getApplicationAttemptId()).getAppAttemptState());
}
@Test
public void testRMRestartFailedApp() throws Exception {
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
MemoryRMStateStore memStore = new MemoryRMStateStore();
memStore.init(conf);
RMState rmState = memStore.getState();
Map<ApplicationId, ApplicationState> rmAppState =
rmState.getApplicationState();
// start RM
MockRM rm1 = new MockRM(conf, memStore);
rm1.start();
MockNM nm1 =
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
nm1.registerNode();
// create app and launch the AM
RMApp app0 = rm1.submitApp(200);
MockAM am0 = launchAM(app0, rm1, nm1);
// fail the AM by sending CONTAINER_FINISHED event without registering.
nm1.nodeHeartbeat(am0.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am0.waitForState(RMAppAttemptState.FAILED);
rm1.waitForState(app0.getApplicationId(), RMAppState.FAILED);
// assert the app/attempt failed state is saved.
ApplicationState appState = rmAppState.get(app0.getApplicationId());
Assert.assertEquals(RMAppState.FAILED, appState.getState());
Assert.assertEquals(RMAppAttemptState.FAILED,
appState.getAttempt(am0.getApplicationAttemptId()).getState());
// start new RM
MockRM rm2 = new MockRM(conf, memStore);
rm2.start();
RMApp loadedApp0 = rm2.getRMContext().getRMApps().get(app0.getApplicationId());
rm2.waitForState(app0.getApplicationId(), RMAppState.FAILED);
rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.FAILED);
// no new attempt is created.
Assert.assertEquals(1, loadedApp0.getAppAttempts().size());
verifyAppReportAfterRMRestart(app0, rm2);
Assert.assertTrue(app0.getDiagnostics().toString()
.contains("Failing the application."));
// failed diagnostics from attempt is lost because the diagnostics from
// attempt is not yet available by the time app is saving the app state.
}
@Test
public void testRMRestartKilledApp() throws Exception{
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
MemoryRMStateStore memStore = new MemoryRMStateStore();
memStore.init(conf);
RMState rmState = memStore.getState();
Map<ApplicationId, ApplicationState> rmAppState =
rmState.getApplicationState();
// start RM
MockRM rm1 = new MockRM(conf, memStore);
rm1.start();
MockNM nm1 =
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
nm1.registerNode();
// create app and launch the AM
RMApp app0 = rm1.submitApp(200);
MockAM am0 = launchAM(app0, rm1, nm1);
// kill the app.
rm1.killApp(app0.getApplicationId());
rm1.waitForState(app0.getApplicationId(), RMAppState.KILLED);
rm1.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.KILLED);
// killed state is saved.
ApplicationState appState = rmAppState.get(app0.getApplicationId());
Assert.assertEquals(RMAppState.KILLED, appState.getState());
Assert.assertEquals(RMAppAttemptState.KILLED,
appState.getAttempt(am0.getApplicationAttemptId()).getState());
// restart rm
MockRM rm2 = new MockRM(conf, memStore);
rm2.start();
RMApp loadedApp0 = rm2.getRMContext().getRMApps().get(app0.getApplicationId());
rm2.waitForState(app0.getApplicationId(), RMAppState.KILLED);
rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.KILLED);
// no new attempt is created.
Assert.assertEquals(1, loadedApp0.getAppAttempts().size());
ApplicationReport appReport = verifyAppReportAfterRMRestart(app0, rm2);
Assert.assertEquals(app0.getDiagnostics().toString(),
appReport.getDiagnostics());
}
@Test
public void testRMRestartSucceededApp() throws Exception {
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
MemoryRMStateStore memStore = new MemoryRMStateStore();
memStore.init(conf);
RMState rmState = memStore.getState();
Map<ApplicationId, ApplicationState> rmAppState =
rmState.getApplicationState();
// start RM
MockRM rm1 = new MockRM(conf, memStore);
rm1.start();
MockNM nm1 =
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
nm1.registerNode();
// create an app and finish the app.
RMApp app0 = rm1.submitApp(200);
MockAM am0 = launchAM(app0, rm1, nm1);
// unregister am
FinishApplicationMasterRequest req =
FinishApplicationMasterRequest.newInstance(
FinalApplicationStatus.SUCCEEDED, "diagnostics", "trackingUrl");
finishApplicationMaster(app0, rm1, nm1, am0, req);
// check the state store about the unregistered info.
ApplicationState appState = rmAppState.get(app0.getApplicationId());
ApplicationAttemptState attemptState0 =
appState.getAttempt(am0.getApplicationAttemptId());
Assert.assertEquals("diagnostics", attemptState0.getDiagnostics());
Assert.assertEquals(FinalApplicationStatus.SUCCEEDED,
attemptState0.getFinalApplicationStatus());
Assert.assertEquals("trackingUrl", attemptState0.getFinalTrackingUrl());
Assert.assertEquals(app0.getFinishTime(), appState.getFinishTime());
// restart rm
MockRM rm2 = new MockRM(conf, memStore);
rm2.start();
// verify application report returns the same app info as the app info
// before RM restarts.
ApplicationReport appReport = verifyAppReportAfterRMRestart(app0, rm2);
Assert.assertEquals(FinalApplicationStatus.SUCCEEDED,
appReport.getFinalApplicationStatus());
Assert.assertEquals("trackingUrl", appReport.getOriginalTrackingUrl());
}
@Test
public void testRMRestartGetApplicationList() throws Exception {
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
MemoryRMStateStore memStore = new MemoryRMStateStore();
memStore.init(conf);
// start RM
MockRM rm1 = new MockRM(conf, memStore);
rm1.start();
MockNM nm1 =
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
nm1.registerNode();
// a succeeded app.
RMApp app0 = rm1.submitApp(200, "name", "user", null,
false, "default", 1, null, "myType");
MockAM am0 = launchAM(app0, rm1, nm1);
finishApplicationMaster(app0, rm1, nm1, am0);
// a failed app.
RMApp app1 = rm1.submitApp(200, "name", "user", null,
false, "default", 1, null, "myType");
MockAM am1 = launchAM(app1, rm1, nm1);
// fail the AM by sending CONTAINER_FINISHED event without registering.
nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am1.waitForState(RMAppAttemptState.FAILED);
rm1.waitForState(app1.getApplicationId(), RMAppState.FAILED);
// a killed app.
RMApp app2 = rm1.submitApp(200, "name", "user", null,
false, "default", 1, null, "myType");
MockAM am2 = launchAM(app2, rm1, nm1);
rm1.killApp(app2.getApplicationId());
rm1.waitForState(app2.getApplicationId(), RMAppState.KILLED);
rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.KILLED);
// restart rm
MockRM rm2 = new MockRM(conf, memStore);
rm2.start();
GetApplicationsRequest request1 =
GetApplicationsRequest.newInstance(EnumSet.of(
YarnApplicationState.FINISHED, YarnApplicationState.KILLED,
YarnApplicationState.FAILED));
GetApplicationsResponse response1 =
rm2.getClientRMService().getApplications(request1);
List<ApplicationReport> appList1 = response1.getApplicationList();
// assert all applications exist according to application state after RM
// restarts.
boolean forApp0 = false, forApp1 = false, forApp2 = false;
for (ApplicationReport report : appList1) {
if (report.getApplicationId().equals(app0.getApplicationId())) {
Assert.assertEquals(YarnApplicationState.FINISHED,
report.getYarnApplicationState());
forApp0 = true;
}
if (report.getApplicationId().equals(app1.getApplicationId())) {
Assert.assertEquals(YarnApplicationState.FAILED,
report.getYarnApplicationState());
forApp1 = true;
}
if (report.getApplicationId().equals(app2.getApplicationId())) {
Assert.assertEquals(YarnApplicationState.KILLED,
report.getYarnApplicationState());
forApp2 = true;
}
}
Assert.assertTrue(forApp0 && forApp1 && forApp2);
// assert all applications exist according to application type after RM
// restarts.
Set<String> appTypes = new HashSet<String>();
appTypes.add("myType");
GetApplicationsRequest request2 =
GetApplicationsRequest.newInstance(appTypes);
GetApplicationsResponse response2 =
rm2.getClientRMService().getApplications(request2);
List<ApplicationReport> appList2 = response2.getApplicationList();
Assert.assertTrue(3 == appList2.size());
}
private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
throws Exception {
RMAppAttempt attempt = app.getCurrentAppAttempt();
nm.nodeHeartbeat(true);
MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
am.registerAppAttempt();
rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
return am;
}
private ApplicationReport verifyAppReportAfterRMRestart(RMApp app, MockRM rm)
throws Exception {
GetApplicationReportRequest reportRequest =
GetApplicationReportRequest.newInstance(app.getApplicationId());
GetApplicationReportResponse response =
rm.getClientRMService().getApplicationReport(reportRequest);
ApplicationReport report = response.getApplicationReport();
Assert.assertEquals(app.getStartTime(), report.getStartTime());
Assert.assertEquals(app.getFinishTime(), report.getFinishTime());
Assert.assertEquals(app.createApplicationState(),
report.getYarnApplicationState());
Assert.assertTrue(1 == report.getProgress());
return response.getApplicationReport();
}
private void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm,
MockAM am) throws Exception {
final FinishApplicationMasterRequest req =
FinishApplicationMasterRequest.newInstance(
FinalApplicationStatus.SUCCEEDED, "", "");
finishApplicationMaster(rmApp, rm, nm, am, req);
}
private void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm,
MockAM am, FinishApplicationMasterRequest req) throws Exception {
RMState rmState =
((MemoryRMStateStore) rm.getRMContext().getStateStore()).getState();
Map<ApplicationId, ApplicationState> rmAppState =
rmState.getApplicationState();
am.unregisterAppAttempt(req);
am.waitForState(RMAppAttemptState.FINISHING);
nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am.waitForState(RMAppAttemptState.FINISHED);
rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
// check that app/attempt is saved with the final state
ApplicationState appState = rmAppState.get(rmApp.getApplicationId());
Assert
.assertEquals(RMAppState.FINISHED, appState.getState());
Assert.assertEquals(RMAppAttemptState.FINISHED,
appState.getAttempt(am.getApplicationAttemptId()).getState());
}
@Test
public void testRMRestartOnMaxAppAttempts() throws Exception {
Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
@ -407,16 +728,17 @@ public class TestRMRestart {
rm2.getRMContext().getRMApps().get(app2.getApplicationId())
.getMaxAppAttempts());
// verify that app2 exists app1 is removed
Assert.assertEquals(1, rm2.getRMContext().getRMApps().size());
Assert.assertNotNull(rm2.getRMContext().getRMApps()
.get(app2.getApplicationId()));
Assert.assertNull(rm2.getRMContext().getRMApps()
.get(app1.getApplicationId()));
// app1 and app2 are loaded back, but app1 failed because it's
// hitting max-retry.
Assert.assertEquals(2, rm2.getRMContext().getRMApps().size());
rm2.waitForState(app1.getApplicationId(), RMAppState.FAILED);
rm2.waitForState(app2.getApplicationId(), RMAppState.ACCEPTED);
// verify that app2 is stored, app1 is removed
Assert.assertNotNull(rmAppState.get(app2.getApplicationId()));
Assert.assertNull(rmAppState.get(app1.getApplicationId()));
// app1 failed state is saved in state store. app2 final saved state is not
// determined yet.
Assert.assertEquals(RMAppState.FAILED,
rmAppState.get(app1.getApplicationId()).getState());
Assert.assertNull(rmAppState.get(app2.getApplicationId()).getState());
// stop the RM
rm1.stop();

View File

@ -26,10 +26,8 @@ import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import javax.crypto.SecretKey;
@ -39,13 +37,7 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.ha.ClientBaseWithFixes;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.delegation.DelegationKey;
@ -54,6 +46,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@ -66,22 +59,20 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.Appli
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMDTSecretManagerState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStoredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.zookeeper.ZooKeeper;
import org.junit.Test;
public class RMStateStoreTestBase extends ClientBaseWithFixes{
public static final Log LOG = LogFactory.getLog(RMStateStoreTestBase.class);
static class TestDispatcher implements
Dispatcher, EventHandler<RMAppAttemptStoredEvent> {
Dispatcher, EventHandler<RMAppAttemptNewSavedEvent> {
ApplicationAttemptId attemptId;
Exception storedException;
@ -95,7 +86,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
}
@Override
public void handle(RMAppAttemptStoredEvent event) {
public void handle(RMAppAttemptNewSavedEvent event) {
assertEquals(attemptId, event.getApplicationAttemptId());
assertEquals(storedException, event.getStoredException());
notified = true;
@ -134,18 +125,19 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
dispatcher.notified = false;
}
void storeApp(
RMStateStore store, ApplicationId appId, long time) throws Exception {
void storeApp(RMStateStore store, ApplicationId appId, long submitTime,
long startTime) throws Exception {
ApplicationSubmissionContext context =
new ApplicationSubmissionContextPBImpl();
context.setApplicationId(appId);
RMApp mockApp = mock(RMApp.class);
when(mockApp.getApplicationId()).thenReturn(appId);
when(mockApp.getSubmitTime()).thenReturn(time);
when(mockApp.getSubmitTime()).thenReturn(submitTime);
when(mockApp.getStartTime()).thenReturn(startTime);
when(mockApp.getApplicationSubmissionContext()).thenReturn(context);
when(mockApp.getUser()).thenReturn("test");
store.storeApplication(mockApp);
store.storeNewApplication(mockApp);
}
ContainerId storeAttempt(RMStateStore store, ApplicationAttemptId attemptId,
@ -163,7 +155,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
.thenReturn(clientTokenMasterKey);
dispatcher.attemptId = attemptId;
dispatcher.storedException = null;
store.storeApplicationAttempt(mockAttempt);
store.storeNewApplicationAttempt(mockAttempt);
waitNotify(dispatcher);
return container.getId();
}
@ -171,6 +163,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
void testRMAppStateStore(RMStateStoreHelper stateStoreHelper)
throws Exception {
long submitTime = System.currentTimeMillis();
long startTime = System.currentTimeMillis() + 1234;
Configuration conf = new YarnConfiguration();
RMStateStore store = stateStoreHelper.getRMStateStore();
TestDispatcher dispatcher = new TestDispatcher();
@ -184,7 +177,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
ApplicationAttemptId attemptId1 = ConverterUtils
.toApplicationAttemptId("appattempt_1352994193343_0001_000001");
ApplicationId appId1 = attemptId1.getApplicationId();
storeApp(store, appId1, submitTime);
storeApp(store, appId1, submitTime, startTime);
// create application token and client token key for attempt1
Token<AMRMTokenIdentifier> appAttemptToken1 =
@ -217,7 +210,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
ApplicationAttemptId attemptIdRemoved = ConverterUtils
.toApplicationAttemptId("appattempt_1352994193343_0002_000001");
ApplicationId appIdRemoved = attemptIdRemoved.getApplicationId();
storeApp(store, appIdRemoved, submitTime);
storeApp(store, appIdRemoved, submitTime, startTime);
storeAttempt(store, attemptIdRemoved,
"container_1352994193343_0002_01_000001", null, null, dispatcher);
@ -241,6 +234,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
// load state
store = stateStoreHelper.getRMStateStore();
store.setRMDispatcher(dispatcher);
RMState state = store.loadState();
Map<ApplicationId, ApplicationState> rmAppState =
state.getApplicationState();
@ -250,6 +244,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
assertNotNull(appState);
// app is loaded correctly
assertEquals(submitTime, appState.getSubmitTime());
assertEquals(startTime, appState.getStartTime());
// submission context is loaded correctly
assertEquals(appId1,
appState.getApplicationSubmissionContext().getApplicationId());
@ -283,6 +278,59 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
attemptState.getAppAttemptCredentials()
.getSecretKey(RMStateStore.AM_CLIENT_TOKEN_MASTER_KEY_NAME));
//******* update application/attempt state *******//
ApplicationState appState2 =
new ApplicationState(appState.submitTime, appState.startTime,
appState.context, appState.user, RMAppState.FINISHED,
"appDiagnostics", 1234);
appState2.attempts.putAll(appState.attempts);
store.updateApplicationState(appState2);
ApplicationAttemptState oldAttemptState = attemptState;
ApplicationAttemptState newAttemptState =
new ApplicationAttemptState(oldAttemptState.getAttemptId(),
oldAttemptState.getMasterContainer(),
oldAttemptState.getAppAttemptCredentials(),
oldAttemptState.getStartTime(), RMAppAttemptState.FINISHED,
"myTrackingUrl", "attemptDiagnostics",
FinalApplicationStatus.SUCCEEDED);
store.updateApplicationAttemptState(newAttemptState);
// let things settle down
Thread.sleep(1000);
store.close();
// check updated application state.
store = stateStoreHelper.getRMStateStore();
store.setRMDispatcher(dispatcher);
RMState newRMState = store.loadState();
Map<ApplicationId, ApplicationState> newRMAppState =
newRMState.getApplicationState();
ApplicationState updatedAppState = newRMAppState.get(appId1);
assertEquals(appState.getAppId(),updatedAppState.getAppId());
assertEquals(appState.getSubmitTime(), updatedAppState.getSubmitTime());
assertEquals(appState.getStartTime(), updatedAppState.getStartTime());
assertEquals(appState.getUser(), updatedAppState.getUser());
// new app state fields
assertEquals( RMAppState.FINISHED, updatedAppState.getState());
assertEquals("appDiagnostics", updatedAppState.getDiagnostics());
assertEquals(1234, updatedAppState.getFinishTime());
// check updated attempt state
ApplicationAttemptState updatedAttemptState =
updatedAppState.getAttempt(newAttemptState.getAttemptId());
assertEquals(oldAttemptState.getAttemptId(),
updatedAttemptState.getAttemptId());
assertEquals(containerId2, updatedAttemptState.getMasterContainer().getId());
assertArrayEquals(clientTokenKey2.getEncoded(),
updatedAttemptState.getAppAttemptCredentials().getSecretKey(
RMStateStore.AM_CLIENT_TOKEN_MASTER_KEY_NAME));
// new attempt state fields
assertEquals(RMAppAttemptState.FINISHED, updatedAttemptState.getState());
assertEquals("myTrackingUrl", updatedAttemptState.getFinalTrackingUrl());
assertEquals("attemptDiagnostics", updatedAttemptState.getDiagnostics());
assertEquals(FinalApplicationStatus.SUCCEEDED,
updatedAttemptState.getFinalApplicationStatus());
// assert store is in expected state after everything is cleaned
assertTrue(stateStoreHelper.isFinalStateValid());

View File

@ -19,19 +19,18 @@
package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
import java.util.Collection;
import java.util.LinkedHashMap;
import java.util.Map;
import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@ -226,5 +225,5 @@ public class MockRMApp implements RMApp {
@Override
public YarnApplicationState createApplicationState() {
return null;
};
}
}

View File

@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assume.assumeTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@ -57,13 +58,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStoredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@ -286,7 +289,8 @@ public class TestRMAppTransitions {
}
// test to make sure times are set when app finishes
private static void assertTimesAtFinish(RMApp application) {
private void assertTimesAtFinish(RMApp application) {
sendAppUpdateSavedEvent(application);
assertStartTimeSet(application);
Assert.assertTrue("application finish time is not greater then 0",
(application.getFinishTime() > 0));
@ -294,11 +298,12 @@ public class TestRMAppTransitions {
(application.getFinishTime() >= application.getStartTime()));
}
private void assertAppRemoved(RMApp application){
verify(store).removeApplication(application);
private void assertAppFinalStateSaved(RMApp application){
verify(store, times(1)).updateApplicationState(any(ApplicationState.class));
}
private static void assertKilled(RMApp application) {
private void assertKilled(RMApp application) {
sendAppUpdateSavedEvent(application);
assertTimesAtFinish(application);
assertAppState(RMAppState.KILLED, application);
assertFinalAppStatus(FinalApplicationStatus.KILLED, application);
@ -307,20 +312,34 @@ public class TestRMAppTransitions {
"Application killed by user.", diag.toString());
}
private static void assertAppAndAttemptKilled(RMApp application) throws InterruptedException {
private void assertAppAndAttemptKilled(RMApp application)
throws InterruptedException {
assertKilled(application);
Assert.assertEquals( RMAppAttemptState.KILLED,
application.getCurrentAppAttempt().getAppAttemptState()
);
// send attempt final state saved event.
application.getCurrentAppAttempt().handle(
new RMAppAttemptUpdateSavedEvent(application.getCurrentAppAttempt()
.getAppAttemptId(), null));
Assert.assertEquals(RMAppAttemptState.KILLED, application
.getCurrentAppAttempt().getAppAttemptState());
assertAppFinalStateSaved(application);
}
private static void assertFailed(RMApp application, String regex) {
private void assertFailed(RMApp application, String regex) {
sendAppUpdateSavedEvent(application);
assertTimesAtFinish(application);
assertAppState(RMAppState.FAILED, application);
assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
StringBuilder diag = application.getDiagnostics();
Assert.assertTrue("application diagnostics is not correct",
diag.toString().matches(regex));
assertAppFinalStateSaved(application);
}
private void sendAppUpdateSavedEvent(RMApp application) {
RMAppEvent event =
new RMAppUpdateSavedEvent(application.getApplicationId(), null);
application.handle(event);
rmDispatcher.await();
}
protected RMApp testCreateAppNewSaving(
@ -340,7 +359,7 @@ public class TestRMAppTransitions {
RMApp application = testCreateAppNewSaving(submissionContext);
// NEW_SAVING => SUBMITTED event RMAppEventType.APP_SAVED
RMAppEvent event =
new RMAppStoredEvent(application.getApplicationId(), null);
new RMAppNewSavedEvent(application.getApplicationId(), null);
application.handle(event);
assertStartTimeSet(application);
assertAppState(RMAppState.SUBMITTED, application);
@ -386,15 +405,15 @@ public class TestRMAppTransitions {
return application;
}
protected RMApp testCreateAppRemoving(
protected RMApp testCreateAppFinalSaving(
ApplicationSubmissionContext submissionContext) throws IOException {
RMApp application = testCreateAppRunning(submissionContext);
RMAppEvent finishingEvent =
new RMAppEvent(application.getApplicationId(),
RMAppEventType.ATTEMPT_UNREGISTERED);
application.handle(finishingEvent);
assertAppState(RMAppState.REMOVING, application);
assertAppRemoved(application);
assertAppState(RMAppState.FINAL_SAVING, application);
assertAppFinalStateSaved(application);
return application;
}
@ -402,11 +421,11 @@ public class TestRMAppTransitions {
ApplicationSubmissionContext submissionContext) throws IOException {
// unmanaged AMs don't use the FINISHING state
assert submissionContext == null || !submissionContext.getUnmanagedAM();
RMApp application = testCreateAppRemoving(submissionContext);
// REMOVING => FINISHING event RMAppEventType.APP_REMOVED
RMAppEvent finishingEvent =
new RMAppRemovedEvent(application.getApplicationId(), null);
application.handle(finishingEvent);
RMApp application = testCreateAppFinalSaving(submissionContext);
// FINAL_SAVING => FINISHING event RMAppEventType.APP_UPDATED
RMAppEvent appUpdated =
new RMAppUpdateSavedEvent(application.getApplicationId(), null);
application.handle(appUpdated);
assertAppState(RMAppState.FINISHING, application);
assertTimesAtFinish(application);
return application;
@ -552,7 +571,6 @@ public class TestRMAppTransitions {
RMAppEventType.KILL);
application.handle(event);
rmDispatcher.await();
assertKilled(application);
assertAppAndAttemptKilled(application);
}
@ -597,7 +615,6 @@ public class TestRMAppTransitions {
RMAppEventType.KILL);
application.handle(event);
rmDispatcher.await();
assertKilled(application);
assertAppAndAttemptKilled(application);
}
@ -611,6 +628,14 @@ public class TestRMAppTransitions {
new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL);
application.handle(event);
rmDispatcher.await();
// Ignore Attempt_Finished if we were supposed to go to Finished.
assertAppState(RMAppState.FINAL_SAVING, application);
RMAppEvent finishEvent =
new RMAppFinishedAttemptEvent(application.getApplicationId(), null);
application.handle(finishEvent);
assertAppState(RMAppState.FINAL_SAVING, application);
assertKilled(application);
}
@ -665,30 +690,6 @@ public class TestRMAppTransitions {
assertFailed(application, ".*Failing the application.*");
}
@Test
public void testAppRemovingFinished() throws IOException {
LOG.info("--- START: testAppRemovingFINISHED ---");
RMApp application = testCreateAppRemoving(null);
// APP_REMOVING => FINISHED event RMAppEventType.ATTEMPT_FINISHED
RMAppEvent finishedEvent = new RMAppFinishedAttemptEvent(
application.getApplicationId(), null);
application.handle(finishedEvent);
rmDispatcher.await();
assertAppState(RMAppState.FINISHED, application);
}
@Test
public void testAppRemovingKilled() throws IOException {
LOG.info("--- START: testAppRemovingKilledD ---");
RMApp application = testCreateAppRemoving(null);
// APP_REMOVING => KILLED event RMAppEventType.KILL
RMAppEvent event =
new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL);
application.handle(event);
rmDispatcher.await();
assertAppState(RMAppState.KILLED, application);
}
@Test
public void testAppFinishingKill() throws IOException {
LOG.info("--- START: testAppFinishedFinished ---");
@ -702,6 +703,33 @@ public class TestRMAppTransitions {
assertAppState(RMAppState.FINISHED, application);
}
// While App is at FINAL_SAVING, Attempt_Finished event may come before
// App_Saved event, we stay on FINAL_SAVING on Attempt_Finished event
// and then directly jump from FINAL_SAVING to FINISHED state on App_Saved
// event
@Test
public void testAppFinalSavingToFinished() throws IOException {
LOG.info("--- START: testAppFinalSavingToFinished ---");
RMApp application = testCreateAppFinalSaving(null);
final String diagMsg = "some diagnostics";
// attempt_finished event comes before attempt_saved event
RMAppEvent event =
new RMAppFinishedAttemptEvent(application.getApplicationId(), diagMsg);
application.handle(event);
assertAppState(RMAppState.FINAL_SAVING, application);
RMAppEvent appUpdated =
new RMAppUpdateSavedEvent(application.getApplicationId(), null);
application.handle(appUpdated);
assertAppState(RMAppState.FINISHED, application);
assertTimesAtFinish(application);
// finished without a proper unregister implies failed
assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
Assert.assertTrue("Finished app missing diagnostics", application
.getDiagnostics().indexOf(diagMsg) != -1);
}
@Test
public void testAppFinishedFinished() throws IOException {
LOG.info("--- START: testAppFinishedFinished ---");
@ -742,7 +770,7 @@ public class TestRMAppTransitions {
assertAppState(RMAppState.FAILED, application);
// FAILED => FAILED event RMAppEventType.APP_SAVED
event = new RMAppStoredEvent(application.getApplicationId(), null);
event = new RMAppNewSavedEvent(application.getApplicationId(), null);
application.handle(event);
rmDispatcher.await();
assertTimesAtFinish(application);
@ -797,7 +825,7 @@ public class TestRMAppTransitions {
assertAppState(RMAppState.KILLED, application);
// KILLED => KILLED event RMAppEventType.APP_SAVED
event = new RMAppStoredEvent(application.getApplicationId(), null);
event = new RMAppNewSavedEvent(application.getApplicationId(), null);
application.handle(event);
rmDispatcher.await();
assertTimesAtFinish(application);
@ -873,7 +901,7 @@ public class TestRMAppTransitions {
attempt.handle(new RMAppAttemptContainerAllocatedEvent(attempt
.getAppAttemptId(), container));
attempt
.handle(new RMAppAttemptStoredEvent(attempt.getAppAttemptId(), null));
.handle(new RMAppAttemptNewSavedEvent(attempt.getAppAttemptId(), null));
attempt.handle(new RMAppAttemptEvent(attempt.getAppAttemptId(),
RMAppAttemptEventType.LAUNCHED));

View File

@ -64,6 +64,7 @@ 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.ApplicationMasterLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@ -75,8 +76,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStoredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@ -114,6 +116,7 @@ public class TestRMAppAttemptTransitions {
private ApplicationMasterLauncher applicationMasterLauncher;
private AMLivelinessMonitor amLivelinessMonitor;
private AMLivelinessMonitor amFinishingMonitor;
private RMStateStore store;
private RMApp application;
private RMAppAttempt applicationAttempt;
@ -209,7 +212,7 @@ public class TestRMAppAttemptTransitions {
new NMTokenSecretManagerInRM(conf),
clientToAMTokenManager);
RMStateStore store = mock(RMStateStore.class);
store = mock(RMStateStore.class);
((RMContextImpl) rmContext).setStateStore(store);
scheduler = mock(YarnScheduler.class);
@ -330,6 +333,7 @@ public class TestRMAppAttemptTransitions {
* {@link RMAppAttemptState#SUBMITTED} -> {@link RMAppAttemptState#FAILED}
*/
private void testAppAttemptSubmittedToFailedState(String diagnostics) {
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState());
assertEquals(diagnostics, applicationAttempt.getDiagnostics());
@ -354,6 +358,7 @@ public class TestRMAppAttemptTransitions {
*/
private void testAppAttemptKilledState(Container amContainer,
String diagnostics) {
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.KILLED,
applicationAttempt.getAppAttemptState());
assertEquals(diagnostics, applicationAttempt.getDiagnostics());
@ -363,6 +368,7 @@ public class TestRMAppAttemptTransitions {
assertEquals(0, applicationAttempt.getRanNodes().size());
assertNull(applicationAttempt.getFinalApplicationStatus());
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
verifyAttemptFinalStateSaved();
}
/**
@ -427,6 +433,7 @@ public class TestRMAppAttemptTransitions {
*/
private void testAppAttemptFailedState(Container container,
String diagnostics) {
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState());
assertEquals(diagnostics, applicationAttempt.getDiagnostics());
@ -437,8 +444,8 @@ public class TestRMAppAttemptTransitions {
// Check events
verify(application, times(2)).handle(any(RMAppFailedAttemptEvent.class));
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
verifyAttemptFinalStateSaved();
}
/**
@ -492,6 +499,7 @@ public class TestRMAppAttemptTransitions {
assertEquals(container, applicationAttempt.getMasterContainer());
assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
verifyTokenCount(applicationAttempt.getAppAttemptId(), 0);
verifyAttemptFinalStateSaved();
}
/**
@ -508,10 +516,10 @@ public class TestRMAppAttemptTransitions {
verifyUrl(trackingUrl, applicationAttempt.getOriginalTrackingUrl());
if (unmanagedAM) {
verifyUrl(trackingUrl, applicationAttempt.getTrackingUrl());
} else {
assertEquals(getProxyUrl(applicationAttempt),
applicationAttempt.getTrackingUrl());
verifyAttemptFinalStateSaved();
}
assertEquals(finishedContainerCount, applicationAttempt
.getJustFinishedContainers().size());
@ -539,7 +547,7 @@ public class TestRMAppAttemptTransitions {
assertEquals(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING,
applicationAttempt.getAppAttemptState());
applicationAttempt.handle(
new RMAppAttemptStoredEvent(
new RMAppAttemptNewSavedEvent(
applicationAttempt.getAppAttemptId(), null));
}
@ -576,7 +584,7 @@ public class TestRMAppAttemptTransitions {
assertEquals(RMAppAttemptState.ALLOCATED_SAVING,
applicationAttempt.getAppAttemptState());
applicationAttempt.handle(
new RMAppAttemptStoredEvent(
new RMAppAttemptNewSavedEvent(
applicationAttempt.getAppAttemptId(), null));
testAppAttemptAllocatedState(container);
@ -617,6 +625,7 @@ public class TestRMAppAttemptTransitions {
new RMAppAttemptUnregistrationEvent(
applicationAttempt.getAppAttemptId(),
trackingUrl, finalStatus, diagnostics));
sendAttemptUpdateSavedEvent(applicationAttempt);
testAppAttemptFinishingState(container, finalStatus,
trackingUrl, diagnostics);
}
@ -648,6 +657,14 @@ public class TestRMAppAttemptTransitions {
true);
}
private void sendAttemptUpdateSavedEvent(RMAppAttempt applicationAttempt) {
assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState());
applicationAttempt.handle(
new RMAppAttemptUpdateSavedEvent(
applicationAttempt.getAppAttemptId(), null));
}
@Test
public void testUnmanagedAMUnexpectedRegistration() {
unmanagedAM = true;
@ -745,6 +762,7 @@ public class TestRMAppAttemptTransitions {
ContainerState.COMPLETE, containerDiagMsg, exitCode);
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
applicationAttempt.getAppAttemptId(), cs));
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState());
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
@ -762,6 +780,20 @@ public class TestRMAppAttemptTransitions {
ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
appAttemptId, cs));
// ignored ContainerFinished and Expire at FinalSaving if we were supposed
// to Failed state.
assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState());
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
amContainer.getId(), ContainerState.COMPLETE, "", 0)));
applicationAttempt.handle(new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState());
assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
@ -782,6 +814,20 @@ public class TestRMAppAttemptTransitions {
new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(),
RMAppAttemptEventType.KILL));
// ignored ContainerFinished and Expire at FinalSaving if we were supposed
// to Killed state.
assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState());
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
amContainer.getId(), ContainerState.COMPLETE, "", 0)));
applicationAttempt.handle(new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.KILLED,
applicationAttempt.getAppAttemptState());
assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
@ -800,6 +846,7 @@ public class TestRMAppAttemptTransitions {
launchApplicationAttempt(amContainer);
applicationAttempt.handle(new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState());
assertTrue("expire diagnostics missing",
@ -818,6 +865,7 @@ public class TestRMAppAttemptTransitions {
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
applicationAttempt.handle(new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState());
assertTrue("expire diagnostics missing",
@ -963,6 +1011,63 @@ public class TestRMAppAttemptTransitions {
diagnostics, 0, false);
}
// While attempt is at FINAL_SAVING, Contaienr_Finished event may come before
// Attempt_Saved event, we stay on FINAL_SAVING on Container_Finished event
// and then directly jump from FINAL_SAVING to FINISHED state on Attempt_Saved
// event
@Test
public void
testFinalSavingToFinishedWithContainerFinished() {
Container amContainer = allocateApplicationAttempt();
launchApplicationAttempt(amContainer);
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED;
String trackingUrl = "mytrackingurl";
String diagnostics = "Successful";
applicationAttempt.handle(new RMAppAttemptUnregistrationEvent(
applicationAttempt.getAppAttemptId(), trackingUrl, finalStatus,
diagnostics));
assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState());
// Container_finished event comes before Attempt_Saved event.
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
amContainer.getId(), ContainerState.COMPLETE, "", 0)));
assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState());
// send attempt_saved
sendAttemptUpdateSavedEvent(applicationAttempt);
testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl,
diagnostics, 0, false);
}
// While attempt is at FINAL_SAVING, Expire event may come before
// Attempt_Saved event, we stay on FINAL_SAVING on Expire event and then
// directly jump from FINAL_SAVING to FINISHED state on Attempt_Saved event.
@Test
public void testFinalSavingToFinishedWithExpire() {
Container amContainer = allocateApplicationAttempt();
launchApplicationAttempt(amContainer);
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED;
String trackingUrl = "mytrackingurl";
String diagnostics = "Successssseeeful";
applicationAttempt.handle(new RMAppAttemptUnregistrationEvent(
applicationAttempt.getAppAttemptId(), trackingUrl, finalStatus,
diagnostics));
assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState());
// Expire event comes before Attempt_saved event.
applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState());
// send attempt_saved
sendAttemptUpdateSavedEvent(applicationAttempt);
testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl,
diagnostics, 0, false);
}
private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) {
verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId);
if (UserGroupInformation.isSecurityEnabled()) {
@ -980,4 +1085,9 @@ public class TestRMAppAttemptTransitions {
assertEquals(url1, url2);
}
}
private void verifyAttemptFinalStateSaved() {
verify(store, times(1)).updateApplicationAttemptState(
any(ApplicationAttemptState.class));
}
}