hadoop-yarn-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From a..@apache.org
Subject svn commit: r1537584 [3/4] - in /hadoop/common/branches/HDFS-2832/hadoop-yarn-project: ./ hadoop-yarn/hadoop-yarn-api/src/main/proto/server/ hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/ hadoop-yarn/hadoop-yarn-client/...
Date Thu, 31 Oct 2013 18:49:56 GMT
Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java?rev=1537584&r1=1537583&r2=1537584&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java Thu Oct 31 18:49:54 2013
@@ -40,7 +40,6 @@ import org.apache.commons.lang.StringUti
 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.res
 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
   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
   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
        // 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
           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
                 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
               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
               RMAppAttemptEventType.REGISTERED,
               RMAppAttemptEventType.CONTAINER_ALLOCATED,
               RMAppAttemptEventType.CONTAINER_ACQUIRED,
-              RMAppAttemptEventType.ATTEMPT_SAVED,
+              RMAppAttemptEventType.ATTEMPT_NEW_SAVED,
               RMAppAttemptEventType.CONTAINER_FINISHED,
               RMAppAttemptEventType.UNREGISTERED,
               RMAppAttemptEventType.KILL,
@@ -411,7 +459,7 @@ public class RMAppAttemptImpl implements
   public RMAppAttemptState getAppAttemptState() {
     this.readLock.lock();
     try {
-      return this.stateMachine.getCurrentState();
+        return this.stateMachine.getCurrentState();
     } finally {
       this.readLock.unlock();
     }
@@ -444,7 +492,7 @@ public class RMAppAttemptImpl implements
     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
   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
   }
 
   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
     }
   }
 
-  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,19 +711,26 @@ public class RMAppAttemptImpl implements
   }
 
   @Override
-  public void recover(RMState state) throws Exception{
-    ApplicationState appState = 
+  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");
-    handle(new RMAppAttemptEvent(getAppAttemptId(), 
-                                 RMAppAttemptEventType.RECOVER));
+    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));
   }
 
   private void recoverAppAttemptCredentials(Credentials appAttemptTokens)
@@ -763,7 +808,7 @@ public class RMAppAttemptImpl implements
       
       // 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
         }
         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
                                                                            0));
       appAttempt.getSubmissionContext().setResource(
           appAttempt.getMasterContainer().getResource());
-      RMStateStore store = appAttempt.rmContext.getStateStore();
-      appAttempt.storeAttempt(store);
+      appAttempt.storeAttempt();
     }
   }
   
@@ -851,6 +893,134 @@ public class RMAppAttemptImpl implements
       appAttempt.launchAttempt();
     }
   }
+
+  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 {
 
@@ -998,15 +1168,20 @@ public class RMAppAttemptImpl implements
           = (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
           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
       // 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
     @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
     @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
     @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
@@ -1185,29 +1401,37 @@ public class RMAppAttemptImpl implements
       // the AMContainer, AppAttempt fails
       if (appAttempt.masterContainer != null
           && appAttempt.masterContainer.getId().equals(
-              containerStatus.getContainerId())) {
-        // 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;
+            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
+      // 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(getAMContainerCrashedDiagnostics(containerFinishedEvent));
+      new FinalTransition(RMAppAttemptState.FAILED).transition(appAttempt,
+        event);
+    }
+  }
+
   private static final class AMFinishingContainerFinishedTransition
       implements
       MultipleArcTransition<RMAppAttemptImpl, RMAppAttemptEvent, RMAppAttemptState> {
@@ -1228,13 +1452,83 @@ public class RMAppAttemptImpl implements
             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
   }
   
   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
     }
   }
   
-  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
               + " AttemptId: " + 
               getAppAttemptId()
               + " MasterContainer: " + masterContainer);
-    store.storeApplicationAttempt(this);
+    rmContext.getStateStore().storeNewApplicationAttempt(this);
   }
 
   private void removeCredentials(RMAppAttemptImpl appAttempt) {

Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptState.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptState.java?rev=1537584&r1=1537583&r2=1537584&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptState.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptState.java Thu Oct 31 18:49:54 2013
@@ -20,5 +20,6 @@ package org.apache.hadoop.yarn.server.re
 
 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
 }

Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptUnregistrationEvent.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptUnregistrationEvent.java?rev=1537584&r1=1537583&r2=1537584&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptUnregistrationEvent.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptUnregistrationEvent.java Thu Oct 31 18:49:54 2013
@@ -25,20 +25,20 @@ import org.apache.hadoop.yarn.server.res
 
 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() {

Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java?rev=1537584&r1=1537583&r2=1537584&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java Thu Oct 31 18:49:54 2013
@@ -438,7 +438,10 @@ public class RMNodeImpl implements RMNod
 
       rmNode.context.getDispatcher().getEventHandler().handle(
           new NodeAddedSchedulerEvent(rmNode));
-      
+      rmNode.context.getDispatcher().getEventHandler().handle(
+          new NodesListManagerEvent(
+              NodesListManagerEventType.NODE_USABLE, rmNode));
+ 
       String host = rmNode.nodeId.getHost();
       if (rmNode.context.getInactiveRMNodes().containsKey(host)) {
         // Old node rejoining
@@ -471,7 +474,7 @@ public class RMNodeImpl implements RMNod
           // Only add new node if old state is not UNHEALTHY
           rmNode.context.getDispatcher().getEventHandler().handle(
               new NodeAddedSchedulerEvent(rmNode));
-         }
+        }
       } else {
         // Reconnected node differs, so replace old node and start new node
         switch (rmNode.getState()) {
@@ -486,6 +489,9 @@ public class RMNodeImpl implements RMNod
         rmNode.context.getDispatcher().getEventHandler().handle(
             new RMNodeEvent(newNode.getNodeID(), RMNodeEventType.STARTED));
       }
+      rmNode.context.getDispatcher().getEventHandler().handle(
+          new NodesListManagerEvent(
+              NodesListManagerEventType.NODE_USABLE, rmNode));
     }
   }
 

Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java?rev=1537584&r1=1537583&r2=1537584&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java Thu Oct 31 18:49:54 2013
@@ -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;
+  }
 }

Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java?rev=1537584&r1=1537583&r2=1537584&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java Thu Oct 31 18:49:54 2013
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.record
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.InlineDispatcher;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -44,6 +45,7 @@ import org.apache.hadoop.yarn.server.res
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeReconnectEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@@ -79,6 +81,18 @@ public class TestRMNodeTransitions {
     }
   }
 
+  private NodesListManagerEvent nodesListManagerEvent = null;
+  
+  private class TestNodeListManagerEventDispatcher implements
+      EventHandler<NodesListManagerEvent> {
+    
+    @Override
+    public void handle(NodesListManagerEvent event) {
+      nodesListManagerEvent = event;
+    }
+
+  }
+
   @Before
   public void setUp() throws Exception {
     InlineDispatcher rmDispatcher = new InlineDispatcher();
@@ -109,8 +123,12 @@ public class TestRMNodeTransitions {
     rmDispatcher.register(SchedulerEventType.class, 
         new TestSchedulerEventDispatcher());
     
+    rmDispatcher.register(NodesListManagerEventType.class,
+        new TestNodeListManagerEventDispatcher());
+
     NodeId nodeId = BuilderUtils.newNodeId("localhost", 0);
     node = new RMNodeImpl(nodeId, rmContext, null, 0, 0, null, null, null);
+    nodesListManagerEvent =  null;
 
   }
   
@@ -431,8 +449,9 @@ public class TestRMNodeTransitions {
 
   private RMNodeImpl getRunningNode() {
     NodeId nodeId = BuilderUtils.newNodeId("localhost", 0);
+    Resource capability = Resource.newInstance(4096, 4);
     RMNodeImpl node = new RMNodeImpl(nodeId, rmContext,null, 0, 0,
-        null, null, null);
+        null, capability, null);
     node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.STARTED));
     Assert.assertEquals(NodeState.RUNNING, node.getState());
     return node;
@@ -447,4 +466,60 @@ public class TestRMNodeTransitions {
     Assert.assertEquals(NodeState.UNHEALTHY, node.getState());
     return node;
   }
+
+
+  private RMNodeImpl getNewNode() {
+    NodeId nodeId = BuilderUtils.newNodeId("localhost", 0);
+    RMNodeImpl node = new RMNodeImpl(nodeId, rmContext, null, 0, 0, null, null, null);
+    return node;
+  }
+
+  @Test
+  public void testAdd() {
+    RMNodeImpl node = getNewNode();
+    ClusterMetrics cm = ClusterMetrics.getMetrics();
+    int initialActive = cm.getNumActiveNMs();
+    int initialLost = cm.getNumLostNMs();
+    int initialUnhealthy = cm.getUnhealthyNMs();
+    int initialDecommissioned = cm.getNumDecommisionedNMs();
+    int initialRebooted = cm.getNumRebootedNMs();
+    node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.STARTED));
+    Assert.assertEquals("Active Nodes", initialActive + 1, cm.getNumActiveNMs());
+    Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs());
+    Assert.assertEquals("Unhealthy Nodes",
+        initialUnhealthy, cm.getUnhealthyNMs());
+    Assert.assertEquals("Decommissioned Nodes",
+        initialDecommissioned, cm.getNumDecommisionedNMs());
+    Assert.assertEquals("Rebooted Nodes",
+        initialRebooted, cm.getNumRebootedNMs());
+    Assert.assertEquals(NodeState.RUNNING, node.getState());
+    Assert.assertNotNull(nodesListManagerEvent);
+    Assert.assertEquals(NodesListManagerEventType.NODE_USABLE, 
+        nodesListManagerEvent.getType());
+  }
+
+  @Test
+  public void testReconnect() {
+    RMNodeImpl node = getRunningNode();
+    ClusterMetrics cm = ClusterMetrics.getMetrics();
+    int initialActive = cm.getNumActiveNMs();
+    int initialLost = cm.getNumLostNMs();
+    int initialUnhealthy = cm.getUnhealthyNMs();
+    int initialDecommissioned = cm.getNumDecommisionedNMs();
+    int initialRebooted = cm.getNumRebootedNMs();
+    node.handle(new RMNodeReconnectEvent(node.getNodeID(), node));
+    Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs());
+    Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs());
+    Assert.assertEquals("Unhealthy Nodes",
+        initialUnhealthy, cm.getUnhealthyNMs());
+    Assert.assertEquals("Decommissioned Nodes",
+        initialDecommissioned, cm.getNumDecommisionedNMs());
+    Assert.assertEquals("Rebooted Nodes",
+        initialRebooted, cm.getNumRebootedNMs());
+    Assert.assertEquals(NodeState.RUNNING, node.getState());
+    Assert.assertNotNull(nodesListManagerEvent);
+    Assert.assertEquals(NodesListManagerEventType.NODE_USABLE,
+        nodesListManagerEvent.getType());
+  }
+
 }

Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java?rev=1537584&r1=1537583&r2=1537584&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java Thu Oct 31 18:49:54 2013
@@ -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.
 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,14 +138,8 @@ 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);
     // assert app1 info is saved
@@ -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,11 +224,17 @@ 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());
     Assert.assertNotNull(loadedApp1);
@@ -331,29 +338,343 @@ public class TestRMRestart {
           new ArrayList<ContainerId>()).getAllocatedContainers());
       Thread.sleep(500);
     }
+    // finish the AMs
+    finishApplicationMaster(loadedApp1, rm2, am1Node, am1);
+    finishApplicationMaster(loadedApp2, rm2, am2Node, am2);
 
-    // 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);
-    
     // 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()));
-
-    // verify that app2 is stored, app1 is removed
-    Assert.assertNotNull(rmAppState.get(app2.getApplicationId()));
-    Assert.assertNull(rmAppState.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);
+
+    // 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();

Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java?rev=1537584&r1=1537583&r2=1537584&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java Thu Oct 31 18:49:54 2013
@@ -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.record
 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.res
 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 extend
     }
 
     @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 extend
     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 extend
         .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 extend
   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 extend
     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 extend
     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 extend
 
     // 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 extend
     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 extend
         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());
 

Modified: hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java?rev=1537584&r1=1537583&r2=1537584&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java Thu Oct 31 18:49:54 2013
@@ -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;
-  };
+  }
 }



Mime
View raw message