Prechádzať zdrojové kódy

Reverting patches from branch-2.3 to help address stabilization issues.
Reverting YARN-1493.
Reverting YARN-1490.
Reverting YARN-1041
Reverting YARN-1166.
Reverting YARN-1566.
Reverting YARN-1689.
Reverting YARN-1661.
Updated CHANGES.txt entry also to exlude these patches.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.3@1565792 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 11 rokov pred
rodič
commit
c520584f02
64 zmenil súbory, kde vykonal 1566 pridanie a 2847 odobranie
  1. 18 24
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
  2. 0 22
      hadoop-yarn-project/CHANGES.txt
  3. 1 30
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
  4. 1 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
  5. 4 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
  6. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  7. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  8. 20 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  9. 1 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  10. 4 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/ContainerLaunchFailAppMaster.java
  11. 0 79
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDSFailedAppMaster.java
  12. 0 29
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  13. 0 52
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterResponsePBImpl.java
  14. 0 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
  15. 9 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  16. 1 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java
  17. 1 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppFailedAttemptEvent.java
  18. 35 80
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  19. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptEventType.java
  20. 73 61
      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
  21. 10 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptRejectedEvent.java
  22. 0 64
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
  23. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ActiveUsersManager.java
  24. 5 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
  25. 25 41
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
  26. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppReport.java
  27. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.java
  28. 367 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java
  29. 0 435
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
  30. 1 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
  31. 10 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
  32. 99 170
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  33. 35 41
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  34. 14 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
  35. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
  36. 4 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
  37. 0 49
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAddedSchedulerEvent.java
  38. 12 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptAddedSchedulerEvent.java
  39. 1 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptRemovedSchedulerEvent.java
  40. 0 43
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppRemovedSchedulerEvent.java
  41. 1 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
  42. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
  43. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java
  44. 80 155
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  45. 67 144
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  46. 4 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
  47. 4 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
  48. 1 74
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  49. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  50. 9 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java
  51. 26 97
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java
  52. 10 142
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  53. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
  54. 269 199
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
  55. 10 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
  56. 21 90
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  57. 24 58
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
  58. 0 25
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
  59. 19 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
  60. 66 29
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
  61. 41 52
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
  62. 112 132
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  63. 24 44
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
  64. 13 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java

+ 18 - 24
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java

@@ -64,9 +64,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@@ -106,8 +105,8 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
 
   private Configuration conf;
   private ResourceScheduler scheduler;
-  private Map<ApplicationId, String> appQueueMap =
-          new ConcurrentHashMap<ApplicationId, String>();
+  private Map<ApplicationAttemptId, String> appQueueMap =
+          new ConcurrentHashMap<ApplicationAttemptId, String>();
   private BufferedWriter jobRuntimeLogBW;
 
   // Priority of the ResourceSchedulerWrapper shutdown hook.
@@ -242,7 +241,7 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
             (AppAttemptRemovedSchedulerEvent) schedulerEvent;
         ApplicationAttemptId appAttemptId =
                 appRemoveEvent.getApplicationAttemptID();
-        String queue = appQueueMap.get(appAttemptId.getApplicationId());
+        String queue = appQueueMap.get(appAttemptId);
         SchedulerAppReport app = scheduler.getSchedulerAppInfo(appAttemptId);
         if (! app.getLiveContainers().isEmpty()) {  // have 0 or 1
           // should have one container which is AM container
@@ -264,18 +263,20 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
       schedulerHandleCounter.inc();
       schedulerHandleCounterMap.get(schedulerEvent.getType()).inc();
 
-      if (schedulerEvent.getType() == SchedulerEventType.APP_REMOVED
-          && schedulerEvent instanceof AppRemovedSchedulerEvent) {
+      if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED
+          && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) {
         SLSRunner.decreaseRemainingApps();
-        AppRemovedSchedulerEvent appRemoveEvent =
-                (AppRemovedSchedulerEvent) schedulerEvent;
-        appQueueMap.remove(appRemoveEvent.getApplicationID());
-      } else if (schedulerEvent.getType() == SchedulerEventType.APP_ADDED
-          && schedulerEvent instanceof AppAddedSchedulerEvent) {
-        AppAddedSchedulerEvent appAddEvent =
-                (AppAddedSchedulerEvent) schedulerEvent;
+        AppAttemptRemovedSchedulerEvent appRemoveEvent =
+                (AppAttemptRemovedSchedulerEvent) schedulerEvent;
+        ApplicationAttemptId appAttemptId =
+                appRemoveEvent.getApplicationAttemptID();
+        appQueueMap.remove(appRemoveEvent.getApplicationAttemptID());
+      } else if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_ADDED
+          && schedulerEvent instanceof AppAttemptAddedSchedulerEvent) {
+        AppAttemptAddedSchedulerEvent appAddEvent =
+                (AppAttemptAddedSchedulerEvent) schedulerEvent;
         String queueName = appAddEvent.getQueue();
-        appQueueMap.put(appAddEvent.getApplicationId(), queueName);
+        appQueueMap.put(appAddEvent.getApplicationAttemptId(), queueName);
       }
     }
   }
@@ -297,9 +298,7 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
           continue;
         }
 
-        String queue =
-            appQueueMap.get(containerId.getApplicationAttemptId()
-              .getApplicationId());
+        String queue = appQueueMap.get(containerId.getApplicationAttemptId());
         int releasedMemory = 0, releasedVCores = 0;
         if (status.getExitStatus() == ContainerExitStatus.SUCCESS) {
           for (RMContainer rmc : app.getLiveContainers()) {
@@ -331,7 +330,7 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
     // update queue information
     Resource pendingResource = Resources.createResource(0, 0);
     Resource allocatedResource = Resources.createResource(0, 0);
-    String queueName = appQueueMap.get(attemptId.getApplicationId());
+    String queueName = appQueueMap.get(attemptId);
     // container requested
     for (ResourceRequest request : resourceRequests) {
       if (request.getResourceName().equals(ResourceRequest.ANY)) {
@@ -861,10 +860,5 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
   public List<ApplicationAttemptId> getAppsInQueue(String queue) {
     return scheduler.getAppsInQueue(queue);
   }
-
-  @Override
-  public RMContainer getRMContainer(ContainerId containerId) {
-    return null;
-  }
 }
 

+ 0 - 22
hadoop-yarn-project/CHANGES.txt

@@ -43,18 +43,8 @@ Release 2.3.0 - UNRELEASED
     YARN-1029. Added embedded leader election in the ResourceManager. (Karthik
     Kambatla via vinodkv)
 
-    YARN-1490. Introduced the ability to make ResourceManager optionally not kill
-    all containers when an ApplicationMaster exits. (Jian He via vinodkv)
-
     YARN-1033. Expose RM active/standby state to Web UI and REST API (kasha)
 
-    YARN-1041. Added the ApplicationMasterProtocol API for applications to use the
-    ability in ResourceManager to optionally not kill containers when the
-    ApplicationMaster exits. (Jian He via vinodkv)
-
-    YARN-1566. Changed Distributed Shell to retain containers across application
-    attempts. (Jian He via vinodkv)
-
   IMPROVEMENTS
 
     YARN-305. Fair scheduler logs too many "Node offered to app" messages.
@@ -218,9 +208,6 @@ Release 2.3.0 - UNRELEASED
     YARN-1541. Changed ResourceManager to invalidate ApplicationMaster host/port
     information once an AM crashes. (Jian He via vinodkv)
 
-    YARN-1493. Changed ResourceManager and Scheduler interfacing to recognize
-    app-attempts separately from apps. (Jian He via vinodkv)
-
     YARN-1482. Modified WebApplicationProxy to make it work across ResourceManager
     fail-over. (Xuan Gong via vinodkv)
 
@@ -406,9 +393,6 @@ Release 2.3.0 - UNRELEASED
     YARN-1574. RMDispatcher should be reset on transition to standby. (Xuan Gong
     via kasha)
 
-    YARN-1166. Fixed app-specific and attempt-specific QueueMetrics to be
-    triggered by accordingly app event and attempt event. 
-
     YARN-1598. HA-related rmadmin commands don't work on a secure cluster (kasha)
 
     YARN-1603. Remove two *.orig files which were unexpectedly committed. 
@@ -436,12 +420,6 @@ Release 2.3.0 - UNRELEASED
     YARN-1628. Fixed the test failure in TestContainerManagerSecurity. (Vinod
     Kumar Vavilapalli via zjshen)
 
-    YARN-1661. Fixed DS ApplicationMaster to write the correct exit log. (Vinod
-    Kumar Vavilapalli via zjshen)
-
-    YARN-1689. Made RMAppAttempt get killed when RMApp is at ACCEPTED. (Vinod
-    Kumar Vavilapalli via zjshen)
-
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 1 - 30
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.api.protocolrecords;
 
 import java.nio.ByteBuffer;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -28,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -49,19 +47,17 @@ import org.apache.hadoop.yarn.util.Records;
 @Public
 @Stable
 public abstract class RegisterApplicationMasterResponse {
-
   @Private
   @Unstable
   public static RegisterApplicationMasterResponse newInstance(
       Resource minCapability, Resource maxCapability,
       Map<ApplicationAccessType, String> acls, ByteBuffer key,
-      List<Container> containersFromPreviousAttempt, String queue) {
+      String queue) {
     RegisterApplicationMasterResponse response =
         Records.newRecord(RegisterApplicationMasterResponse.class);
     response.setMaximumResourceCapability(maxCapability);
     response.setApplicationACLs(acls);
     response.setClientToAMTokenMasterKey(key);
-    response.setContainersFromPreviousAttempt(containersFromPreviousAttempt);
     response.setQueue(queue);
     return response;
   }
@@ -126,29 +122,4 @@ public abstract class RegisterApplicationMasterResponse {
   @Stable
   public abstract void setQueue(String queue);
   
-  /**
-   * <p>
-   * Get the list of running containers as viewed by
-   * <code>ResourceManager</code> from previous application attempt.
-   * </p>
-   * 
-   * @return the list of running containers as viewed by
-   *         <code>ResourceManager</code> from previous application attempt
-   */
-  @Public
-  @Unstable
-  public abstract List<Container> getContainersFromPreviousAttempt();
-
-  /**
-   * Set the list of running containers as viewed by
-   * <code>ResourceManager</code> from previous application attempt.
-   * 
-   * @param containersFromPreviousAttempt
-   *          the list of running containers as viewed by
-   *          <code>ResourceManager</code> from previous application attempt.
-   */
-  @Private
-  @Unstable
-  public abstract void setContainersFromPreviousAttempt(
-      List<Container> containersFromPreviousAttempt);
 }

+ 1 - 47
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java

@@ -24,7 +24,6 @@ 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.ApplicationClientProtocol;
-import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -58,8 +57,7 @@ public abstract class ApplicationSubmissionContext {
       ApplicationId applicationId, String applicationName, String queue,
       Priority priority, ContainerLaunchContext amContainer,
       boolean isUnmanagedAM, boolean cancelTokensWhenComplete,
-      int maxAppAttempts, Resource resource, String applicationType,
-      boolean keepContainers) {
+      int maxAppAttempts, Resource resource, String applicationType) {
     ApplicationSubmissionContext context =
         Records.newRecord(ApplicationSubmissionContext.class);
     context.setApplicationId(applicationId);
@@ -72,22 +70,9 @@ public abstract class ApplicationSubmissionContext {
     context.setMaxAppAttempts(maxAppAttempts);
     context.setResource(resource);
     context.setApplicationType(applicationType);
-    context.setKeepContainersAcrossApplicationAttempts(keepContainers);
     return context;
   }
 
-  @Public
-  @Stable
-  public static ApplicationSubmissionContext newInstance(
-      ApplicationId applicationId, String applicationName, String queue,
-      Priority priority, ContainerLaunchContext amContainer,
-      boolean isUnmanagedAM, boolean cancelTokensWhenComplete,
-      int maxAppAttempts, Resource resource, String applicationType) {
-    return newInstance(applicationId, applicationName, queue, priority,
-      amContainer, isUnmanagedAM, cancelTokensWhenComplete, maxAppAttempts,
-      resource, null, false);
-  }
-
   @Public
   @Stable
   public static ApplicationSubmissionContext newInstance(
@@ -283,35 +268,4 @@ public abstract class ApplicationSubmissionContext {
   @Public
   @Stable
   public abstract void setApplicationType(String applicationType);
-
-
-  /**
-   * Get the flag which indicates whether to keep containers across application
-   * attempts or not.
-   * 
-   * @return the flag which indicates whether to keep containers across
-   *         application attempts or not.
-   */
-  @Public
-  @Stable
-  public abstract boolean getKeepContainersAcrossApplicationAttempts();
-
-  /**
-   * Set the flag which indicates whether to keep containers across application
-   * attempts.
-   * <p>
-   * If the flag is true, running containers will not be killed when application
-   * attempt fails and these containers will be retrieved by the new application
-   * attempt on registration via
-   * {@link ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)}.
-   * </p>
-   * 
-   * @param keepContainers
-   *          the flag which indicates whether to keep containers across
-   *          application attempts.
-   */
-  @Public
-  @Stable
-  public abstract void setKeepContainersAcrossApplicationAttempts(
-      boolean keepContainers);
 }

+ 4 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java

@@ -46,20 +46,10 @@ public abstract class ContainerId implements Comparable<ContainerId>{
   }
 
   /**
-   * Get the <code>ApplicationAttemptId</code> of the application to which the
-   * <code>Container</code> was assigned.
-   * <p>
-   * Note: If containers are kept alive across application attempts via
-   * {@link ApplicationSubmissionContext#setKeepContainersAcrossApplicationAttempts(boolean)}
-   * the <code>ContainerId</code> does not necessarily contain the current
-   * running application attempt's <code>ApplicationAttemptId</code> This
-   * container can be allocated by previously exited application attempt and
-   * managed by the current running attempt thus have the previous application
-   * attempt's <code>ApplicationAttemptId</code>.
-   * </p>
-   * 
-   * @return <code>ApplicationAttemptId</code> of the application to which the
-   *         <code>Container</code> was assigned
+   * Get the <code>ApplicationAttemptId</code> of the application to which
+   * the <code>Container</code> was assigned.
+   * @return <code>ApplicationAttemptId</code> of the application to which
+   *         the <code>Container</code> was assigned
    */
   @Public
   @Stable

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -248,7 +248,6 @@ message ApplicationSubmissionContextProto {
   optional int32 maxAppAttempts = 8 [default = 0];
   optional ResourceProto resource = 9;
   optional string applicationType = 10 [default = "YARN"];
-  optional bool keep_containers_across_application_attempts = 11 [default = false];
 }
 
 enum ApplicationAccessTypeProto {

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto

@@ -44,7 +44,6 @@ message RegisterApplicationMasterResponseProto {
   optional ResourceProto maximumCapability = 1;
   optional bytes client_to_am_token_master_key = 2;
   repeated ApplicationACLMapProto application_ACLs = 3;
-  repeated ContainerProto containers_from_previous_attempt = 4;
   optional string queue = 5;
 }
 

+ 20 - 33
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -37,6 +37,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.HelpFormatter;
@@ -88,8 +89,6 @@ import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * An ApplicationMaster for executing shell commands on a set of launched
  * containers using the YARN framework.
@@ -170,8 +169,7 @@ public class ApplicationMaster {
   private NMCallbackHandler containerListener;
   
   // Application Attempt Id ( combination of attemptId and fail count )
-  @VisibleForTesting
-  protected ApplicationAttemptId appAttemptID;
+  private ApplicationAttemptId appAttemptID;
 
   // TODO
   // For status update for clients - yet to be implemented
@@ -196,15 +194,13 @@ public class ApplicationMaster {
   private AtomicInteger numCompletedContainers = new AtomicInteger();
   // Allocated container count so that we know how many containers has the RM
   // allocated to us
-  @VisibleForTesting
-  protected AtomicInteger numAllocatedContainers = new AtomicInteger();
+  private AtomicInteger numAllocatedContainers = new AtomicInteger();
   // Count of failed containers
   private AtomicInteger numFailedContainers = new AtomicInteger();
   // Count of containers already requested from the RM
   // Needed as once requested, we should not request for containers again.
   // Only request for more if the original requirement changes.
-  @VisibleForTesting
-  protected AtomicInteger numRequestedContainers = new AtomicInteger();
+  private AtomicInteger numRequestedContainers = new AtomicInteger();
 
   // Shell command to be executed
   private String shellCommand = "";
@@ -232,6 +228,7 @@ public class ApplicationMaster {
   private static final String shellArgsPath = "shellArgs";
 
   private volatile boolean done;
+  private volatile boolean success;
 
   private ByteBuffer allTokens;
 
@@ -253,8 +250,7 @@ public class ApplicationMaster {
       if (!doRun) {
         System.exit(0);
       }
-      appMaster.run();
-      result = appMaster.finish();
+      result = appMaster.run();
     } catch (Throwable t) {
       LOG.fatal("Error running ApplicationMaster", t);
       System.exit(1);
@@ -479,7 +475,7 @@ public class ApplicationMaster {
    * @throws IOException
    */
   @SuppressWarnings({ "unchecked" })
-  public void run() throws YarnException, IOException {
+  public boolean run() throws YarnException, IOException {
     LOG.info("Starting ApplicationMaster");
 
     Credentials credentials =
@@ -541,42 +537,35 @@ public class ApplicationMaster {
       containerVirtualCores = maxVCores;
     }
 
-    List<Container> previousAMRunningContainers =
-        response.getContainersFromPreviousAttempt();
-    LOG.info("Received " + previousAMRunningContainers.size()
-        + " previous AM's running containers on AM registration.");
-    numAllocatedContainers.addAndGet(previousAMRunningContainers.size());
-
-    int numTotalContainersToRequest =
-        numTotalContainers - previousAMRunningContainers.size();
     // Setup ask for containers from RM
     // Send request for containers to RM
     // Until we get our fully allocated quota, we keep on polling RM for
     // containers
     // Keep looping until all the containers are launched and shell script
     // executed on them ( regardless of success/failure).
-    for (int i = 0; i < numTotalContainersToRequest; ++i) {
+    for (int i = 0; i < numTotalContainers; ++i) {
       ContainerRequest containerAsk = setupContainerAskForRM();
       amRMClient.addContainerRequest(containerAsk);
     }
-    numRequestedContainers.set(numTotalContainersToRequest);
-  }
-
-  @VisibleForTesting
-  NMCallbackHandler createNMCallbackHandler() {
-    return new NMCallbackHandler(this);
-  }
+    numRequestedContainers.set(numTotalContainers);
 
-  @VisibleForTesting
-  protected boolean finish() {
-    // wait for completion.
     while (!done
         && (numCompletedContainers.get() != numTotalContainers)) {
       try {
         Thread.sleep(200);
       } catch (InterruptedException ex) {}
     }
+    finish();
+    
+    return success;
+  }
+
+  @VisibleForTesting
+  NMCallbackHandler createNMCallbackHandler() {
+    return new NMCallbackHandler(this);
+  }
 
+  private void finish() {
     // Join all launched threads
     // needed for when we time out
     // and we need to release containers
@@ -599,7 +588,7 @@ public class ApplicationMaster {
 
     FinalApplicationStatus appStatus;
     String appMessage = null;
-    boolean success = true;
+    success = true;
     if (numFailedContainers.get() == 0 && 
         numCompletedContainers.get() == numTotalContainers) {
       appStatus = FinalApplicationStatus.SUCCEEDED;
@@ -620,8 +609,6 @@ public class ApplicationMaster {
     }
     
     amRMClient.stop();
-
-    return success;
   }
   
   private class RMCallbackHandler implements AMRMClientAsync.CallbackHandler {

+ 1 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -162,9 +162,6 @@ public class Client {
   // Timeout threshold for client. Kill app after time interval expires.
   private long clientTimeout = 600000;
 
-  // flag to indicate whether to keep containers across application attempts.
-  private boolean keepContainers = false;
-
   // Debug flag
   boolean debugFlag = false;	
 
@@ -246,11 +243,6 @@ public class Client {
     opts.addOption("container_vcores", true, "Amount of virtual cores to be requested to run the shell command");
     opts.addOption("num_containers", true, "No. of containers on which the shell command needs to be executed");
     opts.addOption("log_properties", true, "log4j.properties file");
-    opts.addOption("keep_containers_across_application_attempts", false,
-      "Flag to indicate whether to keep containers across application attempts." +
-      " If the flag is true, running containers will not be killed when" +
-      " application attempt fails and these containers will be retrieved by" +
-      " the new application attempt ");
     opts.addOption("debug", false, "Dump out debug information");
     opts.addOption("help", false, "Print usage");
 
@@ -302,17 +294,12 @@ public class Client {
 
     }
 
-    if (cliParser.hasOption("keep_containers_across_application_attempts")) {
-      LOG.info("keep_containers_across_application_attempts");
-      keepContainers = true;
-    }
-
     appName = cliParser.getOptionValue("appname", "DistributedShell");
     amPriority = Integer.parseInt(cliParser.getOptionValue("priority", "0"));
     amQueue = cliParser.getOptionValue("queue", "default");
     amMemory = Integer.parseInt(cliParser.getOptionValue("master_memory", "10"));		
     amVCores = Integer.parseInt(cliParser.getOptionValue("master_vcores", "1"));
-
+    
     if (amMemory < 0) {
       throw new IllegalArgumentException("Invalid memory specified for application master, exiting."
           + " Specified memory=" + amMemory);
@@ -455,8 +442,6 @@ public class Client {
     // set the application name
     ApplicationSubmissionContext appContext = app.getApplicationSubmissionContext();
     ApplicationId appId = appContext.getApplicationId();
-
-    appContext.setKeepContainersAcrossApplicationAttempts(keepContainers);
     appContext.setApplicationName(appName);
 
     // Set up the container launch context for the application master

+ 4 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/ContainerLaunchFailAppMaster.java

@@ -18,13 +18,13 @@
 
 package org.apache.hadoop.yarn.applications.distributedshell;
 
-import java.nio.ByteBuffer;
-import java.util.Map;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 
+import java.nio.ByteBuffer;
+import java.util.Map;
+
 public class ContainerLaunchFailAppMaster extends ApplicationMaster {
 
   private static final Log LOG =
@@ -66,8 +66,7 @@ public class ContainerLaunchFailAppMaster extends ApplicationMaster {
       if (!doRun) {
         System.exit(0);
       }
-      appMaster.run();
-      result = appMaster.finish();
+      result = appMaster.run();
     } catch (Throwable t) {
       LOG.fatal("Error running ApplicationMaster", t);
       System.exit(1);

+ 0 - 79
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDSFailedAppMaster.java

@@ -1,79 +0,0 @@
-/**
- * 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.applications.distributedshell;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-
-public class TestDSFailedAppMaster extends ApplicationMaster {
-
-  private static final Log LOG = LogFactory.getLog(TestDSFailedAppMaster.class);
-
-  @Override
-  public void run() throws YarnException, IOException {
-    super.run();
-
-    // for the 2nd attempt.
-    if (appAttemptID.getAttemptId() == 2) {
-      // should reuse the earlier running container, so numAllocatedContainers
-      // should be set to 1. And should ask no more containers, so
-      // numRequestedContainers should be set to 0.
-      if (numAllocatedContainers.get() != 1
-          || numRequestedContainers.get() != 0) {
-        LOG.info("NumAllocatedContainers is " + numAllocatedContainers.get()
-            + " and NumRequestedContainers is " + numAllocatedContainers.get()
-            + ".Application Master failed. exiting");
-        System.exit(200);
-      }
-    }
-  }
-
-  public static void main(String[] args) {
-    boolean result = false;
-    try {
-      TestDSFailedAppMaster appMaster = new TestDSFailedAppMaster();
-      boolean doRun = appMaster.init(args);
-      if (!doRun) {
-        System.exit(0);
-      }
-      appMaster.run();
-      if (appMaster.appAttemptID.getAttemptId() == 1) {
-        try {
-          // sleep some time, wait for the AM to launch a container.
-          Thread.sleep(3000);
-        } catch (InterruptedException e) {}
-        // fail the first am.
-        System.exit(100);
-      }
-      result = appMaster.finish();
-    } catch (Throwable t) {
-      System.exit(1);
-    }
-    if (result) {
-      LOG.info("Application Master completed successfully. exiting");
-      System.exit(0);
-    } else {
-      LOG.info("Application Master failed. exiting");
-      System.exit(2);
-    }
-  }
-}

+ 0 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -174,35 +174,6 @@ public class TestDistributedShell {
 
   }
 
-  @Test(timeout=90000)
-  public void testDSRestartWithPreviousRunningContainers() throws Exception {
-    String[] args = {
-        "--jar",
-        APPMASTER_JAR,
-        "--num_containers",
-        "1",
-        "--shell_command",
-        Shell.WINDOWS ? "timeout 8" : "sleep 8",
-        "--master_memory",
-        "512",
-        "--container_memory",
-        "128",
-        "--keep_containers_across_application_attempts"
-      };
-
-      LOG.info("Initializing DS Client");
-      Client client = new Client(TestDSFailedAppMaster.class.getName(),
-        new Configuration(yarnCluster.getConfig()));
-
-      client.init(args);
-      LOG.info("Running DS Client");
-      boolean result = client.run();
-
-      LOG.info("Client run completed. Result=" + result);
-      // application should succeed
-      Assert.assertTrue(result);
-    }
-
   @Test(timeout=90000)
   public void testDSShellWithCustomLogPropertyFile() throws Exception {
     final File basedir =

+ 0 - 52
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterResponsePBImpl.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -30,13 +29,10 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationACLMapProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProtoOrBuilder;
@@ -56,7 +52,6 @@ public class RegisterApplicationMasterResponsePBImpl extends
 
   private Resource maximumResourceCapability;
   private Map<ApplicationAccessType, String> applicationACLS = null;
-  private List<Container> containersFromPreviousAttempt = null;
 
   public RegisterApplicationMasterResponsePBImpl() {
     builder = RegisterApplicationMasterResponseProto.newBuilder();
@@ -110,9 +105,6 @@ public class RegisterApplicationMasterResponsePBImpl extends
     if (this.applicationACLS != null) {
       addApplicationACLs();
     }
-    if (this.containersFromPreviousAttempt != null) {
-      addRunningContainersToProto();
-    }
   }
 
 
@@ -234,24 +226,6 @@ public class RegisterApplicationMasterResponsePBImpl extends
         ByteBuffer.wrap(builder.getClientToAmTokenMasterKey().toByteArray());
     return key;
   }
-
-  @Override
-  public List<Container> getContainersFromPreviousAttempt() {
-    if (this.containersFromPreviousAttempt != null) {
-      return this.containersFromPreviousAttempt;
-    }
-    initRunningContainersList();
-    return this.containersFromPreviousAttempt;
-  }
-
-  @Override
-  public void setContainersFromPreviousAttempt(final List<Container> containers) {
-    if (containers == null) {
-      return;
-    }
-    this.containersFromPreviousAttempt = new ArrayList<Container>();
-    this.containersFromPreviousAttempt.addAll(containers);
-  }
   
   @Override
   public String getQueue() {
@@ -271,25 +245,6 @@ public class RegisterApplicationMasterResponsePBImpl extends
       builder.setQueue(queue);
     }
   }
-
-  private void initRunningContainersList() {
-    RegisterApplicationMasterResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerProto> list = p.getContainersFromPreviousAttemptList();
-    containersFromPreviousAttempt = new ArrayList<Container>();
-    for (ContainerProto c : list) {
-      containersFromPreviousAttempt.add(convertFromProtoFormat(c));
-    }
-  }
-
-  private void addRunningContainersToProto() {
-    maybeInitBuilder();
-    builder.clearContainersFromPreviousAttempt();
-    List<ContainerProto> list = new ArrayList<ContainerProto>();
-    for (Container c : containersFromPreviousAttempt) {
-      list.add(convertToProtoFormat(c));
-    }
-    builder.addAllContainersFromPreviousAttempt(list);
-  }
   
   private Resource convertFromProtoFormat(ResourceProto resource) {
     return new ResourcePBImpl(resource);
@@ -299,11 +254,4 @@ public class RegisterApplicationMasterResponsePBImpl extends
     return ((ResourcePBImpl)resource).getProto();
   }
 
-  private ContainerPBImpl convertFromProtoFormat(ContainerProto p) {
-    return new ContainerPBImpl(p);
-  }
-
-  private ContainerProto convertToProtoFormat(Container t) {
-    return ((ContainerPBImpl) t).getProto();
-  }
 }

+ 0 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java

@@ -298,19 +298,6 @@ extends ApplicationSubmissionContext {
     this.resource = resource;
   }
 
-  @Override
-  public void
-      setKeepContainersAcrossApplicationAttempts(boolean keepContainers) {
-    maybeInitBuilder();
-    builder.setKeepContainersAcrossApplicationAttempts(keepContainers);
-  }
-
-  @Override
-  public boolean getKeepContainersAcrossApplicationAttempts() {
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    return p.getKeepContainersAcrossApplicationAttempts();
-  }
-
   private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
     return new PriorityPBImpl(p);
   }

+ 9 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -49,7 +49,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.PreemptionContainer;
@@ -79,7 +78,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@@ -274,11 +272,6 @@ public class ApplicationMasterService extends AbstractService implements
             .getClientToAMTokenSecretManager()
             .getMasterKey(applicationAttemptId).getEncoded()));        
       }
-
-      List<Container> containerList =
-          ((AbstractYarnScheduler) rScheduler)
-            .getTransferredContainers(applicationAttemptId);
-      response.setContainersFromPreviousAttempt(containerList);
       return response;
     }
   }
@@ -429,26 +422,21 @@ public class ApplicationMasterService extends AbstractService implements
         LOG.warn("Invalid blacklist request by application " + appAttemptId, e);
         throw e;
       }
-
-      RMApp app =
-          this.rmContext.getRMApps().get(appAttemptId.getApplicationId());
-      // In the case of work-preserving AM restart, it's possible for the
-      // AM to release containers from the earlier attempt.
-      if (!app.getApplicationSubmissionContext()
-        .getKeepContainersAcrossApplicationAttempts()) {
-        try {
-          RMServerUtils.validateContainerReleaseRequest(release, appAttemptId);
-        } catch (InvalidContainerReleaseException e) {
-          LOG.warn("Invalid container release by application " + appAttemptId, e);
-          throw e;
-        }
+      
+      try {
+        RMServerUtils.validateContainerReleaseRequest(release, appAttemptId);
+      } catch (InvalidContainerReleaseException e) {
+        LOG.warn("Invalid container release by application " + appAttemptId, e);
+        throw e;
       }
-
+      
       // Send new requests to appAttempt.
       Allocation allocation =
           this.rScheduler.allocate(appAttemptId, ask, release, 
               blacklistAdditions, blacklistRemovals);
 
+      RMApp app = this.rmContext.getRMApps().get(
+          appAttemptId.getApplicationId());
       RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId);
       
       AllocateResponse allocateResponse =

+ 1 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java

@@ -24,13 +24,11 @@ public enum RMAppEventType {
   RECOVER,
   KILL,
 
-  // Source: Scheduler and RMAppManager
+  // Source: RMAppAttempt
   APP_REJECTED,
 
   // Source: Scheduler
   APP_ACCEPTED,
-
-  // Source: RMAppAttempt
   ATTEMPT_REGISTERED,
   ATTEMPT_UNREGISTERED,
   ATTEMPT_FINISHED, // Will send the final state

+ 1 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppFailedAttemptEvent.java

@@ -23,20 +23,14 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 public class RMAppFailedAttemptEvent extends RMAppEvent {
 
   private final String diagnostics;
-  private final boolean transferStateFromPreviousAttempt;
 
   public RMAppFailedAttemptEvent(ApplicationId appId, RMAppEventType event, 
-      String diagnostics, boolean transferStateFromPreviousAttempt) {
+      String diagnostics) {
     super(appId, event);
     this.diagnostics = diagnostics;
-    this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt;
   }
 
   public String getDiagnostics() {
     return this.diagnostics;
   }
-
-  public boolean getTransferStateFromPreviousAttempt() {
-    return transferStateFromPreviousAttempt;
-  }
 }

+ 35 - 80
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -63,12 +63,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppStartAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -77,7 +74,6 @@ import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-@SuppressWarnings({ "rawtypes", "unchecked" })
 public class RMAppImpl implements RMApp, Recoverable {
 
   private static final Log LOG = LogFactory.getLog(RMAppImpl.class);
@@ -140,7 +136,7 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.NEW, RMAppState.NEW_SAVING,
         RMAppEventType.START, new RMAppNewlySavingTransition())
     .addTransition(RMAppState.NEW, EnumSet.of(RMAppState.SUBMITTED,
-            RMAppState.ACCEPTED, RMAppState.FINISHED, RMAppState.FAILED,
+            RMAppState.RUNNING, RMAppState.FINISHED, RMAppState.FAILED,
             RMAppState.KILLED, RMAppState.FINAL_SAVING),
         RMAppEventType.RECOVER, new RMAppRecoveredTransition())
     .addTransition(RMAppState.NEW, RMAppState.KILLED, RMAppEventType.KILL,
@@ -152,7 +148,7 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.NEW_SAVING, RMAppState.NEW_SAVING,
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
     .addTransition(RMAppState.NEW_SAVING, RMAppState.SUBMITTED,
-        RMAppEventType.APP_NEW_SAVED, new AddApplicationToSchedulerTransition())
+        RMAppEventType.APP_NEW_SAVED, new StartAppAttemptTransition())
     .addTransition(RMAppState.NEW_SAVING, RMAppState.FINAL_SAVING,
         RMAppEventType.KILL,
         new FinalSavingTransition(
@@ -170,12 +166,9 @@ public class RMAppImpl implements RMApp, Recoverable {
         new FinalSavingTransition(
           new AppRejectedTransition(), RMAppState.FAILED))
     .addTransition(RMAppState.SUBMITTED, RMAppState.ACCEPTED,
-        RMAppEventType.APP_ACCEPTED, new StartAppAttemptTransition())
-    .addTransition(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING,
-        RMAppEventType.KILL,
-        new FinalSavingTransition(
-          new AppKilledTransition(), RMAppState.KILLED))
-
+        RMAppEventType.APP_ACCEPTED)
+    .addTransition(RMAppState.SUBMITTED, RMAppState.KILLING,
+        RMAppEventType.KILL,new KillAttemptTransition())
 
      // Transitions from ACCEPTED state
     .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
@@ -183,20 +176,11 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.ACCEPTED, RMAppState.RUNNING,
         RMAppEventType.ATTEMPT_REGISTERED)
     .addTransition(RMAppState.ACCEPTED,
-        EnumSet.of(RMAppState.ACCEPTED, RMAppState.FINAL_SAVING),
-        // ACCEPTED state is possible to receive ATTEMPT_FAILED event because
-        // RMAppRecoveredTransition is returning ACCEPTED state directly and
-        // waiting for the previous AM to exit.
+        EnumSet.of(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING),
         RMAppEventType.ATTEMPT_FAILED,
-        new AttemptFailedTransition(RMAppState.ACCEPTED))
+        new AttemptFailedTransition(RMAppState.SUBMITTED))
     .addTransition(RMAppState.ACCEPTED, RMAppState.KILLING,
-        RMAppEventType.KILL, new KillAttemptTransition())
-    // ACCECPTED state can once again receive APP_ACCEPTED event, because on
-    // recovery the app returns ACCEPTED state and the app once again go
-    // through the scheduler and triggers one more APP_ACCEPTED event at
-    // ACCEPTED state.
-    .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
-        RMAppEventType.APP_ACCEPTED)
+        RMAppEventType.KILL,new KillAttemptTransition())
 
      // Transitions from RUNNING state
     .addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
@@ -210,9 +194,9 @@ public class RMAppImpl implements RMApp, Recoverable {
       // UnManagedAM directly jumps to finished
         RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION)
     .addTransition(RMAppState.RUNNING,
-        EnumSet.of(RMAppState.ACCEPTED, RMAppState.FINAL_SAVING),
+        EnumSet.of(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING),
         RMAppEventType.ATTEMPT_FAILED,
-        new AttemptFailedTransition(RMAppState.ACCEPTED))
+        new AttemptFailedTransition(RMAppState.SUBMITTED))
     .addTransition(RMAppState.RUNNING, RMAppState.KILLING,
         RMAppEventType.KILL, new KillAttemptTransition())
 
@@ -630,7 +614,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       this.writeLock.unlock();
     }
   }
-
+  
   @Override
   public void recover(RMState state) throws Exception{
     ApplicationState appState = state.getApplicationState().get(getApplicationId());
@@ -643,28 +627,26 @@ public class RMAppImpl implements RMApp, Recoverable {
 
     for(int i=0; i<appState.getAttemptCount(); ++i) {
       // create attempt
-      createNewAttempt();
+      createNewAttempt(false);
       ((RMAppAttemptImpl)this.currentAttempt).recover(state);
     }
   }
 
-  private void createNewAttempt() {
+  @SuppressWarnings("unchecked")
+  private void createNewAttempt(boolean startAttempt) {
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(applicationId, attempts.size() + 1);
     RMAppAttempt attempt =
         new RMAppAttemptImpl(appAttemptId, rmContext, scheduler, masterService,
-          submissionContext, conf, maxAppAttempts == attempts.size());
+          submissionContext, conf, user);
     attempts.put(appAttemptId, attempt);
     currentAttempt = attempt;
+    if(startAttempt) {
+      handler.handle(
+          new RMAppAttemptEvent(appAttemptId, RMAppAttemptEventType.START));
+    }
   }
   
-  private void
-      createAndStartNewAttempt(boolean transferStateFromPreviousAttempt) {
-    createNewAttempt();
-    handler.handle(new RMAppStartAttemptEvent(currentAttempt.getAppAttemptId(),
-      transferStateFromPreviousAttempt));
-  }
-
   private void processNodeUpdate(RMAppNodeUpdateType type, RMNode node) {
     NodeState nodeState = node.getState();
     updatedNodes.add(node);
@@ -687,6 +669,7 @@ public class RMAppImpl implements RMApp, Recoverable {
     };
   }
 
+  @SuppressWarnings("unchecked")
   private static final class RMAppRecoveredTransition implements
       MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
 
@@ -709,52 +692,37 @@ public class RMAppImpl implements RMApp, Recoverable {
         return app.recoveredFinalState;
       }
 
-      // Notify scheduler about the app on recovery
-      new AddApplicationToSchedulerTransition().transition(app, event);
-
       // No existent attempts means the attempt associated with this app was not
       // started or started but not yet saved.
       if (app.attempts.isEmpty()) {
+        app.createNewAttempt(true);
         return RMAppState.SUBMITTED;
       }
 
-      // YARN-1507 is saving the application state after the application is
-      // accepted. So after YARN-1507, an app is saved meaning it is accepted.
-      // Thus we return ACCECPTED state on recovery.
-      return RMAppState.ACCEPTED;
+      return RMAppState.RUNNING;
     }
   }
 
-  private static final class AddApplicationToSchedulerTransition extends
-      RMAppTransition {
+  private static final class StartAppAttemptTransition extends RMAppTransition {
     @Override
     public void transition(RMAppImpl app, RMAppEvent event) {
-      if (event instanceof RMAppNewSavedEvent) {
-        RMAppNewSavedEvent storeEvent = (RMAppNewSavedEvent) 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
-        if (((RMAppNewSavedEvent) event).getStoredException() != null) {
-          LOG.error(
-            "Failed to store application: " + storeEvent.getApplicationId(),
-            storeEvent.getStoredException());
-          ExitUtil.terminate(1, storeEvent.getStoredException());
-        }
+        LOG.error(
+          "Failed to store application: " + storeEvent.getApplicationId(),
+          storeEvent.getStoredException());
+        ExitUtil.terminate(1, storeEvent.getStoredException());
       }
-      app.handler.handle(new AppAddedSchedulerEvent(app.applicationId,
-        app.submissionContext.getQueue(), app.user));
-    }
-  }
-
-  private static final class StartAppAttemptTransition extends RMAppTransition {
-    @Override
-    public void transition(RMAppImpl app, RMAppEvent event) {
-      app.createAndStartNewAttempt(false);
+      app.createNewAttempt(true);
     };
   }
 
   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;
@@ -955,6 +923,7 @@ public class RMAppImpl implements RMApp, Recoverable {
   }
 
   private static class KillAttemptTransition extends RMAppTransition {
+    @SuppressWarnings("unchecked")
     @Override
     public void transition(RMAppImpl app, RMAppEvent event) {
       app.stateBeforeKilling = app.getState();
@@ -982,6 +951,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       return nodes;
     }
 
+    @SuppressWarnings("unchecked")
     public void transition(RMAppImpl app, RMAppEvent event) {
       Set<NodeId> nodes = getNodesOnWhichAttemptRan(app);
       for (NodeId nodeId : nodes) {
@@ -992,8 +962,6 @@ public class RMAppImpl implements RMApp, Recoverable {
       if (app.finishTime == 0 ) {
         app.finishTime = System.currentTimeMillis();
       }
-      app.handler.handle(new AppRemovedSchedulerEvent(app.applicationId, app
-        .getState()));
       app.handler.handle(
           new RMAppManagerEvent(app.applicationId,
           RMAppManagerEventType.APP_COMPLETED));
@@ -1013,21 +981,7 @@ public class RMAppImpl implements RMApp, Recoverable {
     public RMAppState transition(RMAppImpl app, RMAppEvent event) {
       if (!app.submissionContext.getUnmanagedAM()
           && app.attempts.size() < app.maxAppAttempts) {
-        boolean transferStateFromPreviousAttempt = false;
-        RMAppFailedAttemptEvent failedEvent = (RMAppFailedAttemptEvent) event;
-        transferStateFromPreviousAttempt =
-            failedEvent.getTransferStateFromPreviousAttempt();
-
-        RMAppAttempt oldAttempt = app.currentAttempt;
-        app.createAndStartNewAttempt(transferStateFromPreviousAttempt);
-        // Transfer the state from the previous attempt to the current attempt.
-        // Note that the previous failed attempt may still be collecting the
-        // container events from the scheduler and update its data structures
-        // before the new attempt is created.
-        if (transferStateFromPreviousAttempt) {
-          ((RMAppAttemptImpl) app.currentAttempt)
-            .transferStateFromPreviousAttempt(oldAttempt);
-        }
+        app.createNewAttempt(true);
         return initialState;
       } else {
         app.rememberTargetTransitionsAndStoreState(event,
@@ -1036,6 +990,7 @@ public class RMAppImpl implements RMApp, Recoverable {
         return RMAppState.FINAL_SAVING;
       }
     }
+
   }
 
   @Override

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptEventType.java

@@ -45,7 +45,8 @@ public enum RMAppAttemptEventType {
   ATTEMPT_UPDATE_SAVED,
 
   // Source: Scheduler
-  ATTEMPT_ADDED,
+  APP_REJECTED,
+  APP_ACCEPTED,
   
   // Source: RMAttemptImpl.recover
   RECOVER

+ 73 - 61
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

@@ -75,11 +75,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFinishedAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
 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.RMAppAttemptUnregistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
@@ -129,9 +131,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   private SecretKey clientTokenMasterKey = null;
 
   //nodes on while this attempt's containers ran
-  private Set<NodeId> ranNodes =
+  private final Set<NodeId> ranNodes =
     new HashSet<NodeId>();
-  private List<ContainerStatus> justFinishedContainers =
+  private final List<ContainerStatus> justFinishedContainers =
     new ArrayList<ContainerStatus>();
   private Container masterContainer;
 
@@ -148,7 +150,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   private final StringBuilder diagnostics = new StringBuilder();
 
   private Configuration conf;
-  private final boolean isLastAttempt;
+  private String user;
+  
   private static final ExpiredTransition EXPIRED_TRANSITION =
       new ExpiredTransition();
 
@@ -183,10 +186,14 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           RMAppAttemptEventType.RECOVER, new AttemptRecoveredTransition())
           
       // Transitions from SUBMITTED state
+      .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.ATTEMPT_ADDED,
+          RMAppAttemptEventType.APP_ACCEPTED, 
           new ScheduleTransition())
       .addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.FINAL_SAVING,
           RMAppAttemptEventType.KILL,
@@ -330,12 +337,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
               RMAppAttemptEventType.KILL))
 
       // Transitions from FAILED State
-      // For work-preserving AM restart, failed attempt are still capturing
-      // CONTAINER_FINISHED event and record the finished containers for the
-      // use by the next new attempt.
-      .addTransition(RMAppAttemptState.FAILED, RMAppAttemptState.FAILED,
-          RMAppAttemptEventType.CONTAINER_FINISHED,
-          new ContainerFinishedAtFailedTransition())
       .addTransition(
           RMAppAttemptState.FAILED,
           RMAppAttemptState.FAILED,
@@ -344,7 +345,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
               RMAppAttemptEventType.KILL,
               RMAppAttemptEventType.UNREGISTERED,
               RMAppAttemptEventType.STATUS_UPDATE,
-              RMAppAttemptEventType.CONTAINER_ALLOCATED))
+              RMAppAttemptEventType.CONTAINER_ALLOCATED,
+              RMAppAttemptEventType.CONTAINER_FINISHED))
 
       // Transitions from FINISHING State
       .addTransition(RMAppAttemptState.FINISHING,
@@ -378,7 +380,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       .addTransition(
           RMAppAttemptState.KILLED,
           RMAppAttemptState.KILLED,
-          EnumSet.of(RMAppAttemptEventType.ATTEMPT_ADDED,
+          EnumSet.of(RMAppAttemptEventType.APP_ACCEPTED,
+              RMAppAttemptEventType.APP_REJECTED,
               RMAppAttemptEventType.EXPIRE,
               RMAppAttemptEventType.LAUNCHED,
               RMAppAttemptEventType.LAUNCH_FAILED,
@@ -395,7 +398,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       RMContext rmContext, YarnScheduler scheduler,
       ApplicationMasterService masterService,
       ApplicationSubmissionContext submissionContext,
-      Configuration conf, boolean isLastAttempt) {
+      Configuration conf, String user) {
     this.conf = conf;
     this.applicationAttemptId = appAttemptId;
     this.rmContext = rmContext;
@@ -409,8 +412,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     this.writeLock = lock.writeLock();
 
     this.proxiedTrackingUrl = generateProxyUriWithScheme(null);
-    this.isLastAttempt = isLastAttempt;
+    
     this.stateMachine = stateMachineFactory.make(this);
+    this.user = user;
   }
 
   @Override
@@ -421,7 +425,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   @Override
   public ApplicationSubmissionContext getSubmissionContext() {
     return this.submissionContext;
-  }
+  } 
 
   @Override
   public FinalApplicationStatus getFinalApplicationStatus() {
@@ -690,11 +694,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     this.startTime = attemptState.getStartTime();
   }
 
-  public void transferStateFromPreviousAttempt(RMAppAttempt attempt) {
-    this.justFinishedContainers = attempt.getJustFinishedContainers();
-    this.ranNodes = attempt.getRanNodes();
-  }
-
   private void recoverAppAttemptCredentials(Credentials appAttemptTokens)
       throws IOException {
     if (appAttemptTokens == null) {
@@ -731,12 +730,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     public void transition(RMAppAttemptImpl appAttempt,
         RMAppAttemptEvent event) {
 
-	    boolean transferStateFromPreviousAttempt = false;
-      if (event instanceof RMAppStartAttemptEvent) {
-        transferStateFromPreviousAttempt =
-            ((RMAppStartAttemptEvent) event)
-              .getTransferStateFromPreviousAttempt();
-      }
       appAttempt.startTime = System.currentTimeMillis();
 
       // Register with the ApplicationMasterService
@@ -756,10 +749,36 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           new Token<AMRMTokenIdentifier>(id,
             appAttempt.rmContext.getAMRMTokenSecretManager());
 
-      // Add the applicationAttempt to the scheduler and inform the scheduler
-      // whether to transfer the state from previous attempt.
-      appAttempt.eventHandler.handle(new AppAttemptAddedSchedulerEvent(
-        appAttempt.applicationAttemptId, transferStateFromPreviousAttempt));
+      // Add the applicationAttempt to the scheduler
+      appAttempt.eventHandler.handle(
+          new AppAttemptAddedSchedulerEvent(appAttempt.applicationAttemptId,
+              appAttempt.submissionContext.getQueue(), appAttempt.user));
+    }
+  }
+
+  private static final class AppRejectedTransition extends BaseTransition {
+    @Override
+    public void transition(RMAppAttemptImpl appAttempt,
+        RMAppAttemptEvent event) {
+
+      RMAppAttemptRejectedEvent rejectedEvent = (RMAppAttemptRejectedEvent) event;
+
+      // Tell the AMS. Unregister from the ApplicationMasterService
+      appAttempt.masterService
+          .unregisterAttempt(appAttempt.applicationAttemptId);
+      
+      // Save the diagnostic message
+      String message = rejectedEvent.getMessage();
+      appAttempt.diagnostics.append(message);
+
+      // Send the rejection event to app
+      appAttempt.eventHandler.handle(
+          new RMAppRejectedEvent(
+              rejectedEvent.getApplicationAttemptId().getApplicationId(),
+              message)
+          );
+
+      appAttempt.removeCredentials(appAttempt);
     }
   }
 
@@ -775,6 +794,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
         RMAppAttemptEvent event) {
       if (!appAttempt.submissionContext.getUnmanagedAM()) {
+        // Send the acceptance to the app
+        appAttempt.eventHandler.handle(new RMAppEvent(event
+            .getApplicationAttemptId().getApplicationId(),
+            RMAppEventType.APP_ACCEPTED));
+
         // Request a container for the AM.
         ResourceRequest request =
             BuilderUtils.newResourceRequest(
@@ -894,6 +918,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     FinalApplicationStatus finalStatus = null;
 
     switch (event.getType()) {
+    case APP_REJECTED:
+      RMAppAttemptRejectedEvent rejectedEvent =
+          (RMAppAttemptRejectedEvent) event;
+      diags = rejectedEvent.getMessage();
+      break;
     case LAUNCH_FAILED:
       RMAppAttemptLaunchFailedEvent launchFaileEvent =
           (RMAppAttemptLaunchFailedEvent) event;
@@ -998,7 +1027,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       // Tell the application and the scheduler
       ApplicationId applicationId = appAttemptId.getApplicationId();
       RMAppEvent appEvent = null;
-      boolean keepContainersAcrossAppAttempts = false;
       switch (finalAttemptState) {
         case FINISHED:
         {
@@ -1014,7 +1042,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           appEvent =
               new RMAppFailedAttemptEvent(applicationId,
                   RMAppEventType.ATTEMPT_KILLED,
-                  "Application killed by user.", false);
+                  "Application killed by user.");
         }
         break;
         case FAILED:
@@ -1022,17 +1050,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           // don't leave the tracking URL pointing to a non-existent AM
           appAttempt.setTrackingUrlToRMAppPage();
           appAttempt.invalidateAMHostAndPort();
-          if (appAttempt.submissionContext
-            .getKeepContainersAcrossApplicationAttempts()
-              && !appAttempt.isLastAttempt
-              && !appAttempt.submissionContext.getUnmanagedAM()) {
-            keepContainersAcrossAppAttempts = true;
-          }
           appEvent =
               new RMAppFailedAttemptEvent(applicationId,
-                RMAppEventType.ATTEMPT_FAILED, appAttempt.getDiagnostics(),
-                keepContainersAcrossAppAttempts);
-
+                  RMAppEventType.ATTEMPT_FAILED,
+                  appAttempt.getDiagnostics());
         }
         break;
         default:
@@ -1044,7 +1065,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
       appAttempt.eventHandler.handle(appEvent);
       appAttempt.eventHandler.handle(new AppAttemptRemovedSchedulerEvent(
-        appAttemptId, finalAttemptState, keepContainersAcrossAppAttempts));
+        appAttemptId, finalAttemptState));
       appAttempt.removeCredentials(appAttempt);
     }
   }
@@ -1070,11 +1091,16 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     public void transition(RMAppAttemptImpl appAttempt,
                             RMAppAttemptEvent event) {
       appAttempt.checkAttemptStoreError(event);
-      // TODO Today unmanaged AM client is waiting for app state to be Accepted to
-      // launch the AM. This is broken since we changed to start the attempt
-      // after the application is Accepted. We may need to introduce an attempt
-      // report that client can rely on to query the attempt state and choose to
-      // launch the unmanaged AM.
+      // Send the acceptance to the app
+      // Ideally this should have been done when the scheduler accepted the app.
+      // But its here because until the attempt is saved the client should not
+      // launch the unmanaged AM. Client waits for the app status to be accepted
+      // before doing so. So we have to delay the accepted state until we have 
+      // completed storing the attempt
+      appAttempt.eventHandler.handle(new RMAppEvent(event
+          .getApplicationAttemptId().getApplicationId(),
+          RMAppEventType.APP_ACCEPTED));
+      
       super.transition(appAttempt, event);
     }    
   }
@@ -1376,20 +1402,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
   }
 
-  private static final class ContainerFinishedAtFailedTransition
-      extends BaseTransition {
-    @Override
-    public void
-        transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
-      RMAppAttemptContainerFinishedEvent containerFinishedEvent =
-          (RMAppAttemptContainerFinishedEvent) event;
-      ContainerStatus containerStatus =
-          containerFinishedEvent.getContainerStatus();
-      // Normal container. Add it in completed containers list
-      appAttempt.justFinishedContainers.add(containerStatus);
-    }
-  }
-
   private static class ContainerFinishedFinalStateSavedTransition extends
       BaseTransition {
     @Override

+ 10 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppStartAttemptEvent.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptRejectedEvent.java

@@ -16,21 +16,22 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
+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 RMAppStartAttemptEvent extends RMAppAttemptEvent {
+public class RMAppAttemptRejectedEvent extends RMAppAttemptEvent {
 
-  private final boolean transferStateFromPreviousAttempt;
+  private final String message;
 
-  public RMAppStartAttemptEvent(ApplicationAttemptId appAttemptId,
-      boolean transferStateFromPreviousAttempt) {
-    super(appAttemptId, RMAppAttemptEventType.START);
-    this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt;
+  public RMAppAttemptRejectedEvent(ApplicationAttemptId appAttemptId, String message) {
+    super(appAttemptId, RMAppAttemptEventType.APP_REJECTED);
+    this.message = message;
   }
 
-  public boolean getTransferStateFromPreviousAttempt() {
-    return transferStateFromPreviousAttempt;
+  public String getMessage() {
+    return this.message;
   }
 }

+ 0 - 64
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java

@@ -1,64 +0,0 @@
-/**
- * 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.scheduler;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-
-public class AbstractYarnScheduler {
-
-  protected RMContext rmContext;
-  protected Map<ApplicationId, SchedulerApplication> applications;
-
-  public synchronized List<Container> getTransferredContainers(
-      ApplicationAttemptId currentAttempt) {
-    ApplicationId appId = currentAttempt.getApplicationId();
-    SchedulerApplication app = applications.get(appId);
-    List<Container> containerList = new ArrayList<Container>();
-    RMApp appImpl = this.rmContext.getRMApps().get(appId);
-    if (appImpl.getApplicationSubmissionContext().getUnmanagedAM()) {
-      return containerList;
-    }
-    Collection<RMContainer> liveContainers =
-        app.getCurrentAppAttempt().getLiveContainers();
-    ContainerId amContainerId =
-        rmContext.getRMApps().get(appId).getCurrentAppAttempt()
-          .getMasterContainer().getId();
-    for (RMContainer rmContainer : liveContainers) {
-      if (!rmContainer.getContainerId().equals(amContainerId)) {
-        containerList.add(rmContainer.getContainer());
-      }
-    }
-    return containerList;
-  }
-
-  public Map<ApplicationId, SchedulerApplication> getSchedulerApplications() {
-    return applications;
-  }
-}

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ActiveUsersManager.java

@@ -56,7 +56,7 @@ public class ActiveUsersManager {
    * @param user application user 
    * @param applicationId activated application
    */
-  @Lock({Queue.class, SchedulerApplicationAttempt.class})
+  @Lock({Queue.class, SchedulerApplication.class})
   synchronized public void activateApplication(
       String user, ApplicationId applicationId) {
     Set<ApplicationId> userApps = usersApplications.get(user);
@@ -79,7 +79,7 @@ public class ActiveUsersManager {
    * @param user application user 
    * @param applicationId deactivated application
    */
-  @Lock({Queue.class, SchedulerApplicationAttempt.class})
+  @Lock({Queue.class, SchedulerApplication.class})
   synchronized public void deactivateApplication(
       String user, ApplicationId applicationId) {
     Set<ApplicationId> userApps = usersApplications.get(user);
@@ -102,7 +102,7 @@ public class ActiveUsersManager {
    * resource requests.
    * @return number of active users
    */
-  @Lock({Queue.class, SchedulerApplicationAttempt.class})
+  @Lock({Queue.class, SchedulerApplication.class})
   synchronized public int getNumActiveUsers() {
     return activeUsers;
   }

+ 5 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java

@@ -59,10 +59,10 @@ public class AppSchedulingInfo {
 
   final Set<Priority> priorities = new TreeSet<Priority>(
       new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
-  final Map<Priority, Map<String, ResourceRequest>> requests =
+  final Map<Priority, Map<String, ResourceRequest>> requests = 
     new HashMap<Priority, Map<String, ResourceRequest>>();
-  private Set<String> blacklist = new HashSet<String>();
-
+  final Set<String> blacklist = new HashSet<String>();
+  
   //private final ApplicationStore store;
   private final ActiveUsersManager activeUsersManager;
   
@@ -260,7 +260,7 @@ public class AppSchedulingInfo {
       // once an allocation is done we assume the application is
       // running from scheduler's POV.
       pending = false;
-      metrics.runAppAttempt(applicationId, user);
+      metrics.incrAppsRunning(this, user);
     }
     LOG.debug("allocate: user: " + user + ", memory: "
         + request.getCapability());
@@ -390,7 +390,7 @@ public class AppSchedulingInfo {
                 .getNumContainers()));
       }
     }
-    metrics.finishAppAttempt(applicationId, pending, user);
+    metrics.finishApp(this, rmAppAttemptFinalState);
     
     // Clear requests themselves
     clearRequests();
@@ -399,15 +399,4 @@ public class AppSchedulingInfo {
   public synchronized void setQueue(Queue queue) {
     this.queue = queue;
   }
-
-  public synchronized Set<String> getBlackList() {
-    return this.blacklist;
-  }
-
-  public synchronized void transferStateFromPreviousAppSchedulingInfo(
-      AppSchedulingInfo appInfo) {
-    //    this.priorities = appInfo.getPriorities();
-    //    this.requests = appInfo.getRequests();
-    this.blacklist = appInfo.getBlackList();
-  }
 }

+ 25 - 41
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java

@@ -41,7 +41,7 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
@@ -57,7 +57,7 @@ public class QueueMetrics implements MetricsSource {
   @Metric("# of pending apps") MutableGaugeInt appsPending;
   @Metric("# of apps completed") MutableCounterInt appsCompleted;
   @Metric("# of apps killed") MutableCounterInt appsKilled;
-  @Metric("# of apps failed") MutableCounterInt appsFailed;
+  @Metric("# of apps failed") MutableGaugeInt appsFailed;
 
   @Metric("Allocated memory in MB") MutableGaugeInt allocatedMB;
   @Metric("Allocated CPU in virtual cores") MutableGaugeInt allocatedVCores;
@@ -214,70 +214,54 @@ public class QueueMetrics implements MetricsSource {
     registry.snapshot(collector.addRecord(registry.info()), all);
   }
 
-  public void submitApp(String user) {
-    appsSubmitted.incr();
-    QueueMetrics userMetrics = getUserMetrics(user);
-    if (userMetrics != null) {
-      userMetrics.submitApp(user);
-    }
-    if (parent != null) {
-      parent.submitApp(user);
+  public void submitApp(String user, int attemptId) {
+    if (attemptId == 1) {
+      appsSubmitted.incr();
+    } else {
+      appsFailed.decr();
     }
-  }
-
-  public void submitAppAttempt(String user) {
     appsPending.incr();
     QueueMetrics userMetrics = getUserMetrics(user);
     if (userMetrics != null) {
-      userMetrics.submitAppAttempt(user);
+      userMetrics.submitApp(user, attemptId);
     }
     if (parent != null) {
-      parent.submitAppAttempt(user);
+      parent.submitApp(user, attemptId);
     }
   }
 
-  public void runAppAttempt(ApplicationId appId, String user) {
-    runBuckets.add(appId, System.currentTimeMillis());
+  public void incrAppsRunning(AppSchedulingInfo app, String user) {
+    runBuckets.add(app.getApplicationId(), System.currentTimeMillis());
     appsRunning.incr();
     appsPending.decr();
     QueueMetrics userMetrics = getUserMetrics(user);
     if (userMetrics != null) {
-      userMetrics.runAppAttempt(appId, user);
+      userMetrics.incrAppsRunning(app, user);
     }
     if (parent != null) {
-      parent.runAppAttempt(appId, user);
+      parent.incrAppsRunning(app, user);
     }
   }
 
-  public void finishAppAttempt(
-      ApplicationId appId, boolean isPending, String user) {
-    runBuckets.remove(appId);
-    if (isPending) {
-      appsPending.decr();
-    } else {
-      appsRunning.decr();
-    }
-    QueueMetrics userMetrics = getUserMetrics(user);
-    if (userMetrics != null) {
-      userMetrics.finishAppAttempt(appId, isPending, user);
-    }
-    if (parent != null) {
-      parent.finishAppAttempt(appId, isPending, user);
-    }
-  }
-
-  public void finishApp(String user, RMAppState rmAppFinalState) {
-    switch (rmAppFinalState) {
+  public void finishApp(AppSchedulingInfo app,
+      RMAppAttemptState rmAppAttemptFinalState) {
+    runBuckets.remove(app.getApplicationId());
+    switch (rmAppAttemptFinalState) {
       case KILLED: appsKilled.incr(); break;
       case FAILED: appsFailed.incr(); break;
       default: appsCompleted.incr();  break;
     }
-    QueueMetrics userMetrics = getUserMetrics(user);
+    if (app.isPending()) {
+      appsPending.decr();
+    } else {
+      appsRunning.decr();
+    }
+    QueueMetrics userMetrics = getUserMetrics(app.getUser());
     if (userMetrics != null) {
-      userMetrics.finishApp(user, rmAppFinalState);
+      userMetrics.finishApp(app, rmAppAttemptFinalState);
     }
     if (parent != null) {
-      parent.finishApp(user, rmAppFinalState);
+      parent.finishApp(app, rmAppAttemptFinalState);
     }
   }
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppReport.java

@@ -36,7 +36,7 @@ public class SchedulerAppReport {
   private final Collection<RMContainer> reserved;
   private final boolean pending;
   
-  public SchedulerAppReport(SchedulerApplicationAttempt app) {
+  public SchedulerAppReport(SchedulerApplication app) {
     this.live = app.getLiveContainers();
     this.reserved = app.getReservedContainers();
     this.pending = app.isPending();

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.java

@@ -22,7 +22,7 @@ import org.apache.commons.logging.Log;
 
 public class SchedulerAppUtils {
 
-  public static  boolean isBlacklisted(SchedulerApplicationAttempt application,
+  public static  boolean isBlacklisted(SchedulerApplication application,
       SchedulerNode node, Log LOG) {
     if (application.isBlacklisted(node.getNodeName())) {
       if (LOG.isDebugEnabled()) {

+ 367 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java

@@ -17,41 +17,393 @@
 */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Multiset;
 
+/**
+ * Represents an application attempt from the viewpoint of the scheduler.
+ * Each running app attempt in the RM corresponds to one instance
+ * of this class.
+ */
 @Private
 @Unstable
-public class SchedulerApplication {
+public abstract class SchedulerApplication {
+  
+  private static final Log LOG = LogFactory.getLog(SchedulerApplication.class);
+
+  protected final AppSchedulingInfo appSchedulingInfo;
+  
+  protected final Map<ContainerId, RMContainer> liveContainers =
+      new HashMap<ContainerId, RMContainer>();
+  protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers = 
+      new HashMap<Priority, Map<NodeId, RMContainer>>();
 
-  private final Queue queue;
-  private final String user;
-  private SchedulerApplicationAttempt currentAttempt;
+  private final Multiset<Priority> reReservations = HashMultiset.create();
+  
+  protected final Resource currentReservation = Resource.newInstance(0, 0);
+  private Resource resourceLimit = Resource.newInstance(0, 0);
+  protected final Resource currentConsumption = Resource.newInstance(0, 0);
 
-  public SchedulerApplication(Queue queue, String user) {
+  protected List<RMContainer> newlyAllocatedContainers = 
+      new ArrayList<RMContainer>();
+
+  /**
+   * Count how many times the application has been given an opportunity
+   * to schedule a task at each priority. Each time the scheduler
+   * asks the application for a task at this priority, it is incremented,
+   * and each time the application successfully schedules a task, it
+   * is reset to 0.
+   */
+  Multiset<Priority> schedulingOpportunities = HashMultiset.create();
+  
+  // Time of the last container scheduled at the current allowed level
+  protected Map<Priority, Long> lastScheduledContainer =
+      new HashMap<Priority, Long>();
+
+  protected final Queue queue;
+  protected boolean isStopped = false;
+  
+  protected final RMContext rmContext;
+  
+  public SchedulerApplication(ApplicationAttemptId applicationAttemptId, 
+      String user, Queue queue, ActiveUsersManager activeUsersManager,
+      RMContext rmContext) {
+    this.rmContext = rmContext;
+    this.appSchedulingInfo = 
+        new AppSchedulingInfo(applicationAttemptId, user, queue,  
+            activeUsersManager);
     this.queue = queue;
-    this.user = user;
+  }
+  
+  /**
+   * Get the live containers of the application.
+   * @return live containers of the application
+   */
+  public synchronized Collection<RMContainer> getLiveContainers() {
+    return new ArrayList<RMContainer>(liveContainers.values());
+  }
+  
+  /**
+   * Is this application pending?
+   * @return true if it is else false.
+   */
+  public boolean isPending() {
+    return appSchedulingInfo.isPending();
+  }
+  
+  /**
+   * Get {@link ApplicationAttemptId} of the application master.
+   * @return <code>ApplicationAttemptId</code> of the application master
+   */
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return appSchedulingInfo.getApplicationAttemptId();
+  }
+  
+  public ApplicationId getApplicationId() {
+    return appSchedulingInfo.getApplicationId();
+  }
+  
+  public String getUser() {
+    return appSchedulingInfo.getUser();
+  }
+
+  public Map<String, ResourceRequest> getResourceRequests(Priority priority) {
+    return appSchedulingInfo.getResourceRequests(priority);
+  }
+
+  public int getNewContainerId() {
+    return appSchedulingInfo.getNewContainerId();
+  }
+
+  public Collection<Priority> getPriorities() {
+    return appSchedulingInfo.getPriorities();
+  }
+  
+  public ResourceRequest getResourceRequest(Priority priority, String resourceName) {
+    return this.appSchedulingInfo.getResourceRequest(priority, resourceName);
+  }
+
+  public synchronized int getTotalRequiredResources(Priority priority) {
+    return getResourceRequest(priority, ResourceRequest.ANY).getNumContainers();
+  }
+
+  public Resource getResource(Priority priority) {
+    return appSchedulingInfo.getResource(priority);
+  }
+
+  public String getQueueName() {
+    return appSchedulingInfo.getQueueName();
+  }
+  
+  public synchronized RMContainer getRMContainer(ContainerId id) {
+    return liveContainers.get(id);
+  }
+
+  protected synchronized void resetReReservations(Priority priority) {
+    reReservations.setCount(priority, 0);
   }
 
+  protected synchronized void addReReservation(Priority priority) {
+    reReservations.add(priority);
+  }
+
+  public synchronized int getReReservations(Priority priority) {
+    return reReservations.count(priority);
+  }
+
+  /**
+   * Get total current reservations.
+   * Used only by unit tests
+   * @return total current reservations
+   */
+  @Stable
+  @Private
+  public synchronized Resource getCurrentReservation() {
+    return currentReservation;
+  }
+  
   public Queue getQueue() {
     return queue;
   }
+  
+  public synchronized void updateResourceRequests(
+      List<ResourceRequest> requests) {
+    if (!isStopped) {
+      appSchedulingInfo.updateResourceRequests(requests);
+    }
+  }
+  
+  public synchronized void stop(RMAppAttemptState rmAppAttemptFinalState) {
+    // Cleanup all scheduling information
+    isStopped = true;
+    appSchedulingInfo.stop(rmAppAttemptFinalState);
+  }
 
-  public String getUser() {
-    return user;
+  public synchronized boolean isStopped() {
+    return isStopped;
   }
 
-  public SchedulerApplicationAttempt getCurrentAppAttempt() {
-    return currentAttempt;
+  /**
+   * Get the list of reserved containers
+   * @return All of the reserved containers.
+   */
+  public synchronized List<RMContainer> getReservedContainers() {
+    List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
+    for (Map.Entry<Priority, Map<NodeId, RMContainer>> e : 
+      this.reservedContainers.entrySet()) {
+      reservedContainers.addAll(e.getValue().values());
+    }
+    return reservedContainers;
+  }
+  
+  public synchronized RMContainer reserve(SchedulerNode node, Priority priority,
+      RMContainer rmContainer, Container container) {
+    // Create RMContainer if necessary
+    if (rmContainer == null) {
+      rmContainer = 
+          new RMContainerImpl(container, getApplicationAttemptId(), 
+              node.getNodeID(), rmContext.getDispatcher().getEventHandler(), 
+              rmContext.getContainerAllocationExpirer());
+        
+      Resources.addTo(currentReservation, container.getResource());
+      
+      // Reset the re-reservation count
+      resetReReservations(priority);
+    } else {
+      // Note down the re-reservation
+      addReReservation(priority);
+    }
+    rmContainer.handle(new RMContainerReservedEvent(container.getId(), 
+        container.getResource(), node.getNodeID(), priority));
+    
+    Map<NodeId, RMContainer> reservedContainers = 
+        this.reservedContainers.get(priority);
+    if (reservedContainers == null) {
+      reservedContainers = new HashMap<NodeId, RMContainer>();
+      this.reservedContainers.put(priority, reservedContainers);
+    }
+    reservedContainers.put(node.getNodeID(), rmContainer);
+    
+    LOG.info("Application " + getApplicationId() 
+        + " reserved container " + rmContainer
+        + " on node " + node + ", currently has " + reservedContainers.size()
+        + " at priority " + priority 
+        + "; currentReservation " + currentReservation.getMemory());
+    
+    return rmContainer;
+  }
+  
+  /**
+   * Has the application reserved the given <code>node</code> at the
+   * given <code>priority</code>?
+   * @param node node to be checked
+   * @param priority priority of reserved container
+   * @return true is reserved, false if not
+   */
+  public synchronized boolean isReserved(SchedulerNode node, Priority priority) {
+    Map<NodeId, RMContainer> reservedContainers = 
+        this.reservedContainers.get(priority);
+    if (reservedContainers != null) {
+      return reservedContainers.containsKey(node.getNodeID());
+    }
+    return false;
+  }
+  
+  public synchronized void setHeadroom(Resource globalLimit) {
+    this.resourceLimit = globalLimit; 
   }
 
-  public void setCurrentAppAttempt(SchedulerApplicationAttempt currentAttempt) {
-    this.currentAttempt = currentAttempt;
+  /**
+   * Get available headroom in terms of resources for the application's user.
+   * @return available resource headroom
+   */
+  public synchronized Resource getHeadroom() {
+    // Corner case to deal with applications being slightly over-limit
+    if (resourceLimit.getMemory() < 0) {
+      resourceLimit.setMemory(0);
+    }
+    
+    return resourceLimit;
   }
+  
+  public synchronized int getNumReservedContainers(Priority priority) {
+    Map<NodeId, RMContainer> reservedContainers = 
+        this.reservedContainers.get(priority);
+    return (reservedContainers == null) ? 0 : reservedContainers.size();
+  }
+  
+  @SuppressWarnings("unchecked")
+  public synchronized void containerLaunchedOnNode(ContainerId containerId,
+      NodeId nodeId) {
+    // Inform the container
+    RMContainer rmContainer = getRMContainer(containerId);
+    if (rmContainer == null) {
+      // Some unknown container sneaked into the system. Kill it.
+      rmContext.getDispatcher().getEventHandler()
+        .handle(new RMNodeCleanContainerEvent(nodeId, containerId));
+      return;
+    }
 
-  public void stop(RMAppState rmAppFinalState) {
-    queue.getMetrics().finishApp(user, rmAppFinalState);
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.LAUNCHED));
+  }
+  
+  public synchronized void showRequests() {
+    if (LOG.isDebugEnabled()) {
+      for (Priority priority : getPriorities()) {
+        Map<String, ResourceRequest> requests = getResourceRequests(priority);
+        if (requests != null) {
+          LOG.debug("showRequests:" + " application=" + getApplicationId() + 
+              " headRoom=" + getHeadroom() + 
+              " currentConsumption=" + currentConsumption.getMemory());
+          for (ResourceRequest request : requests.values()) {
+            LOG.debug("showRequests:" + " application=" + getApplicationId()
+                + " request=" + request);
+          }
+        }
+      }
+    }
+  }
+  
+  public Resource getCurrentConsumption() {
+    return currentConsumption;
+  }
+
+  public synchronized List<Container> pullNewlyAllocatedContainers() {
+    List<Container> returnContainerList = new ArrayList<Container>(
+        newlyAllocatedContainers.size());
+    for (RMContainer rmContainer : newlyAllocatedContainers) {
+      rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
+          RMContainerEventType.ACQUIRED));
+      returnContainerList.add(rmContainer.getContainer());
+    }
+    newlyAllocatedContainers.clear();
+    return returnContainerList;
+  }
+
+  public synchronized void updateBlacklist(
+      List<String> blacklistAdditions, List<String> blacklistRemovals) {
+    if (!isStopped) {
+      this.appSchedulingInfo.updateBlacklist(
+          blacklistAdditions, blacklistRemovals);
+    }
+  }
+  
+  public boolean isBlacklisted(String resourceName) {
+    return this.appSchedulingInfo.isBlacklisted(resourceName);
+  }
+
+  public synchronized void addSchedulingOpportunity(Priority priority) {
+    schedulingOpportunities.setCount(priority,
+        schedulingOpportunities.count(priority) + 1);
+  }
+  
+  public synchronized void subtractSchedulingOpportunity(Priority priority) {
+    int count = schedulingOpportunities.count(priority) - 1;
+    this.schedulingOpportunities.setCount(priority, Math.max(count,  0));
+  }
+
+  /**
+   * Return the number of times the application has been given an opportunity
+   * to schedule a task at the given priority since the last time it
+   * successfully did so.
+   */
+  public synchronized int getSchedulingOpportunities(Priority priority) {
+    return schedulingOpportunities.count(priority);
+  }
+  
+  /**
+   * Should be called when an application has successfully scheduled a container,
+   * or when the scheduling locality threshold is relaxed.
+   * Reset various internal counters which affect delay scheduling
+   *
+   * @param priority The priority of the container scheduled.
+   */
+  public synchronized void resetSchedulingOpportunities(Priority priority) {
+    resetSchedulingOpportunities(priority, System.currentTimeMillis());
+  }
+  // used for continuous scheduling
+  public synchronized void resetSchedulingOpportunities(Priority priority,
+      long currentTimeMs) {
+    lastScheduledContainer.put(priority, currentTimeMs);
+    schedulingOpportunities.setCount(priority, 0);
+  }
+  
+  public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
+    return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
+        reservedContainers.size(), Resources.clone(currentConsumption),
+        Resources.clone(currentReservation),
+        Resources.add(currentConsumption, currentReservation));
   }
 
 }

+ 0 - 435
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java

@@ -1,435 +0,0 @@
-/**
-* 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.scheduler;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Stable;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.Multiset;
-
-/**
- * Represents an application attempt from the viewpoint of the scheduler.
- * Each running app attempt in the RM corresponds to one instance
- * of this class.
- */
-@Private
-@Unstable
-public abstract class SchedulerApplicationAttempt {
-  
-  private static final Log LOG = LogFactory
-    .getLog(SchedulerApplicationAttempt.class);
-
-  protected final AppSchedulingInfo appSchedulingInfo;
-  
-  protected Map<ContainerId, RMContainer> liveContainers =
-      new HashMap<ContainerId, RMContainer>();
-  protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers = 
-      new HashMap<Priority, Map<NodeId, RMContainer>>();
-
-  private final Multiset<Priority> reReservations = HashMultiset.create();
-  
-  protected final Resource currentReservation = Resource.newInstance(0, 0);
-  private Resource resourceLimit = Resource.newInstance(0, 0);
-  protected Resource currentConsumption = Resource.newInstance(0, 0);
-
-  protected List<RMContainer> newlyAllocatedContainers = 
-      new ArrayList<RMContainer>();
-
-  /**
-   * Count how many times the application has been given an opportunity
-   * to schedule a task at each priority. Each time the scheduler
-   * asks the application for a task at this priority, it is incremented,
-   * and each time the application successfully schedules a task, it
-   * is reset to 0.
-   */
-  Multiset<Priority> schedulingOpportunities = HashMultiset.create();
-  
-  // Time of the last container scheduled at the current allowed level
-  protected Map<Priority, Long> lastScheduledContainer =
-      new HashMap<Priority, Long>();
-
-  protected final Queue queue;
-  protected boolean isStopped = false;
-  
-  protected final RMContext rmContext;
-  
-  public SchedulerApplicationAttempt(ApplicationAttemptId applicationAttemptId, 
-      String user, Queue queue, ActiveUsersManager activeUsersManager,
-      RMContext rmContext) {
-    this.rmContext = rmContext;
-    this.appSchedulingInfo = 
-        new AppSchedulingInfo(applicationAttemptId, user, queue,  
-            activeUsersManager);
-    this.queue = queue;
-  }
-  
-  /**
-   * Get the live containers of the application.
-   * @return live containers of the application
-   */
-  public synchronized Collection<RMContainer> getLiveContainers() {
-    return new ArrayList<RMContainer>(liveContainers.values());
-  }
-  
-  /**
-   * Is this application pending?
-   * @return true if it is else false.
-   */
-  public boolean isPending() {
-    return appSchedulingInfo.isPending();
-  }
-  
-  /**
-   * Get {@link ApplicationAttemptId} of the application master.
-   * @return <code>ApplicationAttemptId</code> of the application master
-   */
-  public ApplicationAttemptId getApplicationAttemptId() {
-    return appSchedulingInfo.getApplicationAttemptId();
-  }
-  
-  public ApplicationId getApplicationId() {
-    return appSchedulingInfo.getApplicationId();
-  }
-  
-  public String getUser() {
-    return appSchedulingInfo.getUser();
-  }
-
-  public Map<String, ResourceRequest> getResourceRequests(Priority priority) {
-    return appSchedulingInfo.getResourceRequests(priority);
-  }
-
-  public int getNewContainerId() {
-    return appSchedulingInfo.getNewContainerId();
-  }
-
-  public Collection<Priority> getPriorities() {
-    return appSchedulingInfo.getPriorities();
-  }
-  
-  public ResourceRequest getResourceRequest(Priority priority, String resourceName) {
-    return this.appSchedulingInfo.getResourceRequest(priority, resourceName);
-  }
-
-  public synchronized int getTotalRequiredResources(Priority priority) {
-    return getResourceRequest(priority, ResourceRequest.ANY).getNumContainers();
-  }
-
-  public Resource getResource(Priority priority) {
-    return appSchedulingInfo.getResource(priority);
-  }
-
-  public String getQueueName() {
-    return appSchedulingInfo.getQueueName();
-  }
-  
-  public synchronized RMContainer getRMContainer(ContainerId id) {
-    return liveContainers.get(id);
-  }
-
-  protected synchronized void resetReReservations(Priority priority) {
-    reReservations.setCount(priority, 0);
-  }
-
-  protected synchronized void addReReservation(Priority priority) {
-    reReservations.add(priority);
-  }
-
-  public synchronized int getReReservations(Priority priority) {
-    return reReservations.count(priority);
-  }
-
-  /**
-   * Get total current reservations.
-   * Used only by unit tests
-   * @return total current reservations
-   */
-  @Stable
-  @Private
-  public synchronized Resource getCurrentReservation() {
-    return currentReservation;
-  }
-  
-  public Queue getQueue() {
-    return queue;
-  }
-  
-  public synchronized void updateResourceRequests(
-      List<ResourceRequest> requests) {
-    if (!isStopped) {
-      appSchedulingInfo.updateResourceRequests(requests);
-    }
-  }
-  
-  public synchronized void stop(RMAppAttemptState rmAppAttemptFinalState) {
-    // Cleanup all scheduling information
-    isStopped = true;
-    appSchedulingInfo.stop(rmAppAttemptFinalState);
-  }
-
-  public synchronized boolean isStopped() {
-    return isStopped;
-  }
-
-  /**
-   * Get the list of reserved containers
-   * @return All of the reserved containers.
-   */
-  public synchronized List<RMContainer> getReservedContainers() {
-    List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
-    for (Map.Entry<Priority, Map<NodeId, RMContainer>> e : 
-      this.reservedContainers.entrySet()) {
-      reservedContainers.addAll(e.getValue().values());
-    }
-    return reservedContainers;
-  }
-  
-  public synchronized RMContainer reserve(SchedulerNode node, Priority priority,
-      RMContainer rmContainer, Container container) {
-    // Create RMContainer if necessary
-    if (rmContainer == null) {
-      rmContainer = 
-          new RMContainerImpl(container, getApplicationAttemptId(), 
-              node.getNodeID(), rmContext.getDispatcher().getEventHandler(), 
-              rmContext.getContainerAllocationExpirer());
-        
-      Resources.addTo(currentReservation, container.getResource());
-      
-      // Reset the re-reservation count
-      resetReReservations(priority);
-    } else {
-      // Note down the re-reservation
-      addReReservation(priority);
-    }
-    rmContainer.handle(new RMContainerReservedEvent(container.getId(), 
-        container.getResource(), node.getNodeID(), priority));
-    
-    Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(priority);
-    if (reservedContainers == null) {
-      reservedContainers = new HashMap<NodeId, RMContainer>();
-      this.reservedContainers.put(priority, reservedContainers);
-    }
-    reservedContainers.put(node.getNodeID(), rmContainer);
-    
-    LOG.info("Application " + getApplicationId() 
-        + " reserved container " + rmContainer
-        + " on node " + node + ", currently has " + reservedContainers.size()
-        + " at priority " + priority 
-        + "; currentReservation " + currentReservation.getMemory());
-    
-    return rmContainer;
-  }
-  
-  /**
-   * Has the application reserved the given <code>node</code> at the
-   * given <code>priority</code>?
-   * @param node node to be checked
-   * @param priority priority of reserved container
-   * @return true is reserved, false if not
-   */
-  public synchronized boolean isReserved(SchedulerNode node, Priority priority) {
-    Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(priority);
-    if (reservedContainers != null) {
-      return reservedContainers.containsKey(node.getNodeID());
-    }
-    return false;
-  }
-  
-  public synchronized void setHeadroom(Resource globalLimit) {
-    this.resourceLimit = globalLimit; 
-  }
-
-  /**
-   * Get available headroom in terms of resources for the application's user.
-   * @return available resource headroom
-   */
-  public synchronized Resource getHeadroom() {
-    // Corner case to deal with applications being slightly over-limit
-    if (resourceLimit.getMemory() < 0) {
-      resourceLimit.setMemory(0);
-    }
-    
-    return resourceLimit;
-  }
-  
-  public synchronized int getNumReservedContainers(Priority priority) {
-    Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(priority);
-    return (reservedContainers == null) ? 0 : reservedContainers.size();
-  }
-  
-  @SuppressWarnings("unchecked")
-  public synchronized void containerLaunchedOnNode(ContainerId containerId,
-      NodeId nodeId) {
-    // Inform the container
-    RMContainer rmContainer = getRMContainer(containerId);
-    if (rmContainer == null) {
-      // Some unknown container sneaked into the system. Kill it.
-      rmContext.getDispatcher().getEventHandler()
-        .handle(new RMNodeCleanContainerEvent(nodeId, containerId));
-      return;
-    }
-
-    rmContainer.handle(new RMContainerEvent(containerId,
-        RMContainerEventType.LAUNCHED));
-  }
-  
-  public synchronized void showRequests() {
-    if (LOG.isDebugEnabled()) {
-      for (Priority priority : getPriorities()) {
-        Map<String, ResourceRequest> requests = getResourceRequests(priority);
-        if (requests != null) {
-          LOG.debug("showRequests:" + " application=" + getApplicationId() + 
-              " headRoom=" + getHeadroom() + 
-              " currentConsumption=" + currentConsumption.getMemory());
-          for (ResourceRequest request : requests.values()) {
-            LOG.debug("showRequests:" + " application=" + getApplicationId()
-                + " request=" + request);
-          }
-        }
-      }
-    }
-  }
-  
-  public Resource getCurrentConsumption() {
-    return currentConsumption;
-  }
-
-  public synchronized List<Container> pullNewlyAllocatedContainers() {
-    List<Container> returnContainerList = new ArrayList<Container>(
-        newlyAllocatedContainers.size());
-    for (RMContainer rmContainer : newlyAllocatedContainers) {
-      rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
-          RMContainerEventType.ACQUIRED));
-      returnContainerList.add(rmContainer.getContainer());
-    }
-    newlyAllocatedContainers.clear();
-    return returnContainerList;
-  }
-
-  public synchronized void updateBlacklist(
-      List<String> blacklistAdditions, List<String> blacklistRemovals) {
-    if (!isStopped) {
-      this.appSchedulingInfo.updateBlacklist(
-          blacklistAdditions, blacklistRemovals);
-    }
-  }
-  
-  public boolean isBlacklisted(String resourceName) {
-    return this.appSchedulingInfo.isBlacklisted(resourceName);
-  }
-
-  public synchronized void addSchedulingOpportunity(Priority priority) {
-    schedulingOpportunities.setCount(priority,
-        schedulingOpportunities.count(priority) + 1);
-  }
-  
-  public synchronized void subtractSchedulingOpportunity(Priority priority) {
-    int count = schedulingOpportunities.count(priority) - 1;
-    this.schedulingOpportunities.setCount(priority, Math.max(count,  0));
-  }
-
-  /**
-   * Return the number of times the application has been given an opportunity
-   * to schedule a task at the given priority since the last time it
-   * successfully did so.
-   */
-  public synchronized int getSchedulingOpportunities(Priority priority) {
-    return schedulingOpportunities.count(priority);
-  }
-  
-  /**
-   * Should be called when an application has successfully scheduled a container,
-   * or when the scheduling locality threshold is relaxed.
-   * Reset various internal counters which affect delay scheduling
-   *
-   * @param priority The priority of the container scheduled.
-   */
-  public synchronized void resetSchedulingOpportunities(Priority priority) {
-    resetSchedulingOpportunities(priority, System.currentTimeMillis());
-  }
-  // used for continuous scheduling
-  public synchronized void resetSchedulingOpportunities(Priority priority,
-      long currentTimeMs) {
-    lastScheduledContainer.put(priority, currentTimeMs);
-    schedulingOpportunities.setCount(priority, 0);
-  }
-  
-  public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
-    return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
-        reservedContainers.size(), Resources.clone(currentConsumption),
-        Resources.clone(currentReservation),
-        Resources.add(currentConsumption, currentReservation));
-  }
-
-  public synchronized Map<ContainerId, RMContainer> getLiveContainersMap() {
-    return this.liveContainers;
-  }
-
-  public synchronized Resource getResourceLimit() {
-    return this.resourceLimit;
-  }
-
-  public synchronized Map<Priority, Long> getLastScheduledContainer() {
-    return this.lastScheduledContainer;
-  }
-
-  public synchronized void transferStateFromPreviousAttempt(
-      SchedulerApplicationAttempt appAttempt) {
-    this.liveContainers = appAttempt.getLiveContainersMap();
-    // this.reReservations = appAttempt.reReservations;
-    this.currentConsumption = appAttempt.getCurrentConsumption();
-    this.resourceLimit = appAttempt.getResourceLimit();
-    // this.currentReservation = appAttempt.currentReservation;
-    // this.newlyAllocatedContainers = appAttempt.newlyAllocatedContainers;
-    // this.schedulingOpportunities = appAttempt.schedulingOpportunities;
-    this.lastScheduledContainer = appAttempt.getLastScheduledContainer();
-    this.appSchedulingInfo
-      .transferStateFromPreviousAppSchedulingInfo(appAttempt.appSchedulingInfo);
-  }
-}

+ 1 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java

@@ -19,13 +19,13 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
@@ -37,7 +37,6 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 
 /**
@@ -171,13 +170,4 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
   @LimitedPrivate("yarn")
   @Stable
   public List<ApplicationAttemptId> getAppsInQueue(String queueName);
-
-  /**
-   * Get the container for the given containerId.
-   * @param containerId
-   * @return the container for the given containerId.
-   */
-  @LimitedPrivate("yarn")
-  @Unstable
-  public RMContainer getRMContainer(ContainerId containerId);
 }

+ 10 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java

@@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -36,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 
@@ -155,32 +155,21 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
   
   /**
    * Submit a new application to the queue.
-   * @param applicationId the applicationId of the application being submitted
+   * @param application application being submitted
    * @param user user who submitted the application
    * @param queue queue to which the application is submitted
    */
-  public void submitApplication(ApplicationId applicationId, String user,
-      String queue) throws AccessControlException;
-
-  /**
-   * Submit an application attempt to the queue.
-   */
-  public void submitApplicationAttempt(FiCaSchedulerApp application,
-      String userName);
-
+  public void submitApplication(FiCaSchedulerApp application, String user, 
+      String queue) 
+  throws AccessControlException;
+  
   /**
    * An application submitted to this queue has finished.
-   * @param applicationId
-   * @param user user who submitted the application
+   * @param application
+   * @param queue application queue 
    */
-  public void finishApplication(ApplicationId applicationId, String user);
-
-  /**
-   * An application attempt submitted to this queue has finished.
-   */
-  public void finishApplicationAttempt(FiCaSchedulerApp application,
-      String queue);
-
+  public void finishApplication(FiCaSchedulerApp application, String queue);
+  
   /**
    * Assign containers to applications in the queue or it's children (if any).
    * @param clusterResource the resource of the cluster.

+ 99 - 170
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -36,7 +36,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -54,20 +53,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 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.RMAppAttemptRejectedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
@@ -77,10 +71,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeRepo
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@@ -96,7 +88,7 @@ import com.google.common.annotations.VisibleForTesting;
 @LimitedPrivate("yarn")
 @Evolving
 @SuppressWarnings("unchecked")
-public class CapacityScheduler extends AbstractYarnScheduler
+public class CapacityScheduler
   implements PreemptableResourceScheduler, CapacitySchedulerContext,
              Configurable {
 
@@ -178,6 +170,7 @@ public class CapacityScheduler extends AbstractYarnScheduler
 
   private CapacitySchedulerConfiguration conf;
   private Configuration yarnConf;
+  private RMContext rmContext;
 
   private Map<String, CSQueue> queues = new ConcurrentHashMap<String, CSQueue>();
 
@@ -191,6 +184,10 @@ public class CapacityScheduler extends AbstractYarnScheduler
   private Resource minimumAllocation;
   private Resource maximumAllocation;
 
+  @VisibleForTesting
+  protected Map<ApplicationAttemptId, FiCaSchedulerApp> applications = 
+      new ConcurrentHashMap<ApplicationAttemptId, FiCaSchedulerApp>();
+
   private boolean initialized = false;
 
   private ResourceCalculator calculator;
@@ -267,10 +264,9 @@ public class CapacityScheduler extends AbstractYarnScheduler
       this.maximumAllocation = this.conf.getMaximumAllocation();
       this.calculator = this.conf.getResourceCalculator();
       this.usePortForNodeName = this.conf.getUsePortForNodeName();
-      this.applications =
-          new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
-      this.rmContext = rmContext;
 
+      this.rmContext = rmContext;
+      
       initializeQueues(this.conf);
       
       initialized = true;
@@ -419,141 +415,105 @@ public class CapacityScheduler extends AbstractYarnScheduler
   synchronized CSQueue getQueue(String queueName) {
     return queues.get(queueName);
   }
+  
+  private synchronized void
+      addApplicationAttempt(ApplicationAttemptId applicationAttemptId,
+          String queueName, String user) {
 
-  private synchronized void addApplication(ApplicationId applicationId,
-      String queueName, String user) {
-    // santiy checks.
+    // Sanity checks
     CSQueue queue = getQueue(queueName);
     if (queue == null) {
-      String message = "Application " + applicationId + 
+      String message = "Application " + applicationAttemptId + 
       " submitted by user " + user + " to unknown queue: " + queueName;
-      this.rmContext.getDispatcher().getEventHandler()
-          .handle(new RMAppRejectedEvent(applicationId, message));
+      this.rmContext.getDispatcher().getEventHandler().handle(
+          new RMAppAttemptRejectedEvent(applicationAttemptId, message));
       return;
     }
     if (!(queue instanceof LeafQueue)) {
-      String message = "Application " + applicationId + 
+      String message = "Application " + applicationAttemptId + 
           " submitted by user " + user + " to non-leaf queue: " + queueName;
-      this.rmContext.getDispatcher().getEventHandler()
-          .handle(new RMAppRejectedEvent(applicationId, message));
+      this.rmContext.getDispatcher().getEventHandler().handle(
+          new RMAppAttemptRejectedEvent(applicationAttemptId, message));
       return;
     }
+
+    // TODO: Fix store
+    FiCaSchedulerApp SchedulerApp = 
+        new FiCaSchedulerApp(applicationAttemptId, user, queue, 
+            queue.getActiveUsersManager(), rmContext);
+
     // Submit to the queue
     try {
-      queue.submitApplication(applicationId, user, queueName);
+      queue.submitApplication(SchedulerApp, user, queueName);
     } catch (AccessControlException ace) {
-      LOG.info("Failed to submit application " + applicationId + " to queue "
-          + queueName + " from user " + user, ace);
-      this.rmContext.getDispatcher().getEventHandler()
-          .handle(new RMAppRejectedEvent(applicationId, ace.toString()));
+      LOG.info("Failed to submit application " + applicationAttemptId + 
+          " to queue " + queueName + " from user " + user, ace);
+      this.rmContext.getDispatcher().getEventHandler().handle(
+          new RMAppAttemptRejectedEvent(applicationAttemptId, 
+              ace.toString()));
       return;
     }
-    SchedulerApplication application =
-        new SchedulerApplication(queue, user);
-    applications.put(applicationId, application);
-    LOG.info("Accepted application " + applicationId + " from user: " + user
-        + ", in queue: " + queueName);
-    rmContext.getDispatcher().getEventHandler()
-        .handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
-  }
 
-  private synchronized void addApplicationAttempt(
-      ApplicationAttemptId applicationAttemptId,
-      boolean transferStateFromPreviousAttempt) {
-    SchedulerApplication application =
-        applications.get(applicationAttemptId.getApplicationId());
-    CSQueue queue = (CSQueue) application.getQueue();
-
-    FiCaSchedulerApp attempt =
-        new FiCaSchedulerApp(applicationAttemptId, application.getUser(),
-          queue, queue.getActiveUsersManager(), rmContext);
-    if (transferStateFromPreviousAttempt) {
-      attempt.transferStateFromPreviousAttempt(application
-        .getCurrentAppAttempt());
-    }
-    application.setCurrentAppAttempt(attempt);
+    applications.put(applicationAttemptId, SchedulerApp);
 
-    queue.submitApplicationAttempt(attempt, application.getUser());
-    LOG.info("Added Application Attempt " + applicationAttemptId
-        + " to scheduler from user " + application.getUser() + " in queue "
-        + queue.getQueueName());
-    rmContext.getDispatcher().getEventHandler() .handle(
-        new RMAppAttemptEvent(applicationAttemptId,
-          RMAppAttemptEventType.ATTEMPT_ADDED));
-  }
+    LOG.info("Application Submission: " + applicationAttemptId + 
+        ", user: " + user +
+        " queue: " + queue +
+        ", currently active: " + applications.size());
 
-  private synchronized void doneApplication(ApplicationId applicationId,
-      RMAppState finalState) {
-    SchedulerApplication application = applications.get(applicationId);
-    if (application == null){
-      // The AppRemovedSchedulerEvent maybe sent on recovery for completed apps,
-      // ignore it.
-      LOG.warn("Couldn't find application " + applicationId);
-      return;
-    }
-    CSQueue queue = (CSQueue) application.getQueue();
-    if (!(queue instanceof LeafQueue)) {
-      LOG.error("Cannot finish application " + "from non-leaf queue: "
-          + queue.getQueueName());
-    } else {
-      queue.finishApplication(applicationId, application.getUser());
-    }
-    application.stop(finalState);
-    applications.remove(applicationId);
+    rmContext.getDispatcher().getEventHandler().handle(
+        new RMAppAttemptEvent(applicationAttemptId,
+            RMAppAttemptEventType.APP_ACCEPTED));
   }
 
   private synchronized void doneApplicationAttempt(
       ApplicationAttemptId applicationAttemptId,
-      RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
-    LOG.info("Application Attempt " + applicationAttemptId + " is done." +
+      RMAppAttemptState rmAppAttemptFinalState) {
+    LOG.info("Application " + applicationAttemptId + " is done." +
     		" finalState=" + rmAppAttemptFinalState);
     
-    FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
-    SchedulerApplication application =
-        applications.get(applicationAttemptId.getApplicationId());
+    FiCaSchedulerApp application = getApplication(applicationAttemptId);
 
-    if (application == null || attempt == null) {
+    if (application == null) {
+      //      throw new IOException("Unknown application " + applicationId + 
+      //          " has completed!");
       LOG.info("Unknown application " + applicationAttemptId + " has completed!");
       return;
     }
-
-    // Release all the allocated, acquired, running containers
-    for (RMContainer rmContainer : attempt.getLiveContainers()) {
-      if (keepContainers
-          && rmContainer.getState().equals(RMContainerState.RUNNING)) {
-        // do not kill the running container in the case of work-preserving AM
-        // restart.
-        LOG.info("Skip killing " + rmContainer.getContainerId());
-        continue;
-      }
-      completedContainer(
-        rmContainer,
-        SchedulerUtils.createAbnormalContainerStatus(
-          rmContainer.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION),
-        RMContainerEventType.KILL);
+    
+    // Release all the running containers 
+    for (RMContainer rmContainer : application.getLiveContainers()) {
+      completedContainer(rmContainer, 
+          SchedulerUtils.createAbnormalContainerStatus(
+              rmContainer.getContainerId(), 
+              SchedulerUtils.COMPLETED_APPLICATION), 
+          RMContainerEventType.KILL);
     }
-
-    // Release all reserved containers
-    for (RMContainer rmContainer : attempt.getReservedContainers()) {
-      completedContainer(
-        rmContainer,
-        SchedulerUtils.createAbnormalContainerStatus(
-          rmContainer.getContainerId(), "Application Complete"),
-        RMContainerEventType.KILL);
+    
+     // Release all reserved containers
+    for (RMContainer rmContainer : application.getReservedContainers()) {
+      completedContainer(rmContainer, 
+          SchedulerUtils.createAbnormalContainerStatus(
+              rmContainer.getContainerId(), 
+              "Application Complete"), 
+          RMContainerEventType.KILL);
     }
-
+    
     // Clean up pending requests, metrics etc.
-    attempt.stop(rmAppAttemptFinalState);
-
+    application.stop(rmAppAttemptFinalState);
+    
     // Inform the queue
-    String queueName = attempt.getQueue().getQueueName();
+    String queueName = application.getQueue().getQueueName();
     CSQueue queue = queues.get(queueName);
     if (!(queue instanceof LeafQueue)) {
       LOG.error("Cannot finish application " + "from non-leaf queue: "
           + queueName);
     } else {
-      queue.finishApplicationAttempt(attempt, queue.getQueueName());
+      queue.finishApplication(application, queue.getQueueName());
     }
+    
+    // Remove from our data-structure
+    applications.remove(applicationAttemptId);
   }
 
   private static final Allocation EMPTY_ALLOCATION = 
@@ -565,7 +525,7 @@ public class CapacityScheduler extends AbstractYarnScheduler
       List<ResourceRequest> ask, List<ContainerId> release, 
       List<String> blacklistAdditions, List<String> blacklistRemovals) {
 
-    FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
+    FiCaSchedulerApp application = getApplication(applicationAttemptId);
     if (application == null) {
       LOG.info("Calling allocate on removed " +
           "or non existant application " + applicationAttemptId);
@@ -707,8 +667,8 @@ public class CapacityScheduler extends AbstractYarnScheduler
 
     RMContainer reservedContainer = node.getReservedContainer();
     if (reservedContainer != null) {
-      FiCaSchedulerApp reservedApplication =
-          getCurrentAttemptForContainer(reservedContainer.getContainerId());
+      FiCaSchedulerApp reservedApplication = 
+          getApplication(reservedContainer.getApplicationAttemptId());
       
       // Try to fulfill the reservation
       LOG.info("Trying to fulfill reservation for application " + 
@@ -745,11 +705,12 @@ public class CapacityScheduler extends AbstractYarnScheduler
 
   private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) {
     // Get the application for the finished container
-    FiCaSchedulerApp application = getCurrentAttemptForContainer(containerId);
+    ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId();
+    FiCaSchedulerApp application = getApplication(applicationAttemptId);
     if (application == null) {
-      LOG.info("Unknown application "
-          + containerId.getApplicationAttemptId().getApplicationId()
-          + " launched container " + containerId + " on node: " + node);
+      LOG.info("Unknown application: " + applicationAttemptId + 
+          " launched container " + containerId +
+          " on node: " + node);
       this.rmContext.getDispatcher().getEventHandler()
         .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
       return;
@@ -779,26 +740,12 @@ public class CapacityScheduler extends AbstractYarnScheduler
       nodeUpdate(nodeUpdatedEvent.getRMNode());
     }
     break;
-    case APP_ADDED:
-    {
-      AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
-      addApplication(appAddedEvent.getApplicationId(),
-        appAddedEvent.getQueue(), appAddedEvent.getUser());
-    }
-    break;
-    case APP_REMOVED:
-    {
-      AppRemovedSchedulerEvent appRemovedEvent = (AppRemovedSchedulerEvent)event;
-      doneApplication(appRemovedEvent.getApplicationID(),
-        appRemovedEvent.getFinalState());
-    }
-    break;
     case APP_ATTEMPT_ADDED:
     {
       AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
           (AppAttemptAddedSchedulerEvent) event;
       addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
-        appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
+        appAttemptAddedEvent.getQueue(), appAttemptAddedEvent.getUser());
     }
     break;
     case APP_ATTEMPT_REMOVED:
@@ -806,8 +753,7 @@ public class CapacityScheduler extends AbstractYarnScheduler
       AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
           (AppAttemptRemovedSchedulerEvent) event;
       doneApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(),
-        appAttemptRemovedEvent.getFinalAttemptState(),
-        appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
+        appAttemptRemovedEvent.getFinalAttemptState());
     }
     break;
     case CONTAINER_EXPIRED:
@@ -882,13 +828,13 @@ public class CapacityScheduler extends AbstractYarnScheduler
     Container container = rmContainer.getContainer();
     
     // Get the application for the finished container
-    FiCaSchedulerApp application =
-        getCurrentAttemptForContainer(container.getId());
-    ApplicationId appId =
-        container.getId().getApplicationAttemptId().getApplicationId();
+    ApplicationAttemptId applicationAttemptId =
+      container.getId().getApplicationAttemptId();
+    FiCaSchedulerApp application = getApplication(applicationAttemptId);
     if (application == null) {
-      LOG.info("Container " + container + " of" + " unknown application "
-          + appId + " completed with event " + event);
+      LOG.info("Container " + container + " of" +
+      		" unknown application " + applicationAttemptId + 
+          " completed with event " + event);
       return;
     }
     
@@ -900,33 +846,28 @@ public class CapacityScheduler extends AbstractYarnScheduler
     queue.completedContainer(clusterResource, application, node, 
         rmContainer, containerStatus, event, null);
 
-    LOG.info("Application attempt " + application.getApplicationAttemptId()
-        + " released container " + container.getId() + " on node: " + node
-        + " with event: " + event);
+    LOG.info("Application " + applicationAttemptId + 
+        " released container " + container.getId() +
+        " on node: " + node + 
+        " with event: " + event);
   }
 
   @Lock(Lock.NoLock.class)
-  FiCaSchedulerApp getApplicationAttempt(
-      ApplicationAttemptId applicationAttemptId) {
-    SchedulerApplication app =
-        applications.get(applicationAttemptId.getApplicationId());
-    if (app != null) {
-      return (FiCaSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
+  FiCaSchedulerApp getApplication(ApplicationAttemptId applicationAttemptId) {
+    return applications.get(applicationAttemptId);
   }
 
   @Override
   public SchedulerAppReport getSchedulerAppInfo(
       ApplicationAttemptId applicationAttemptId) {
-    FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
+    FiCaSchedulerApp app = getApplication(applicationAttemptId);
     return app == null ? null : new SchedulerAppReport(app);
   }
   
   @Override
   public ApplicationResourceUsageReport getAppResourceUsageReport(
       ApplicationAttemptId applicationAttemptId) {
-    FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
+    FiCaSchedulerApp app = getApplication(applicationAttemptId);
     return app == null ? null : app.getResourceUsageReport();
   }
   
@@ -935,22 +876,10 @@ public class CapacityScheduler extends AbstractYarnScheduler
     return nodes.get(nodeId);
   }
 
-  @Override
-  public RMContainer getRMContainer(ContainerId containerId) {
-    FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
-    return (attempt == null) ? null : attempt.getRMContainer(containerId);
-  }
-
-  @VisibleForTesting
-  public FiCaSchedulerApp getCurrentAttemptForContainer(
-      ContainerId containerId) {
-    SchedulerApplication app =
-        applications.get(containerId.getApplicationAttemptId()
-          .getApplicationId());
-    if (app != null) {
-      return (FiCaSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
+  private RMContainer getRMContainer(ContainerId containerId) {
+    FiCaSchedulerApp application = 
+        getApplication(containerId.getApplicationAttemptId());
+    return (application == null) ? null : application.getRMContainer(containerId);
   }
 
   @Override
@@ -983,7 +912,7 @@ public class CapacityScheduler extends AbstractYarnScheduler
       LOG.debug("PREEMPT_CONTAINER: application:" + aid.toString() +
           " container: " + cont.toString());
     }
-    FiCaSchedulerApp app = getApplicationAttempt(aid);
+    FiCaSchedulerApp app = applications.get(aid);
     if (app != null) {
       app.addPreemptContainer(cont.getContainerId());
     }

+ 35 - 41
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -38,7 +38,6 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -60,6 +59,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -99,7 +99,7 @@ public class LeafQueue implements CSQueue {
   private volatile int numContainers;
 
   Set<FiCaSchedulerApp> activeApplications;
-  Map<ApplicationAttemptId, FiCaSchedulerApp> applicationAttemptMap = 
+  Map<ApplicationAttemptId, FiCaSchedulerApp> applicationsMap = 
       new HashMap<ApplicationAttemptId, FiCaSchedulerApp>();
   
   Set<FiCaSchedulerApp> pendingApplications;
@@ -635,21 +635,7 @@ public class LeafQueue implements CSQueue {
   }
 
   @Override
-  public void submitApplicationAttempt(FiCaSchedulerApp application,
-      String userName) {
-    // Careful! Locking order is important!
-    synchronized (this) {
-      User user = getUser(userName);
-      // Add the attempt to our data-structures
-      addApplicationAttempt(application, user);
-    }
-
-    metrics.submitAppAttempt(userName);
-    getParent().submitApplicationAttempt(application, userName);
-  }
-
-  @Override
-  public void submitApplication(ApplicationId applicationId, String userName,
+  public void submitApplication(FiCaSchedulerApp application, String userName,
       String queue)  throws AccessControlException {
     // Careful! Locking order is important!
 
@@ -667,7 +653,8 @@ public class LeafQueue implements CSQueue {
       // Check if the queue is accepting jobs
       if (getState() != QueueState.RUNNING) {
         String msg = "Queue " + getQueuePath() +
-        " is STOPPED. Cannot accept submission of application: " + applicationId;
+        " is STOPPED. Cannot accept submission of application: " +
+        application.getApplicationId();
         LOG.info(msg);
         throw new AccessControlException(msg);
       }
@@ -676,7 +663,8 @@ public class LeafQueue implements CSQueue {
       if (getNumApplications() >= getMaxApplications()) {
         String msg = "Queue " + getQueuePath() + 
         " already has " + getNumApplications() + " applications," +
-        " cannot accept submission of application: " + applicationId;
+        " cannot accept submission of application: " + 
+        application.getApplicationId();
         LOG.info(msg);
         throw new AccessControlException(msg);
       }
@@ -687,22 +675,28 @@ public class LeafQueue implements CSQueue {
         String msg = "Queue " + getQueuePath() + 
         " already has " + user.getTotalApplications() + 
         " applications from user " + userName + 
-        " cannot accept submission of application: " + applicationId;
+        " cannot accept submission of application: " + 
+        application.getApplicationId();
         LOG.info(msg);
         throw new AccessControlException(msg);
       }
+
+      // Add the application to our data-structures
+      addApplication(application, user);
     }
 
+    int attemptId = application.getApplicationAttemptId().getAttemptId();
+    metrics.submitApp(userName, attemptId);
+
     // Inform the parent queue
     try {
-      getParent().submitApplication(applicationId, userName, queue);
+      getParent().submitApplication(application, userName, queue);
     } catch (AccessControlException ace) {
       LOG.info("Failed to submit application to parent-queue: " + 
           getParent().getQueuePath(), ace);
+      removeApplication(application, user);
       throw ace;
     }
-
-    metrics.submitApp(userName);
   }
 
   private synchronized void activateApplications() {
@@ -728,11 +722,11 @@ public class LeafQueue implements CSQueue {
     }
   }
   
-  private synchronized void addApplicationAttempt(FiCaSchedulerApp application, User user) {
+  private synchronized void addApplication(FiCaSchedulerApp application, User user) {
     // Accept 
     user.submitApplication();
     pendingApplications.add(application);
-    applicationAttemptMap.put(application.getApplicationAttemptId(), application);
+    applicationsMap.put(application.getApplicationAttemptId(), application);
 
     // Activate applications
     activateApplications();
@@ -748,28 +742,22 @@ public class LeafQueue implements CSQueue {
   }
 
   @Override
-  public void finishApplication(ApplicationId application, String user) {
-    // Inform the activeUsersManager
-    activeUsersManager.deactivateApplication(user, application);
-    // Inform the parent queue
-    getParent().finishApplication(application, user);
-  }
-
-  @Override
-  public void finishApplicationAttempt(FiCaSchedulerApp application, String queue) {
+  public void finishApplication(FiCaSchedulerApp application, String queue) {
     // Careful! Locking order is important!
     synchronized (this) {
-      removeApplicationAttempt(application, getUser(application.getUser()));
+      removeApplication(application, getUser(application.getUser()));
     }
-    getParent().finishApplicationAttempt(application, queue);
+
+    // Inform the parent queue
+    getParent().finishApplication(application, queue);
   }
 
-  public synchronized void removeApplicationAttempt(FiCaSchedulerApp application, User user) {
+  public synchronized void removeApplication(FiCaSchedulerApp application, User user) {
     boolean wasActive = activeApplications.remove(application);
     if (!wasActive) {
       pendingApplications.remove(application);
     }
-    applicationAttemptMap.remove(application.getApplicationAttemptId());
+    applicationsMap.remove(application.getApplicationAttemptId());
 
     user.finishApplication(wasActive);
     if (user.getTotalApplications() == 0) {
@@ -778,7 +766,13 @@ public class LeafQueue implements CSQueue {
 
     // Check if we can activate more applications
     activateApplications();
-
+    
+    // Inform the activeUsersManager
+    synchronized (application) {
+      activeUsersManager.deactivateApplication(
+          application.getUser(), application.getApplicationId());
+    }
+    
     LOG.info("Application removed -" +
         " appId: " + application.getApplicationId() + 
         " user: " + application.getUser() + 
@@ -789,10 +783,10 @@ public class LeafQueue implements CSQueue {
         " #queue-active-applications: " + getNumActiveApplications()
         );
   }
-
+  
   private synchronized FiCaSchedulerApp getApplication(
       ApplicationAttemptId applicationAttemptId) {
-    return applicationAttemptMap.get(applicationAttemptId);
+    return applicationsMap.get(applicationAttemptId);
   }
 
   private static final CSAssignment NULL_ASSIGNMENT =

+ 14 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -52,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -442,7 +442,7 @@ public class ParentQueue implements CSQueue {
   }
 
   @Override
-  public void submitApplication(ApplicationId applicationId, String user,
+  public void submitApplication(FiCaSchedulerApp application, String user,
       String queue) throws AccessControlException {
     
     synchronized (this) {
@@ -455,70 +455,57 @@ public class ParentQueue implements CSQueue {
       if (state != QueueState.RUNNING) {
         throw new AccessControlException("Queue " + getQueuePath() +
             " is STOPPED. Cannot accept submission of application: " +
-            applicationId);
+            application.getApplicationId());
       }
 
-      addApplication(applicationId, user);
+      addApplication(application, user);
     }
     
     // Inform the parent queue
     if (parent != null) {
       try {
-        parent.submitApplication(applicationId, user, queue);
+        parent.submitApplication(application, user, queue);
       } catch (AccessControlException ace) {
         LOG.info("Failed to submit application to parent-queue: " + 
             parent.getQueuePath(), ace);
-        removeApplication(applicationId, user);
+        removeApplication(application, user);
         throw ace;
       }
     }
   }
 
-
-  @Override
-  public void submitApplicationAttempt(FiCaSchedulerApp application,
-      String userName) {
-    // submit attempt logic.
-  }
-
-  @Override
-  public void finishApplicationAttempt(FiCaSchedulerApp application,
-      String queue) {
-    // finish attempt logic.
-  }
-
-  private synchronized void addApplication(ApplicationId applicationId,
+  private synchronized void addApplication(FiCaSchedulerApp application, 
       String user) {
-
+  
     ++numApplications;
 
     LOG.info("Application added -" +
-        " appId: " + applicationId + 
+        " appId: " + application.getApplicationId() + 
         " user: " + user + 
         " leaf-queue of parent: " + getQueueName() + 
         " #applications: " + getNumApplications());
   }
   
   @Override
-  public void finishApplication(ApplicationId application, String user) {
+  public void finishApplication(FiCaSchedulerApp application, String queue) {
     
     synchronized (this) {
-      removeApplication(application, user);
+      removeApplication(application, application.getUser());
     }
     
     // Inform the parent queue
     if (parent != null) {
-      parent.finishApplication(application, user);
+      parent.finishApplication(application, queue);
     }
   }
 
-  public synchronized void removeApplication(ApplicationId applicationId, 
+  public synchronized void removeApplication(FiCaSchedulerApp application, 
       String user) {
     
     --numApplications;
 
     LOG.info("Application removed -" +
-        " appId: " + applicationId + 
+        " appId: " + application.getApplicationId() + 
         " user: " + user + 
         " leaf-queue of parent: " + getQueueName() + 
         " #applications: " + getNumApplications());

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java

@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 
@@ -57,7 +57,7 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
  */
 @Private
 @Unstable
-public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
+public class FiCaSchedulerApp extends SchedulerApplication {
 
   private static final Log LOG = LogFactory.getLog(FiCaSchedulerApp.class);
 

+ 4 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -206,7 +206,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
   }
 
   public synchronized void reserveResource(
-      SchedulerApplicationAttempt application, Priority priority, 
+      SchedulerApplication application, Priority priority, 
       RMContainer reservedContainer) {
     // Check if it's already reserved
     if (this.reservedContainer != null) {
@@ -219,8 +219,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
             " on node " + this.reservedContainer.getReservedNode());
       }
       
-      // Cannot reserve more than one application attempt on a given node!
-      // Reservation is still against attempt.
+      // Cannot reserve more than one application on a given node!
       if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals(
           reservedContainer.getContainer().getId().getApplicationAttemptId())) {
         throw new IllegalStateException("Trying to reserve" +
@@ -242,7 +241,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
   }
 
   public synchronized void unreserveResource(
-      SchedulerApplicationAttempt application) {
+      SchedulerApplication application) {
     
     // adding NP checks as this can now be called for preemption
     if (reservedContainer != null

+ 0 - 49
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAddedSchedulerEvent.java

@@ -1,49 +0,0 @@
-/**
- * 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.scheduler.event;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-
-public class AppAddedSchedulerEvent extends SchedulerEvent {
-
-  private final ApplicationId applicationId;
-  private final String queue;
-  private final String user;
-
-  public AppAddedSchedulerEvent(
-      ApplicationId applicationId, String queue, String user) {
-    super(SchedulerEventType.APP_ADDED);
-    this.applicationId = applicationId;
-    this.queue = queue;
-    this.user = user;
-  }
-
-  public ApplicationId getApplicationId() {
-    return applicationId;
-  }
-
-  public String getQueue() {
-    return queue;
-  }
-
-  public String getUser() {
-    return user;
-  }
-
-}

+ 12 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptAddedSchedulerEvent.java

@@ -23,21 +23,27 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 public class AppAttemptAddedSchedulerEvent extends SchedulerEvent {
 
   private final ApplicationAttemptId applicationAttemptId;
-  private final boolean transferStateFromPreviousAttempt;
+  private final String queue;
+  private final String user;
 
   public AppAttemptAddedSchedulerEvent(
-      ApplicationAttemptId applicationAttemptId,
-      boolean transferStateFromPreviousAttempt) {
+      ApplicationAttemptId applicationAttemptId, String queue, String user) {
     super(SchedulerEventType.APP_ATTEMPT_ADDED);
     this.applicationAttemptId = applicationAttemptId;
-    this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt;
+    this.queue = queue;
+    this.user = user;
   }
 
   public ApplicationAttemptId getApplicationAttemptId() {
     return applicationAttemptId;
   }
 
-  public boolean getTransferStateFromPreviousAttempt() {
-    return transferStateFromPreviousAttempt;
+  public String getQueue() {
+    return queue;
   }
+
+  public String getUser() {
+    return user;
+  }
+
 }

+ 1 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptRemovedSchedulerEvent.java

@@ -25,15 +25,13 @@ public class AppAttemptRemovedSchedulerEvent extends SchedulerEvent {
 
   private final ApplicationAttemptId applicationAttemptId;
   private final RMAppAttemptState finalAttemptState;
-  private final boolean keepContainersAcrossAppAttempts;
 
   public AppAttemptRemovedSchedulerEvent(
       ApplicationAttemptId applicationAttemptId,
-      RMAppAttemptState finalAttemptState, boolean keepContainers) {
+      RMAppAttemptState finalAttemptState) {
     super(SchedulerEventType.APP_ATTEMPT_REMOVED);
     this.applicationAttemptId = applicationAttemptId;
     this.finalAttemptState = finalAttemptState;
-    this.keepContainersAcrossAppAttempts = keepContainers;
   }
 
   public ApplicationAttemptId getApplicationAttemptID() {
@@ -43,8 +41,4 @@ public class AppAttemptRemovedSchedulerEvent extends SchedulerEvent {
   public RMAppAttemptState getFinalAttemptState() {
     return this.finalAttemptState;
   }
-
-  public boolean getKeepContainersAcrossAppAttempts() {
-    return this.keepContainersAcrossAppAttempts;
-  }
 }

+ 0 - 43
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppRemovedSchedulerEvent.java

@@ -1,43 +0,0 @@
-/**
- * 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.scheduler.event;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-
-public class AppRemovedSchedulerEvent extends SchedulerEvent {
-
-  private final ApplicationId applicationId;
-  private final RMAppState finalState;
-
-  public AppRemovedSchedulerEvent(ApplicationId applicationId,
-      RMAppState finalState) {
-    super(SchedulerEventType.APP_REMOVED);
-    this.applicationId = applicationId;
-    this.finalState = finalState;
-  }
-
-  public ApplicationId getApplicationID() {
-    return this.applicationId;
-  }
-
-  public RMAppState getFinalState() {
-    return this.finalState;
-  }
-}

+ 1 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java

@@ -24,11 +24,7 @@ public enum SchedulerEventType {
   NODE_ADDED,
   NODE_REMOVED,
   NODE_UPDATE,
-
-  // Source: RMApp
-  APP_ADDED,
-  APP_REMOVED,
-
+  
   // Source: RMAppAttempt
   APP_ATTEMPT_ADDED,
   APP_ATTEMPT_REMOVED,

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 
 @Private
 @Unstable

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFini
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
@@ -52,7 +52,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
  */
 @Private
 @Unstable
-public class FSSchedulerApp extends SchedulerApplicationAttempt {
+public class FSSchedulerApp extends SchedulerApplication {
 
   private static final Log LOG = LogFactory.getLog(FSSchedulerApp.class);
 

+ 80 - 155
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -38,7 +38,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -59,19 +58,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 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;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 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.RMAppAttemptRejectedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
@@ -80,10 +75,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppRepor
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@@ -121,10 +114,10 @@ import com.google.common.annotations.VisibleForTesting;
 @LimitedPrivate("yarn")
 @Unstable
 @SuppressWarnings("unchecked")
-public class FairScheduler extends AbstractYarnScheduler implements
-    ResourceScheduler {
+public class FairScheduler implements ResourceScheduler {
   private boolean initialized;
   private FairSchedulerConfiguration conf;
+  private RMContext rmContext;
   private Resource minimumAllocation;
   private Resource maximumAllocation;
   private Resource incrAllocation;
@@ -158,6 +151,12 @@ public class FairScheduler extends AbstractYarnScheduler implements
   // Time we last ran preemptTasksIfNecessary
   private long lastPreemptCheckTime;
 
+  // This stores per-application scheduling information, indexed by
+  // attempt ID's for fast lookup.
+  @VisibleForTesting
+  protected Map<ApplicationAttemptId, FSSchedulerApp> applications = 
+      new ConcurrentHashMap<ApplicationAttemptId, FSSchedulerApp>();
+
   // Nodes in the cluster, indexed by NodeId
   private Map<NodeId, FSSchedulerNode> nodes = 
       new ConcurrentHashMap<NodeId, FSSchedulerNode>();
@@ -252,21 +251,10 @@ public class FairScheduler extends AbstractYarnScheduler implements
     return queueMgr;
   }
 
-  @Override
-  public RMContainer getRMContainer(ContainerId containerId) {
-    FSSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
-    return (attempt == null) ? null : attempt.getRMContainer(containerId);
-  }
-
-  private FSSchedulerApp getCurrentAttemptForContainer(
-      ContainerId containerId) {
-    SchedulerApplication app =
-        applications.get(containerId.getApplicationAttemptId()
-          .getApplicationId());
-    if (app != null) {
-      return (FSSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
+  private RMContainer getRMContainer(ContainerId containerId) {
+    FSSchedulerApp application = 
+        applications.get(containerId.getApplicationAttemptId());
+    return (application == null) ? null : application.getRMContainer(containerId);
   }
 
   /**
@@ -603,87 +591,64 @@ public class FairScheduler extends AbstractYarnScheduler implements
    * user. This will accept a new app even if the user or queue is above
    * configured limits, but the app will not be marked as runnable.
    */
-  protected synchronized void addApplication(ApplicationId applicationId,
-      String queueName, String user) {
+  protected synchronized void addApplicationAttempt(
+      ApplicationAttemptId applicationAttemptId, String queueName, String user) {
     if (queueName == null || queueName.isEmpty()) {
-      String message = "Reject application " + applicationId +
+      String message = "Reject application " + applicationAttemptId +
               " submitted by user " + user + " with an empty queue name.";
       LOG.info(message);
-      rmContext.getDispatcher().getEventHandler()
-          .handle(new RMAppRejectedEvent(applicationId, message));
+      rmContext.getDispatcher().getEventHandler().handle(
+              new RMAppAttemptRejectedEvent(applicationAttemptId, message));
       return;
     }
 
-    RMApp rmApp = rmContext.getRMApps().get(applicationId);
+    RMApp rmApp = rmContext.getRMApps().get(
+        applicationAttemptId.getApplicationId());
     FSLeafQueue queue = assignToQueue(rmApp, queueName, user);
     if (queue == null) {
       rmContext.getDispatcher().getEventHandler().handle(
-          new RMAppRejectedEvent(applicationId,
+          new RMAppAttemptRejectedEvent(applicationAttemptId,
               "Application rejected by queue placement policy"));
       return;
     }
 
+    FSSchedulerApp schedulerApp =
+        new FSSchedulerApp(applicationAttemptId, user,
+            queue, new ActiveUsersManager(getRootQueueMetrics()),
+            rmContext);
+
     // Enforce ACLs
     UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(user);
 
     if (!queue.hasAccess(QueueACL.SUBMIT_APPLICATIONS, userUgi)
         && !queue.hasAccess(QueueACL.ADMINISTER_QUEUE, userUgi)) {
       String msg = "User " + userUgi.getUserName() +
-              " cannot submit applications to queue " + queue.getName();
+    	        " cannot submit applications to queue " + queue.getName();
       LOG.info(msg);
-      rmContext.getDispatcher().getEventHandler()
-          .handle(new RMAppRejectedEvent(applicationId, msg));
+      rmContext.getDispatcher().getEventHandler().handle(
+    	        new RMAppAttemptRejectedEvent(applicationAttemptId, msg));
       return;
     }
-  
-    SchedulerApplication application =
-        new SchedulerApplication(queue, user);
-    applications.put(applicationId, application);
-    queue.getMetrics().submitApp(user);
-
-    LOG.info("Accepted application " + applicationId + " from user: " + user
-        + ", in queue: " + queueName + ", currently num of applications: "
-        + applications.size());
-    rmContext.getDispatcher().getEventHandler()
-        .handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
-  }
-
-  /**
-   * Add a new application attempt to the scheduler.
-   */
-  protected synchronized void addApplicationAttempt(
-      ApplicationAttemptId applicationAttemptId,
-      boolean transferStateFromPreviousAttempt) {
-    SchedulerApplication application =
-        applications.get(applicationAttemptId.getApplicationId());
-    String user = application.getUser();
-    FSLeafQueue queue = (FSLeafQueue) application.getQueue();
-
-    FSSchedulerApp attempt =
-        new FSSchedulerApp(applicationAttemptId, user,
-            queue, new ActiveUsersManager(getRootQueueMetrics()),
-            rmContext);
-    if (transferStateFromPreviousAttempt) {
-      attempt.transferStateFromPreviousAttempt(application
-        .getCurrentAppAttempt());
-    }
-    application.setCurrentAppAttempt(attempt);
 
     boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user);
-    queue.addApp(attempt, runnable);
+    queue.addApp(schedulerApp, runnable);
     if (runnable) {
-      maxRunningEnforcer.trackRunnableApp(attempt);
+      maxRunningEnforcer.trackRunnableApp(schedulerApp);
     } else {
-      maxRunningEnforcer.trackNonRunnableApp(attempt);
+      maxRunningEnforcer.trackNonRunnableApp(schedulerApp);
     }
     
-    queue.getMetrics().submitAppAttempt(user);
+    queue.getMetrics().submitApp(user, applicationAttemptId.getAttemptId());
+
+    applications.put(applicationAttemptId, schedulerApp);
+
+    LOG.info("Application Submission: " + applicationAttemptId +
+        ", user: "+ user +
+        ", currently active: " + applications.size());
 
-    LOG.info("Added Application Attempt " + applicationAttemptId
-        + " to scheduler from user: " + user);
     rmContext.getDispatcher().getEventHandler().handle(
         new RMAppAttemptEvent(applicationAttemptId,
-            RMAppAttemptEventType.ATTEMPT_ADDED));
+            RMAppAttemptEventType.APP_ACCEPTED));
   }
   
   @VisibleForTesting
@@ -709,40 +674,21 @@ public class FairScheduler extends AbstractYarnScheduler implements
     return queue;
   }
 
-  private synchronized void removeApplication(ApplicationId applicationId,
-      RMAppState finalState) {
-    SchedulerApplication application = applications.get(applicationId);
-    if (application == null){
-      LOG.warn("Couldn't find application " + applicationId);
-      return;
-    }
-    application.stop(finalState);
-    applications.remove(applicationId);
-  }
-
   private synchronized void removeApplicationAttempt(
       ApplicationAttemptId applicationAttemptId,
-      RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
+      RMAppAttemptState rmAppAttemptFinalState) {
     LOG.info("Application " + applicationAttemptId + " is done." +
         " finalState=" + rmAppAttemptFinalState);
-    SchedulerApplication application =
-        applications.get(applicationAttemptId.getApplicationId());
-    FSSchedulerApp attempt = getSchedulerApp(applicationAttemptId);
 
-    if (attempt == null || application == null) {
+    FSSchedulerApp application = applications.get(applicationAttemptId);
+
+    if (application == null) {
       LOG.info("Unknown application " + applicationAttemptId + " has completed!");
       return;
     }
 
     // Release all the running containers
-    for (RMContainer rmContainer : attempt.getLiveContainers()) {
-      if (keepContainers
-          && rmContainer.getState().equals(RMContainerState.RUNNING)) {
-        // do not kill the running container in the case of work-preserving AM
-        // restart.
-        LOG.info("Skip killing " + rmContainer.getContainerId());
-        continue;
-      }
+    for (RMContainer rmContainer : application.getLiveContainers()) {
       completedContainer(rmContainer,
           SchedulerUtils.createAbnormalContainerStatus(
               rmContainer.getContainerId(),
@@ -751,26 +697,30 @@ public class FairScheduler extends AbstractYarnScheduler implements
     }
 
     // Release all reserved containers
-    for (RMContainer rmContainer : attempt.getReservedContainers()) {
+    for (RMContainer rmContainer : application.getReservedContainers()) {
       completedContainer(rmContainer,
           SchedulerUtils.createAbnormalContainerStatus(
               rmContainer.getContainerId(),
               "Application Complete"),
-              RMContainerEventType.KILL);
+          RMContainerEventType.KILL);
     }
+
     // Clean up pending requests, metrics etc.
-    attempt.stop(rmAppAttemptFinalState);
+    application.stop(rmAppAttemptFinalState);
 
     // Inform the queue
-    FSLeafQueue queue = queueMgr.getLeafQueue(attempt.getQueue()
+    FSLeafQueue queue = queueMgr.getLeafQueue(application.getQueue()
         .getQueueName(), false);
-    boolean wasRunnable = queue.removeApp(attempt);
+    boolean wasRunnable = queue.removeApp(application);
 
     if (wasRunnable) {
-      maxRunningEnforcer.updateRunnabilityOnAppRemoval(attempt);
+      maxRunningEnforcer.updateRunnabilityOnAppRemoval(application);
     } else {
-      maxRunningEnforcer.untrackNonRunnableApp(attempt);
+      maxRunningEnforcer.untrackNonRunnableApp(application);
     }
+    
+    // Remove from our data-structure
+    applications.remove(applicationAttemptId);
   }
 
   /**
@@ -786,13 +736,11 @@ public class FairScheduler extends AbstractYarnScheduler implements
     Container container = rmContainer.getContainer();
 
     // Get the application for the finished container
-    FSSchedulerApp application =
-        getCurrentAttemptForContainer(container.getId());
-    ApplicationId appId =
-        container.getId().getApplicationAttemptId().getApplicationId();
+    ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId();
+    FSSchedulerApp application = applications.get(applicationAttemptId);
     if (application == null) {
       LOG.info("Container " + container + " of" +
-          " unknown application attempt " + appId +
+          " unknown application " + applicationAttemptId +
           " completed with event " + event);
       return;
     }
@@ -809,9 +757,10 @@ public class FairScheduler extends AbstractYarnScheduler implements
       updateRootQueueMetrics();
     }
 
-    LOG.info("Application attempt " + application.getApplicationAttemptId()
-        + " released container " + container.getId() + " on node: " + node
-        + " with event: " + event);
+    LOG.info("Application " + applicationAttemptId +
+        " released container " + container.getId() +
+        " on node: " + node +
+        " with event: " + event);
   }
 
   private synchronized void addNode(RMNode node) {
@@ -862,7 +811,7 @@ public class FairScheduler extends AbstractYarnScheduler implements
       List<ResourceRequest> ask, List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
 
     // Make sure this application exists
-    FSSchedulerApp application = getSchedulerApp(appAttemptId);
+    FSSchedulerApp application = applications.get(appAttemptId);
     if (application == null) {
       LOG.info("Calling allocate on removed " +
           "or non existant application " + appAttemptId);
@@ -932,11 +881,12 @@ public class FairScheduler extends AbstractYarnScheduler implements
    */
   private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) {
     // Get the application for the finished container
-    FSSchedulerApp application = getCurrentAttemptForContainer(containerId);
+    ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId();
+    FSSchedulerApp application = applications.get(applicationAttemptId);
     if (application == null) {
-      LOG.info("Unknown application "
-          + containerId.getApplicationAttemptId().getApplicationId()
-          + " launched container " + containerId + " on node: " + node);
+      LOG.info("Unknown application: " + applicationAttemptId +
+          " launched container " + containerId +
+          " on node: " + node);
       return;
     }
 
@@ -1075,34 +1025,28 @@ public class FairScheduler extends AbstractYarnScheduler implements
   }
   
   public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) {
-    SchedulerApplication app =
-        applications.get(appAttemptId.getApplicationId());
-    if (app != null) {
-      return (FSSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
+    return applications.get(appAttemptId);
   }
   
   @Override
   public SchedulerAppReport getSchedulerAppInfo(
       ApplicationAttemptId appAttemptId) {
-    FSSchedulerApp attempt = getSchedulerApp(appAttemptId);
-    if (attempt == null) {
+    if (!applications.containsKey(appAttemptId)) {
       LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
       return null;
     }
-    return new SchedulerAppReport(attempt);
+    return new SchedulerAppReport(applications.get(appAttemptId));
   }
   
   @Override
   public ApplicationResourceUsageReport getAppResourceUsageReport(
       ApplicationAttemptId appAttemptId) {
-    FSSchedulerApp attempt = getSchedulerApp(appAttemptId);
-    if (attempt == null) {
+    FSSchedulerApp app = applications.get(appAttemptId);
+    if (app == null) {
       LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
       return null;
     }
-    return attempt.getResourceUsageReport();
+    return app.getResourceUsageReport();
   }
   
   /**
@@ -1146,30 +1090,15 @@ public class FairScheduler extends AbstractYarnScheduler implements
       NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)event;
       nodeUpdate(nodeUpdatedEvent.getRMNode());
       break;
-    case APP_ADDED:
-      if (!(event instanceof AppAddedSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
-      addApplication(appAddedEvent.getApplicationId(),
-        appAddedEvent.getQueue(), appAddedEvent.getUser());
-      break;
-    case APP_REMOVED:
-      if (!(event instanceof AppRemovedSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      AppRemovedSchedulerEvent appRemovedEvent = (AppRemovedSchedulerEvent)event;
-      removeApplication(appRemovedEvent.getApplicationID(),
-        appRemovedEvent.getFinalState());
-      break;
     case APP_ATTEMPT_ADDED:
       if (!(event instanceof AppAttemptAddedSchedulerEvent)) {
         throw new RuntimeException("Unexpected event type: " + event);
       }
       AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
           (AppAttemptAddedSchedulerEvent) event;
+      String queue = appAttemptAddedEvent.getQueue();
       addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
-        appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
+        queue, appAttemptAddedEvent.getUser());
       break;
     case APP_ATTEMPT_REMOVED:
       if (!(event instanceof AppAttemptRemovedSchedulerEvent)) {
@@ -1177,10 +1106,8 @@ public class FairScheduler extends AbstractYarnScheduler implements
       }
       AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
           (AppAttemptRemovedSchedulerEvent) event;
-      removeApplicationAttempt(
-          appAttemptRemovedEvent.getApplicationAttemptID(),
-          appAttemptRemovedEvent.getFinalAttemptState(),
-          appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
+      removeApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(),
+        appAttemptRemovedEvent.getFinalAttemptState());
       break;
     case CONTAINER_EXPIRED:
       if (!(event instanceof ContainerExpiredSchedulerEvent)) {
@@ -1231,9 +1158,6 @@ public class FairScheduler extends AbstractYarnScheduler implements
       
       rootMetrics = FSQueueMetrics.forQueue("root", null, true, conf);
       this.rmContext = rmContext;
-      // This stores per-application scheduling information
-      this.applications =
-          new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
       this.eventLog = new FairSchedulerEventLog();
       eventLog.init(this.conf);
 
@@ -1356,4 +1280,5 @@ public class FairScheduler extends AbstractYarnScheduler implements
     queue.collectSchedulerApplications(apps);
     return apps;
   }
+
 }

+ 67 - 144
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -59,19 +58,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 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.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
@@ -80,15 +74,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@@ -105,8 +96,7 @@ import com.google.common.annotations.VisibleForTesting;
 @LimitedPrivate("yarn")
 @Evolving
 @SuppressWarnings("unchecked")
-public class FifoScheduler extends AbstractYarnScheduler implements
-    ResourceScheduler, Configurable {
+public class FifoScheduler implements ResourceScheduler, Configurable {
 
   private static final Log LOG = LogFactory.getLog(FifoScheduler.class);
 
@@ -117,6 +107,7 @@ public class FifoScheduler extends AbstractYarnScheduler implements
 
   private final static Container[] EMPTY_CONTAINER_ARRAY = new Container[] {};
   private final static List<Container> EMPTY_CONTAINER_LIST = Arrays.asList(EMPTY_CONTAINER_ARRAY);
+  private RMContext rmContext;
 
   protected Map<NodeId, FiCaSchedulerNode> nodes = new ConcurrentHashMap<NodeId, FiCaSchedulerNode>();
 
@@ -125,6 +116,11 @@ public class FifoScheduler extends AbstractYarnScheduler implements
   private Resource maximumAllocation;
   private boolean usePortForNodeName;
 
+  // Use ConcurrentSkipListMap because applications need to be ordered
+  @VisibleForTesting
+  protected Map<ApplicationAttemptId, FiCaSchedulerApp> applications
+      = new ConcurrentSkipListMap<ApplicationAttemptId, FiCaSchedulerApp>();
+  
   private ActiveUsersManager activeUsersManager;
 
   private static final String DEFAULT_QUEUE_NAME = "default";
@@ -239,9 +235,6 @@ public class FifoScheduler extends AbstractYarnScheduler implements
     if (!this.initialized) {
       validateConf(conf);
       this.rmContext = rmContext;
-      //Use ConcurrentSkipListMap because applications need to be ordered
-      this.applications =
-          new ConcurrentSkipListMap<ApplicationId, SchedulerApplication>();
       this.minimumAllocation = 
         Resources.createResource(conf.getInt(
             YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
@@ -266,7 +259,7 @@ public class FifoScheduler extends AbstractYarnScheduler implements
   public Allocation allocate(
       ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
       List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
-    FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
+    FiCaSchedulerApp application = getApplication(applicationAttemptId);
     if (application == null) {
       LOG.error("Calling allocate on removed " +
           "or non existant application " + applicationAttemptId);
@@ -332,114 +325,74 @@ public class FifoScheduler extends AbstractYarnScheduler implements
   }
 
   @VisibleForTesting
-  FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId applicationAttemptId) {
-    SchedulerApplication app =
-        applications.get(applicationAttemptId.getApplicationId());
-    if (app != null) {
-      return (FiCaSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
+  FiCaSchedulerApp getApplication(
+      ApplicationAttemptId applicationAttemptId) {
+    return applications.get(applicationAttemptId);
   }
 
   @Override
   public SchedulerAppReport getSchedulerAppInfo(
       ApplicationAttemptId applicationAttemptId) {
-    FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
+    FiCaSchedulerApp app = getApplication(applicationAttemptId);
     return app == null ? null : new SchedulerAppReport(app);
   }
   
   @Override
   public ApplicationResourceUsageReport getAppResourceUsageReport(
       ApplicationAttemptId applicationAttemptId) {
-    FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
+    FiCaSchedulerApp app = getApplication(applicationAttemptId);
     return app == null ? null : app.getResourceUsageReport();
   }
   
   private FiCaSchedulerNode getNode(NodeId nodeId) {
     return nodes.get(nodeId);
   }
-
-  private synchronized void addApplication(ApplicationId applicationId,
-      String queue, String user) {
-    SchedulerApplication application =
-        new SchedulerApplication(DEFAULT_QUEUE, user);
-    applications.put(applicationId, application);
-    metrics.submitApp(user);
-    LOG.info("Accepted application " + applicationId + " from user: " + user
-        + ", currently num of applications: " + applications.size());
-    rmContext.getDispatcher().getEventHandler()
-        .handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
-  }
-
-  private synchronized void
-      addApplicationAttempt(ApplicationAttemptId appAttemptId,
-          boolean transferStateFromPreviousAttempt) {
-    SchedulerApplication application =
-        applications.get(appAttemptId.getApplicationId());
-    String user = application.getUser();
+  
+  private synchronized void addApplicationAttempt(ApplicationAttemptId appAttemptId,
+      String user) {
     // TODO: Fix store
-    FiCaSchedulerApp schedulerApp =
-        new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE,
-          activeUsersManager, this.rmContext);
-
-    if (transferStateFromPreviousAttempt) {
-      schedulerApp.transferStateFromPreviousAttempt(application
-        .getCurrentAppAttempt());
-    }
-    application.setCurrentAppAttempt(schedulerApp);
-
-    metrics.submitAppAttempt(user);
-    LOG.info("Added Application Attempt " + appAttemptId
-        + " to scheduler from user " + application.getUser());
+    FiCaSchedulerApp schedulerApp = 
+        new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE, activeUsersManager,
+            this.rmContext);
+    applications.put(appAttemptId, schedulerApp);
+    metrics.submitApp(user, appAttemptId.getAttemptId());
+    LOG.info("Application Submission: " + appAttemptId.getApplicationId() + 
+        " from " + user + ", currently active: " + applications.size());
     rmContext.getDispatcher().getEventHandler().handle(
         new RMAppAttemptEvent(appAttemptId,
-            RMAppAttemptEventType.ATTEMPT_ADDED));
-  }
-
-  private synchronized void doneApplication(ApplicationId applicationId,
-      RMAppState finalState) {
-    SchedulerApplication application = applications.get(applicationId);
-    if (application == null){
-      LOG.warn("Couldn't find application " + applicationId);
-      return;
-    }
-
-    // Inform the activeUsersManager
-    activeUsersManager.deactivateApplication(application.getUser(),
-      applicationId);
-    application.stop(finalState);
-    applications.remove(applicationId);
+            RMAppAttemptEventType.APP_ACCEPTED));
   }
 
   private synchronized void doneApplicationAttempt(
       ApplicationAttemptId applicationAttemptId,
-      RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers)
+      RMAppAttemptState rmAppAttemptFinalState)
       throws IOException {
-    FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
-    SchedulerApplication application =
-        applications.get(applicationAttemptId.getApplicationId());
-    if (application == null || attempt == null) {
+    FiCaSchedulerApp application = getApplication(applicationAttemptId);
+    if (application == null) {
       throw new IOException("Unknown application " + applicationAttemptId + 
       " has completed!");
     }
 
     // Kill all 'live' containers
-    for (RMContainer container : attempt.getLiveContainers()) {
-      if (keepContainers
-          && container.getState().equals(RMContainerState.RUNNING)) {
-        // do not kill the running container in the case of work-preserving AM
-        // restart.
-        LOG.info("Skip killing " + container.getContainerId());
-        continue;
-      }
-      containerCompleted(container,
-        SchedulerUtils.createAbnormalContainerStatus(
-          container.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION),
-        RMContainerEventType.KILL);
+    for (RMContainer container : application.getLiveContainers()) {
+      containerCompleted(container, 
+          SchedulerUtils.createAbnormalContainerStatus(
+              container.getContainerId(), 
+              SchedulerUtils.COMPLETED_APPLICATION),
+          RMContainerEventType.KILL);
+    }
+
+    // Inform the activeUsersManager
+    synchronized (application) {
+      activeUsersManager.deactivateApplication(
+          application.getUser(), application.getApplicationId());
     }
 
     // Clean up pending requests, metrics etc.
-    attempt.stop(rmAppAttemptFinalState);
+    application.stop(rmAppAttemptFinalState);
+
+    // Remove the application
+    applications.remove(applicationAttemptId);
   }
   
   /**
@@ -453,10 +406,9 @@ public class FifoScheduler extends AbstractYarnScheduler implements
         " #applications=" + applications.size());
 
     // Try to assign containers to applications in fifo order
-    for (Map.Entry<ApplicationId, SchedulerApplication> e : applications
+    for (Map.Entry<ApplicationAttemptId, FiCaSchedulerApp> e : applications
         .entrySet()) {
-      FiCaSchedulerApp application =
-          (FiCaSchedulerApp) e.getValue().getCurrentAppAttempt();
+      FiCaSchedulerApp application = e.getValue();
       LOG.debug("pre-assignContainers");
       application.showRequests();
       synchronized (application) {
@@ -493,10 +445,8 @@ public class FifoScheduler extends AbstractYarnScheduler implements
 
     // Update the applications' headroom to correctly take into
     // account the containers assigned in this update.
-    for (SchedulerApplication application : applications.values()) {
-      FiCaSchedulerApp attempt =
-          (FiCaSchedulerApp) application.getCurrentAppAttempt();
-      attempt.setHeadroom(Resources.subtract(clusterResource, usedResource));
+    for (FiCaSchedulerApp application : applications.values()) {
+      application.setHeadroom(Resources.subtract(clusterResource, usedResource));
     }
   }
 
@@ -747,26 +697,12 @@ public class FifoScheduler extends AbstractYarnScheduler implements
       nodeUpdate(nodeUpdatedEvent.getRMNode());
     }
     break;
-    case APP_ADDED:
-    {
-      AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
-      addApplication(appAddedEvent.getApplicationId(),
-        appAddedEvent.getQueue(), appAddedEvent.getUser());
-    }
-    break;
-    case APP_REMOVED:
-    {
-      AppRemovedSchedulerEvent appRemovedEvent = (AppRemovedSchedulerEvent)event;
-      doneApplication(appRemovedEvent.getApplicationID(),
-        appRemovedEvent.getFinalState());
-    }
-    break;
     case APP_ATTEMPT_ADDED:
     {
       AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
           (AppAttemptAddedSchedulerEvent) event;
       addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
-        appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
+        appAttemptAddedEvent.getUser());
     }
     break;
     case APP_ATTEMPT_REMOVED:
@@ -776,8 +712,7 @@ public class FifoScheduler extends AbstractYarnScheduler implements
       try {
         doneApplicationAttempt(
           appAttemptRemovedEvent.getApplicationAttemptID(),
-          appAttemptRemovedEvent.getFinalAttemptState(),
-          appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
+          appAttemptRemovedEvent.getFinalAttemptState());
       } catch(IOException ie) {
         LOG.error("Unable to remove application "
             + appAttemptRemovedEvent.getApplicationAttemptID(), ie);
@@ -803,11 +738,12 @@ public class FifoScheduler extends AbstractYarnScheduler implements
 
   private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) {
     // Get the application for the finished container
-    FiCaSchedulerApp application = getCurrentAttemptForContainer(containerId);
+    ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId();
+    FiCaSchedulerApp application = getApplication(applicationAttemptId);
     if (application == null) {
-      LOG.info("Unknown application "
-          + containerId.getApplicationAttemptId().getApplicationId()
-          + " launched container " + containerId + " on node: " + node);
+      LOG.info("Unknown application: " + applicationAttemptId + 
+          " launched container " + containerId +
+          " on node: " + node);
       // Some unknown container sneaked into the system. Kill it.
       this.rmContext.getDispatcher().getEventHandler()
         .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
@@ -828,16 +764,14 @@ public class FifoScheduler extends AbstractYarnScheduler implements
 
     // Get the application for the finished container
     Container container = rmContainer.getContainer();
-    FiCaSchedulerApp application =
-        getCurrentAttemptForContainer(container.getId());
-    ApplicationId appId =
-        container.getId().getApplicationAttemptId().getApplicationId();
+    ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId();
+    FiCaSchedulerApp application = getApplication(applicationAttemptId);
     
     // Get the node on which the container was allocated
     FiCaSchedulerNode node = getNode(container.getNodeId());
     
     if (application == null) {
-      LOG.info("Unknown application: " + appId + 
+      LOG.info("Unknown application: " + applicationAttemptId + 
           " released container " + container.getId() +
           " on node: " + node + 
           " with event: " + event);
@@ -853,7 +787,7 @@ public class FifoScheduler extends AbstractYarnScheduler implements
     // Update total usage
     Resources.subtractFrom(usedResource, container.getResource());
 
-    LOG.info("Application attempt " + application.getApplicationAttemptId() + 
+    LOG.info("Application " + applicationAttemptId + 
         " released container " + container.getId() +
         " on node: " + node + 
         " with event: " + event);
@@ -911,22 +845,11 @@ public class FifoScheduler extends AbstractYarnScheduler implements
     FiCaSchedulerNode node = getNode(nodeId);
     return node == null ? null : new SchedulerNodeReport(node);
   }
-
-  @Override
-  public RMContainer getRMContainer(ContainerId containerId) {
-    FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
-    return (attempt == null) ? null : attempt.getRMContainer(containerId);
-  }
-
-  private FiCaSchedulerApp getCurrentAttemptForContainer(
-      ContainerId containerId) {
-    SchedulerApplication app =
-        applications.get(containerId.getApplicationAttemptId()
-          .getApplicationId());
-    if (app != null) {
-      return (FiCaSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
+  
+  private RMContainer getRMContainer(ContainerId containerId) {
+    FiCaSchedulerApp application = 
+        getApplication(containerId.getApplicationAttemptId());
+    return (application == null) ? null : application.getRMContainer(containerId);
   }
 
   @Override
@@ -943,12 +866,12 @@ public class FifoScheduler extends AbstractYarnScheduler implements
   @Override
   public synchronized List<ApplicationAttemptId> getAppsInQueue(String queueName) {
     if (queueName.equals(DEFAULT_QUEUE.getQueueName())) {
-      List<ApplicationAttemptId> attempts = new ArrayList<ApplicationAttemptId>(
+      List<ApplicationAttemptId> apps = new ArrayList<ApplicationAttemptId>(
           applications.size());
-      for (SchedulerApplication app : applications.values()) {
-        attempts.add(app.getCurrentAppAttempt().getApplicationAttemptId());
+      for (FiCaSchedulerApp app : applications.values()) {
+        apps.add(app.getApplicationAttemptId());
       }
-      return attempts;
+      return apps;
     } else {
       return null;
     }

+ 4 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java

@@ -57,7 +57,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.Task.State;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -165,14 +164,11 @@ public class Application {
     final ResourceScheduler scheduler = resourceManager.getResourceScheduler();
     
     resourceManager.getClientRMService().submitApplication(request);
-
+    
     // Notify scheduler
-    AppAddedSchedulerEvent addAppEvent =
-        new AppAddedSchedulerEvent(this.applicationId, this.queue, "user");
-    scheduler.handle(addAppEvent);
-    AppAttemptAddedSchedulerEvent addAttemptEvent =
-        new AppAttemptAddedSchedulerEvent(this.applicationAttemptId, false);
-    scheduler.handle(addAttemptEvent);
+    AppAttemptAddedSchedulerEvent appAddedEvent1 = new AppAttemptAddedSchedulerEvent(
+            this.applicationAttemptId, this.queue, this.user);
+    scheduler.handle(appAddedEvent1);
   }
   
   public synchronized void addResourceRequestSpec(

+ 4 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java

@@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
-import org.mortbay.log.Log;
 
 public class MockNM {
 
@@ -131,13 +130,12 @@ public class MockNM {
       int containerId, ContainerState containerState) throws Exception {
     HashMap<ApplicationId, List<ContainerStatus>> nodeUpdate =
         new HashMap<ApplicationId, List<ContainerStatus>>(1);
-    ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
-        BuilderUtils.newContainerId(attemptId, containerId), containerState,
-        "Success", 0);
+    ContainerStatus amContainerStatus = BuilderUtils.newContainerStatus(
+        BuilderUtils.newContainerId(attemptId, 1),
+        ContainerState.COMPLETE, "Success", 0);
     ArrayList<ContainerStatus> containerStatusList =
         new ArrayList<ContainerStatus>(1);
-    containerStatusList.add(containerStatus);
-    Log.info("ContainerStatus: " + containerStatus);
+    containerStatusList.add(amContainerStatus);
     nodeUpdate.put(attemptId.getApplicationId(), containerStatusList);
     return nodeHeartbeat(nodeUpdate, true);
   }
@@ -154,7 +152,6 @@ public class MockNM {
     status.setResponseId(resId);
     status.setNodeId(nodeId);
     for (Map.Entry<ApplicationId, List<ContainerStatus>> entry : conts.entrySet()) {
-      Log.info("entry.getValue() " + entry.getValue());
       status.setContainersStatuses(entry.getValue());
     }
     NodeHealthStatus healthStatus = Records.newRecord(NodeHealthStatus.class);

+ 1 - 74
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -30,7 +30,6 @@ import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
@@ -41,10 +40,7 @@ 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.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -60,8 +56,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
 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.RMAppAttemptLaunchFailedEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
@@ -128,33 +122,6 @@ public class MockRM extends ResourceManager {
         attempt.getAppAttemptState());
   }
 
-  public void waitForContainerAllocated(MockNM nm, ContainerId containerId)
-      throws Exception {
-    int timeoutSecs = 0;
-    while (getResourceScheduler().getRMContainer(containerId) == null
-        && timeoutSecs++ < 40) {
-      System.out.println("Waiting for" + containerId + " to be allocated.");
-      nm.nodeHeartbeat(true);
-      Thread.sleep(200);
-    }
-  }
-
-  public void waitForState(MockNM nm, ContainerId containerId,
-      RMContainerState containerState) throws Exception {
-    RMContainer container = getResourceScheduler().getRMContainer(containerId);
-    Assert.assertNotNull("Container shouldn't be null", container);
-    int timeoutSecs = 0;
-    while (!containerState.equals(container.getState()) && timeoutSecs++ < 40) {
-      System.out.println("Container : " + containerId + " State is : "
-          + container.getState() + " Waiting for state : " + containerState);
-      nm.nodeHeartbeat(true);
-      Thread.sleep(300);
-    }
-    System.out.println("Container State is : " + container.getState());
-    Assert.assertEquals("Container state is not correct (timedout)",
-      containerState, container.getState());
-  }
-
   // get new application id
   public GetNewApplicationResponse getNewAppId() throws Exception {
     ApplicationClientProtocol client = getClientRMService();
@@ -205,17 +172,7 @@ public class MockRM extends ResourceManager {
   public RMApp submitApp(int masterMemory, String name, String user,
       Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
       int maxAppAttempts, Credentials ts, String appType,
-      boolean waitForAccepted)
-      throws Exception {
-    return submitApp(masterMemory, name, user, acls, unmanaged, queue,
-      maxAppAttempts, ts, appType, waitForAccepted, false);
-  }
-
-  public RMApp submitApp(int masterMemory, String name, String user,
-      Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
-      int maxAppAttempts, Credentials ts, String appType,
-      boolean waitForAccepted, boolean keepContainers)
-      throws Exception {
+      boolean waitForAccepted) throws Exception {
     ApplicationClientProtocol client = getClientRMService();
     GetNewApplicationResponse resp = client.getNewApplication(Records
         .newRecord(GetNewApplicationRequest.class));
@@ -225,7 +182,6 @@ public class MockRM extends ResourceManager {
         .newRecord(SubmitApplicationRequest.class);
     ApplicationSubmissionContext sub = Records
         .newRecord(ApplicationSubmissionContext.class);
-    sub.setKeepContainersAcrossApplicationAttempts(keepContainers);
     sub.setApplicationId(appId);
     sub.setApplicationName(name);
     sub.setMaxAppAttempts(maxAppAttempts);
@@ -465,33 +421,4 @@ public class MockRM extends ResourceManager {
     // override to disable webapp
   }
 
-  public static void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm,
-      MockAM am) throws Exception {
-    FinishApplicationMasterRequest req =
-        FinishApplicationMasterRequest.newInstance(
-          FinalApplicationStatus.SUCCEEDED, "", "");
-    am.unregisterAppAttempt(req);
-    am.waitForState(RMAppAttemptState.FINISHING);
-    nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
-    am.waitForState(RMAppAttemptState.FINISHED);
-    rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
-  }
-
-  public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
-      throws Exception {
-    RMAppAttempt attempt = app.getCurrentAppAttempt();
-    nm.nodeHeartbeat(true);
-    MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
-    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
-    return am;
-  }
-
-  public static MockAM launchAndRegisterAM(RMApp app, MockRM rm, MockNM nm)
-      throws Exception {
-    MockAM am = launchAM(app, rm, nm);
-    am.registerAppAttempt();
-    rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
-    return am;
-  }
-
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -649,7 +649,7 @@ public class TestClientRMService {
             .currentTimeMillis(), "YARN"));
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(applicationId3, 1);
     RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId,
-        rmContext, yarnScheduler, null, asContext, config, false);
+        rmContext, yarnScheduler, null, asContext, config, null);
     when(app.getCurrentAppAttempt()).thenReturn(rmAppAttemptImpl);
     return app;
   }

+ 9 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java

@@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@@ -298,12 +297,9 @@ public class TestFifoScheduler {
     ApplicationId appId1 = BuilderUtils.newApplicationId(100, 1);
     ApplicationAttemptId appAttemptId1 = BuilderUtils.newApplicationAttemptId(
         appId1, 1);
-    SchedulerEvent appEvent =
-        new AppAddedSchedulerEvent(appId1, "queue", "user");
-    fs.handle(appEvent);
-    SchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId1, false);
-    fs.handle(attemptEvent);
+    SchedulerEvent event1 =
+        new AppAttemptAddedSchedulerEvent(appAttemptId1, "queue", "user");
+    fs.handle(event1);
 
     List<ContainerId> emptyId = new ArrayList<ContainerId>();
     List<ResourceRequest> emptyAsk = new ArrayList<ResourceRequest>();
@@ -392,22 +388,16 @@ public class TestFifoScheduler {
     ApplicationId appId1 = BuilderUtils.newApplicationId(100, 1);
     ApplicationAttemptId appAttemptId1 = BuilderUtils.newApplicationAttemptId(
         appId1, 1);
-    SchedulerEvent appEvent =
-        new AppAddedSchedulerEvent(appId1, "queue", "user");
-    fs.handle(appEvent);
-    SchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId1, false);
-    fs.handle(attemptEvent);
+    SchedulerEvent event1 =
+        new AppAttemptAddedSchedulerEvent(appAttemptId1, "queue", "user");
+    fs.handle(event1);
 
     ApplicationId appId2 = BuilderUtils.newApplicationId(200, 2);
     ApplicationAttemptId appAttemptId2 = BuilderUtils.newApplicationAttemptId(
         appId2, 1);
-    SchedulerEvent appEvent2 =
-        new AppAddedSchedulerEvent(appId2, "queue", "user");
-    fs.handle(appEvent2);
-    SchedulerEvent attemptEvent2 =
-        new AppAttemptAddedSchedulerEvent(appAttemptId2, false);
-    fs.handle(attemptEvent2);
+    SchedulerEvent event2 =
+        new AppAttemptAddedSchedulerEvent(appAttemptId2, "queue", "user");
+    fs.handle(event2);
 
     List<ContainerId> emptyId = new ArrayList<ContainerId>();
     List<ResourceRequest> emptyAsk = new ArrayList<ResourceRequest>();

+ 26 - 97
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java

@@ -18,10 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-import static org.mockito.Matchers.argThat;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.spy;
-
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -33,33 +29,26 @@ import junit.framework.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 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.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
 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;
 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.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.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
@@ -67,9 +56,7 @@ import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.junit.Test;
-import org.mockito.ArgumentMatcher;
 
-@SuppressWarnings({"unchecked", "rawtypes"})
 public class TestRM {
 
   private static final Log LOG = LogFactory.getLog(TestRM.class);
@@ -314,8 +301,6 @@ public class TestRM {
         nm2.nodeHeartbeat(attempt.getAppAttemptId(), container.getId().getId(),
             ContainerState.COMPLETE);
       }
-      nm1.nodeHeartbeat(am.getApplicationAttemptId(), 1,
-        ContainerState.COMPLETE);
       am.waitForState(RMAppAttemptState.FINISHED);
       Assert.assertFalse(nmTokenSecretManager
           .isApplicationAttemptRegistered(attempt.getAppAttemptId()));
@@ -412,19 +397,19 @@ public class TestRM {
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-    MockRM.finishApplicationMaster(app1, rm1, nm1, am1);
+    MockAM am1 = launchAM(app1, rm1, nm1);
+    finishApplicationMaster(app1, rm1, nm1, am1);
 
     // a failed app
     RMApp app2 = rm1.submitApp(200);
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+    MockAM am2 = launchAM(app2, rm1, nm1);
     nm1.nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
     am2.waitForState(RMAppAttemptState.FAILED);
     rm1.waitForState(app2.getApplicationId(), RMAppState.FAILED);
 
     // a killed app
     RMApp app3 = rm1.submitApp(200);
-    MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm1);
+    MockAM am3 = launchAM(app3, rm1, nm1);
     rm1.killApp(app3.getApplicationId());
     rm1.waitForState(app3.getApplicationId(), RMAppState.KILLED);
     rm1.waitForState(am3.getApplicationAttemptId(), RMAppAttemptState.KILLED);
@@ -464,7 +449,7 @@ public class TestRM {
 
     // a failed app
     RMApp app2 = rm1.submitApp(200);
-    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+    MockAM am2 = launchAM(app2, rm1, nm1);
     nm1
       .nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
     am2.waitForState(RMAppAttemptState.FAILED);
@@ -481,77 +466,26 @@ public class TestRM {
     Assert.assertEquals(-1, report1.getRpcPort());
   }
 
-  /**
-   * Validate killing an application when it is at accepted state.
-   * @throws Exception exception
-   */
-  @Test (timeout = 60000)
-  public void testApplicationKillAtAcceptedState() throws Exception {
-
-    YarnConfiguration conf = new YarnConfiguration();
-    final Dispatcher dispatcher = new AsyncDispatcher() {
-      @Override
-      public EventHandler getEventHandler() {
-
-        class EventArgMatcher extends ArgumentMatcher<AbstractEvent> {
-          @Override
-          public boolean matches(Object argument) {
-            if (argument instanceof RMAppAttemptEvent) {
-              if (((RMAppAttemptEvent) argument).getType().equals(
-                RMAppAttemptEventType.KILL)) {
-                return true;
-              }
-            }
-            return false;
-          }
-        }
-
-        EventHandler handler = spy(super.getEventHandler());
-        doNothing().when(handler).handle(argThat(new EventArgMatcher()));
-        return handler;
-      }
-    };
-
-    MockRM rm = new MockRM(conf) {
-      @Override
-      protected Dispatcher createDispatcher() {
-        return dispatcher;
-      }
-    };
-
-    rm.start();
-    MockNM nm1 =
-        new MockNM("127.0.0.1:1234", 15120, rm.getResourceTrackerService());
-    nm1.registerNode();
+  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;
+  }
 
-    // a failed app
-    RMApp application = rm.submitApp(200);
-    MockAM am = MockRM.launchAM(application, rm, nm1);
-    am.waitForState(RMAppAttemptState.LAUNCHED);
-    nm1.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.RUNNING);
-    rm.waitForState(application.getApplicationId(), RMAppState.ACCEPTED);
-
-    // Now kill the application before new attempt is launched, the app report
-    // returns the invalid AM host and port.
-    KillApplicationRequest request =
-        KillApplicationRequest.newInstance(application.getApplicationId());
-    rm.getClientRMService().forceKillApplication(request);
-
-    // Specific test for YARN-1689 follows
-    // Now let's say a race causes AM to register now. This should not crash RM.
-    am.registerAppAttempt(false);
-
-    // We explicitly intercepted the kill-event to RMAppAttempt, so app should
-    // still be in KILLING state.
-    rm.waitForState(application.getApplicationId(), RMAppState.KILLING);
-    // AM should now be in running
-    rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.RUNNING);
-
-    // Simulate that appAttempt is killed.
-    rm.getRMContext().getDispatcher().getEventHandler().handle(
-        new RMAppEvent(application.getApplicationId(),
-          RMAppEventType.ATTEMPT_KILLED));
-    rm.waitForState(application.getApplicationId(), RMAppState.KILLED);
+  private void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm,
+      MockAM am) throws Exception {
+    FinishApplicationMasterRequest req =
+        FinishApplicationMasterRequest.newInstance(
+          FinalApplicationStatus.SUCCEEDED, "", "");
+    am.unregisterAppAttempt(req);
+    am.waitForState(RMAppAttemptState.FINISHING);
+    nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
+    am.waitForState(RMAppAttemptState.FINISHED);
+    rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
   }
 
   public static void main(String[] args) throws Exception {
@@ -559,10 +493,5 @@ public class TestRM {
     t.testGetNewAppId();
     t.testAppWithNoContainers();
     t.testAppOnMultiNode();
-    t.testNMToken();
-    t.testActivatingApplicationAfterAddingNM();
-    t.testInvalidateAMHostPortWhenAMFailedOrKilled();
-    t.testInvalidatedAMHostPortOnAMRestart();
-    t.testApplicationKillAtAcceptedState();
   }
 }

+ 10 - 142
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java

@@ -84,7 +84,6 @@ 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.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.log4j.Level;
@@ -180,7 +179,7 @@ public class TestRMRestart {
     am1.registerAppAttempt();
 
     // AM request for containers
-    am1.allocate("127.0.0.1" , 1000, 1, new ArrayList<ContainerId>());   
+    am1.allocate("127.0.0.1" , 1000, 1, new ArrayList<ContainerId>());    
     // kick the scheduler
     nm1.nodeHeartbeat(true);
     List<Container> conts = am1.allocate(new ArrayList<ResourceRequest>(),
@@ -249,7 +248,7 @@ public class TestRMRestart {
     // verify correct number of attempts and other data
     RMApp loadedApp1 = rm2.getRMContext().getRMApps().get(app1.getApplicationId());
     Assert.assertNotNull(loadedApp1);
-    Assert.assertEquals(1, loadedApp1.getAppAttempts().size());
+    //Assert.assertEquals(1, loadedApp1.getAppAttempts().size());
     Assert.assertEquals(app1.getApplicationSubmissionContext()
         .getApplicationId(), loadedApp1.getApplicationSubmissionContext()
         .getApplicationId());
@@ -262,7 +261,7 @@ public class TestRMRestart {
         .getApplicationId());
     
     // verify state machine kicked into expected states
-    rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.ACCEPTED);
+    rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.RUNNING);
     rm2.waitForState(loadedApp2.getApplicationId(), RMAppState.ACCEPTED);
     
     // verify attempts for apps
@@ -300,11 +299,7 @@ public class TestRMRestart {
     nm2.registerNode();
     
     rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.ACCEPTED);
-    // wait for the 2nd attempt to be started.
-    int timeoutSecs = 0;
-    while (loadedApp1.getAppAttempts().size() != 2 && timeoutSecs++ < 40) {;
-      Thread.sleep(200);
-    }
+    Assert.assertEquals(2, loadedApp1.getAppAttempts().size());    
 
     // verify no more reboot response sent
     hbResponse = nm1.nodeHeartbeat(true);
@@ -481,10 +476,10 @@ public class TestRMRestart {
     Assert.assertEquals(NodeAction.RESYNC, res.getNodeAction());
     
     RMApp rmApp = rm2.getRMContext().getRMApps().get(app1.getApplicationId());
-    // application should be in ACCEPTED state
-    rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
+    // application should be in running state
+    rm2.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
     
-    Assert.assertEquals(RMAppState.ACCEPTED, rmApp.getState());
+    Assert.assertEquals(RMAppState.RUNNING, rmApp.getState());
     // new attempt should not be started
     Assert.assertEquals(2, rmApp.getAppAttempts().size());
     // am1 attempt should be in FAILED state where as am2 attempt should be in
@@ -521,9 +516,9 @@ public class TestRMRestart {
     nm1.setResourceTrackerService(rm3.getResourceTrackerService());
     
     rmApp = rm3.getRMContext().getRMApps().get(app1.getApplicationId());
-    // application should be in ACCEPTED state
-    rm3.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
-    Assert.assertEquals(rmApp.getState(), RMAppState.ACCEPTED);
+    // application should be in running state
+    rm3.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
+    Assert.assertEquals(rmApp.getState(), RMAppState.RUNNING);
     // new attempt should not be started
     Assert.assertEquals(3, rmApp.getAppAttempts().size());
     // am1 and am2 attempts should be in FAILED state where as am3 should be
@@ -567,11 +562,6 @@ public class TestRMRestart {
     
     rmApp = rm4.getRMContext().getRMApps().get(app1.getApplicationId());
     rm4.waitForState(rmApp.getApplicationId(), RMAppState.ACCEPTED);
-    // wait for the attempt to be created.
-    int timeoutSecs = 0;
-    while (rmApp.getAppAttempts().size() != 2 && timeoutSecs++ < 40) {
-      Thread.sleep(200);
-    }
     Assert.assertEquals(4, rmApp.getAppAttempts().size());
     Assert.assertEquals(RMAppState.ACCEPTED, rmApp.getState());
     rm4.waitForState(latestAppAttemptId, RMAppAttemptState.SCHEDULED);
@@ -1544,128 +1534,6 @@ public class TestRMRestart {
     Assert.assertEquals(2, ((TestMemoryRMStateStore) memStore).updateApp);
   }
 
-  @SuppressWarnings("resource")
-  @Test
-  public void testQueueMetricsOnRMRestart() throws Exception {
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
-        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
-    // PHASE 1: create state in an RM
-    // start RM
-    MockRM rm1 = new MockRM(conf, memStore);
-    rm1.start();
-    MockNM nm1 =
-        new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
-    nm1.registerNode();
-    QueueMetrics qm1 = rm1.getResourceScheduler().getRootQueueMetrics();
-    resetQueueMetrics(qm1);
-    assertQueueMetrics(qm1, 0, 0, 0, 0);
-
-    // create app that gets launched and does allocate before RM restart
-    RMApp app1 = rm1.submitApp(200);
-    assertQueueMetrics(qm1, 1, 1, 0, 0);
-    nm1.nodeHeartbeat(true);
-    RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
-    ApplicationAttemptId attemptId1 = attempt1.getAppAttemptId();
-    rm1.waitForState(attemptId1, RMAppAttemptState.ALLOCATED);
-    MockAM am1 = rm1.sendAMLaunched(attempt1.getAppAttemptId());
-    am1.registerAppAttempt();
-    am1.allocate("127.0.0.1" , 1000, 1, new ArrayList<ContainerId>()); 
-    nm1.nodeHeartbeat(true);
-    List<Container> conts = am1.allocate(new ArrayList<ResourceRequest>(),
-        new ArrayList<ContainerId>()).getAllocatedContainers();
-    while (conts.size() == 0) {
-      nm1.nodeHeartbeat(true);
-      conts.addAll(am1.allocate(new ArrayList<ResourceRequest>(),
-          new ArrayList<ContainerId>()).getAllocatedContainers());
-      Thread.sleep(500);
-    }
-    assertQueueMetrics(qm1, 1, 0, 1, 0);
-
-    // PHASE 2: create new RM and start from old state
-    // create new RM to represent restart and recover state
-    MockRM rm2 = new MockRM(conf, memStore);
-    rm2.start();
-    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
-    QueueMetrics qm2 = rm2.getResourceScheduler().getRootQueueMetrics();
-    resetQueueMetrics(qm2);
-    assertQueueMetrics(qm2, 0, 0, 0, 0);
-    // recover app
-    RMApp loadedApp1 = rm2.getRMContext().getRMApps().get(app1.getApplicationId());
-    am1.setAMRMProtocol(rm2.getApplicationMasterService());
-    am1.allocate(new ArrayList<ResourceRequest>(), new ArrayList<ContainerId>());
-    nm1.nodeHeartbeat(true);
-    nm1 = new MockNM("127.0.0.1:1234", 15120, rm2.getResourceTrackerService());
-    List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
-    ContainerStatus containerStatus =
-        BuilderUtils.newContainerStatus(BuilderUtils.newContainerId(loadedApp1
-            .getCurrentAppAttempt().getAppAttemptId(), 1),
-            ContainerState.COMPLETE, "Killed AM container", 143);
-    containerStatuses.add(containerStatus);
-    nm1.registerNode(containerStatuses);
-    int timeoutSecs = 0;
-    while (loadedApp1.getAppAttempts().size() != 2 && timeoutSecs++ < 40) {;
-      Thread.sleep(200);
-    }
-
-    assertQueueMetrics(qm2, 1, 1, 0, 0);
-    nm1.nodeHeartbeat(true);
-    attempt1 = loadedApp1.getCurrentAppAttempt();
-    attemptId1 = attempt1.getAppAttemptId();
-    rm2.waitForState(attemptId1, RMAppAttemptState.ALLOCATED);
-    assertQueueMetrics(qm2, 1, 0, 1, 0);
-    am1 = rm2.sendAMLaunched(attempt1.getAppAttemptId());
-    am1.registerAppAttempt();
-    am1.allocate("127.0.0.1" , 1000, 3, new ArrayList<ContainerId>());
-    nm1.nodeHeartbeat(true);
-    conts = am1.allocate(new ArrayList<ResourceRequest>(),
-        new ArrayList<ContainerId>()).getAllocatedContainers();
-    while (conts.size() == 0) {
-      nm1.nodeHeartbeat(true);
-      conts.addAll(am1.allocate(new ArrayList<ResourceRequest>(),
-          new ArrayList<ContainerId>()).getAllocatedContainers());
-      Thread.sleep(500);
-    }
-
-    // finish the AMs
-    finishApplicationMaster(loadedApp1, rm2, nm1, am1);
-    assertQueueMetrics(qm2, 1, 0, 0, 1);
-
-    // stop RM's
-    rm2.stop();
-    rm1.stop();
-  }
-
-
-  // The metrics has some carry-on value from the previous RM, because the
-  // test case is in-memory, for the same queue name (e.g. root), there's
-  // always a singleton QueueMetrics object.
-  private int appsSubmittedCarryOn = 0;
-  private int appsPendingCarryOn = 0;
-  private int appsRunningCarryOn = 0;
-  private int appsCompletedCarryOn = 0;
-
-  private void resetQueueMetrics(QueueMetrics qm) {
-    appsSubmittedCarryOn = qm.getAppsSubmitted();
-    appsPendingCarryOn = qm.getAppsPending();
-    appsRunningCarryOn = qm.getAppsRunning();
-    appsCompletedCarryOn = qm.getAppsCompleted();
-  }
-
-  private void assertQueueMetrics(QueueMetrics qm, int appsSubmitted,
-      int appsPending, int appsRunning, int appsCompleted) {
-    Assert.assertEquals(qm.getAppsSubmitted(),
-        appsSubmitted + appsSubmittedCarryOn);
-    Assert.assertEquals(qm.getAppsPending(),
-        appsPending + appsPendingCarryOn);
-    Assert.assertEquals(qm.getAppsRunning(),
-        appsRunning + appsRunningCarryOn);
-    Assert.assertEquals(qm.getAppsCompleted(),
-        appsCompleted + appsCompletedCarryOn);
-  }
-
   public class TestMemoryRMStateStore extends MemoryRMStateStore {
     int count = 0;
     public int updateApp = 0;

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java

@@ -164,7 +164,7 @@ public class TestResourceManager {
     // Notify scheduler application is finished.
     AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
         new AppAttemptRemovedSchedulerEvent(
-          application.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false);
+          application.getApplicationAttemptId(), RMAppAttemptState.FINISHED);
     resourceManager.getResourceScheduler().handle(appRemovedEvent1);
     
     checkResourceUsage(nm1, nm2);

+ 269 - 199
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java

@@ -18,31 +18,49 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager;
 
-import java.util.ArrayList;
-import java.util.HashMap;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import junit.framework.Assert;
 
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+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.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
-import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
-import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-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.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
+import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 /**
@@ -50,186 +68,238 @@ import org.junit.Test;
  *
  */
 public class TestAMRestart {
-
-  @Test
-  public void testAMRestartWithExistingContainers() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-
-    MockRM rm1 = new MockRM(conf);
-    rm1.start();
-    RMApp app1 =
-        rm1.submitApp(200, "name", "user",
-          new HashMap<ApplicationAccessType, String>(), false, "default", -1,
-          null, "MAPREDUCE", false, true);
-    MockNM nm1 =
-        new MockNM("127.0.0.1:1234", 10240, rm1.getResourceTrackerService());
-    nm1.registerNode();
-    MockNM nm2 =
-        new MockNM("127.0.0.1:2351", 4089, rm1.getResourceTrackerService());
-    nm2.registerNode();
-
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-    int NUM_CONTAINERS = 3;
-    // allocate NUM_CONTAINERS containers
-    am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS,
-      new ArrayList<ContainerId>());
-    nm1.nodeHeartbeat(true);
-
-    // wait for containers to be allocated.
-    List<Container> containers =
-        am1.allocate(new ArrayList<ResourceRequest>(),
-          new ArrayList<ContainerId>()).getAllocatedContainers();
-    while (containers.size() != NUM_CONTAINERS) {
-      nm1.nodeHeartbeat(true);
-      containers.addAll(am1.allocate(new ArrayList<ResourceRequest>(),
-        new ArrayList<ContainerId>()).getAllocatedContainers());
-      Thread.sleep(200);
-    }
-
-    // launch the 2nd container, for testing running container transferred.
-    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
-    ContainerId containerId2 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
-    rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
-
-    // launch the 3rd container, for testing container allocated by previous
-    // attempt is completed by the next new attempt/
-    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.RUNNING);
-    ContainerId containerId3 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 3);
-    rm1.waitForState(nm1, containerId3, RMContainerState.RUNNING);
-
-    // 4th container still in AQUIRED state. for testing Acquired container is
-    // always killed.
-    ContainerId containerId4 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 4);
-    rm1.waitForState(nm1, containerId4, RMContainerState.ACQUIRED);
-
-    // 5th container is in Allocated state. for testing allocated container is
-    // always killed.
-    am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
-    nm1.nodeHeartbeat(true);
-    ContainerId containerId5 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 5);
-    rm1.waitForContainerAllocated(nm1, containerId5);
-    rm1.waitForState(nm1, containerId5, RMContainerState.ALLOCATED);
-
-    // 6th container is in Reserved state.
-    am1.allocate("127.0.0.1", 6000, 1, new ArrayList<ContainerId>());
-    ContainerId containerId6 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 6);
-    nm1.nodeHeartbeat(true);
-    SchedulerApplicationAttempt schedulerAttempt =
-        ((CapacityScheduler) rm1.getResourceScheduler())
-          .getCurrentAttemptForContainer(containerId6);
-    while (schedulerAttempt.getReservedContainers().size() == 0) {
-      System.out.println("Waiting for container " + containerId6
-          + " to be reserved.");
-      nm1.nodeHeartbeat(true);
-      Thread.sleep(200);
-    }
-    // assert containerId6 is reserved.
-    Assert.assertEquals(containerId6, schedulerAttempt.getReservedContainers()
-      .get(0).getContainerId());
-
-    // fail the AM by sending CONTAINER_FINISHED event without registering.
-    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
-    am1.waitForState(RMAppAttemptState.FAILED);
-
-    // wait for some time. previous AM's running containers should still remain
-    // in scheduler even though am failed
-    Thread.sleep(3000);
-    rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
-    // acquired/allocated containers are cleaned up.
-    Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId4));
-    Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId5));
-
-    // wait for app to start a new attempt.
-    rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
-    // assert this is a new AM.
-    ApplicationAttemptId newAttemptId =
-        app1.getCurrentAppAttempt().getAppAttemptId();
-    Assert.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId()));
-
-    // launch the new AM
-    RMAppAttempt attempt2 = app1.getCurrentAppAttempt();
-    nm1.nodeHeartbeat(true);
-    MockAM am2 = rm1.sendAMLaunched(attempt2.getAppAttemptId());
-    RegisterApplicationMasterResponse registerResponse =
-        am2.registerAppAttempt();
-
-    // Assert two containers are running: container2 and container3;
-    Assert.assertEquals(2, registerResponse.getContainersFromPreviousAttempt()
-      .size());
-    boolean containerId2Exists = false, containerId3Exists = false;
-    for (Container container : registerResponse
-      .getContainersFromPreviousAttempt()) {
-      if (container.getId().equals(containerId2)) {
-        containerId2Exists = true;
-      }
-      if (container.getId().equals(containerId3)) {
-        containerId3Exists = true;
-      }
-    }
-    Assert.assertTrue(containerId2Exists && containerId3Exists);
-    rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
-
-    // complete container by sending the container complete event which has earlier
-    // attempt's attemptId
-    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.COMPLETE);
-    rm1.waitForState(nm1, containerId3, RMContainerState.COMPLETED);
-
-    // Even though the completed container containerId3 event was sent to the
-    // earlier failed attempt, new RMAppAttempt can also capture this container
-    // info.
-    // completed containerId4 is also transferred to the new attempt.
-    RMAppAttempt newAttempt =
-        app1.getRMAppAttempt(am2.getApplicationAttemptId());
-    // 4 containers finished, acquired/allocated/reserved/completed.
-    Assert.assertEquals(4, newAttempt.getJustFinishedContainers().size());
-    boolean container3Exists = false, container4Exists = false, container5Exists =
-        false, container6Exists = false;
-    for(ContainerStatus status :  newAttempt.getJustFinishedContainers()) {
-      if(status.getContainerId().equals(containerId3)) {
-        // containerId3 is the container ran by previous attempt but finished by the
-        // new attempt.
-        container3Exists = true;
-      }
-      if (status.getContainerId().equals(containerId4)) {
-        // containerId4 is the Acquired Container killed by the previous attempt,
-        // it's now inside new attempt's finished container list.
-        container4Exists = true;
-      }
-      if (status.getContainerId().equals(containerId5)) {
-        // containerId5 is the Allocated container killed by previous failed attempt.
-        container5Exists = true;
-      }
-      if (status.getContainerId().equals(containerId6)) {
-        // containerId6 is the reserved container killed by previous failed attempt.
-        container6Exists = true;
-      }
-    }
-    Assert.assertTrue(container3Exists && container4Exists && container5Exists
-        && container6Exists);
-
-    // New SchedulerApplicationAttempt also has the containers info.
-    rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
-
-    // record the scheduler attempt for testing.
-    SchedulerApplicationAttempt schedulerNewAttempt =
-        ((CapacityScheduler) rm1.getResourceScheduler())
-          .getCurrentAttemptForContainer(containerId2);
-    // finish this application
-    MockRM.finishApplicationMaster(app1, rm1, nm1, am2);
-
-    // the 2nd attempt released the 1st attempt's running container, when the
-    // 2nd attempt finishes.
-    Assert.assertFalse(schedulerNewAttempt.getLiveContainers().contains(
-      containerId2));
-    // all 4 normal containers finished.
-    Assert.assertEquals(5, newAttempt.getJustFinishedContainers().size());
-
-    rm1.stop();
-  }
+//  private static final Log LOG = LogFactory.getLog(TestAMRestart.class);
+//  ApplicationsManagerImpl appImpl;
+//  RMContext asmContext = new RMContextImpl(new MemStore());
+//  ApplicationTokenSecretManager appTokenSecretManager =
+//    new ApplicationTokenSecretManager();
+//  DummyResourceScheduler scheduler;
+//  private ClientRMService clientRMService;
+//  int count = 0;
+//  ApplicationId appID;
+//  final int maxFailures = 3;
+//  AtomicInteger launchNotify = new AtomicInteger();
+//  AtomicInteger schedulerNotify = new AtomicInteger();
+//  volatile boolean stop = false;
+//  int schedulerAddApplication = 0;
+//  int schedulerRemoveApplication = 0;
+//  int launcherLaunchCalled = 0;
+//  int launcherCleanupCalled = 0;
+//  private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+//
+//  private class ExtApplicationsManagerImpl extends ApplicationsManagerImpl {
+//    public ExtApplicationsManagerImpl(
+//        ApplicationTokenSecretManager applicationTokenSecretManager,
+//        YarnScheduler scheduler, RMContext asmContext) {
+//      super(applicationTokenSecretManager, scheduler, asmContext);
+//    }
+//
+//    @Override
+//    public EventHandler<ASMEvent<AMLauncherEventType>> createNewApplicationMasterLauncher(
+//        ApplicationTokenSecretManager tokenSecretManager) {
+//      return new DummyAMLauncher();
+//    }
+//  }
+//
+//  private class DummyAMLauncher implements EventHandler<ASMEvent<AMLauncherEventType>> {
+//
+//    public DummyAMLauncher() {
+//      asmContext.getDispatcher().register(AMLauncherEventType.class, this);
+//      new Thread() {
+//        public void run() {
+//          while (!stop) {
+//            LOG.info("DEBUG -- waiting for launch");
+//            synchronized(launchNotify) {
+//              while (launchNotify.get() == 0) {
+//                try {
+//                  launchNotify.wait();
+//                } catch (InterruptedException e) {
+//                }
+//              }
+//              asmContext.getDispatcher().getEventHandler().handle(
+//                  new ApplicationEvent(
+//                      ApplicationEventType.LAUNCHED, appID));
+//              launchNotify.addAndGet(-1);
+//            }
+//          }
+//        }
+//      }.start();
+//    }
+//
+//    @Override
+//    public void handle(ASMEvent<AMLauncherEventType> event) {
+//      switch (event.getType()) {
+//      case CLEANUP:
+//        launcherCleanupCalled++;
+//        break;
+//      case LAUNCH:
+//        LOG.info("DEBUG -- launching");
+//        launcherLaunchCalled++;
+//        synchronized (launchNotify) {
+//          launchNotify.addAndGet(1);
+//          launchNotify.notify();
+//        }
+//        break;
+//      default:
+//        break;
+//      }
+//    }
+//  }
+//
+//  private class DummyResourceScheduler implements ResourceScheduler {
+//
+//    @Override
+//    public void removeNode(RMNode node) {
+//    }
+//
+//    @Override
+//    public Allocation allocate(ApplicationId applicationId,
+//        List<ResourceRequest> ask, List<Container> release) throws IOException {
+//      Container container = recordFactory.newRecordInstance(Container.class);
+//      container.setContainerToken(recordFactory.newRecordInstance(ContainerToken.class));
+//      container.setNodeId(recordFactory.newRecordInstance(NodeId.class));
+//      container.setContainerManagerAddress("localhost");
+//      container.setNodeHttpAddress("localhost:8042");
+//      container.setId(recordFactory.newRecordInstance(ContainerId.class));
+//      container.getId().setAppId(appID);
+//      container.getId().setId(count);
+//      count++;
+//      return new Allocation(Arrays.asList(container), Resources.none());
+//    }
+//
+//    @Override
+//    public void handle(ASMEvent<ApplicationTrackerEventType> event) {
+//      switch (event.getType()) {
+//      case ADD:
+//        schedulerAddApplication++;
+//        break;
+//      case EXPIRE:
+//        schedulerRemoveApplication++;
+//        LOG.info("REMOVING app : " + schedulerRemoveApplication);
+//        if (schedulerRemoveApplication == maxFailures) {
+//          synchronized (schedulerNotify) {
+//            schedulerNotify.addAndGet(1);
+//            schedulerNotify.notify();
+//          }
+//        }
+//        break;
+//      default:
+//        break;
+//      }
+//    }
+//
+//    @Override
+//    public QueueInfo getQueueInfo(String queueName,
+//        boolean includeChildQueues,
+//        boolean recursive) throws IOException {
+//      return null;
+//    }
+//    @Override
+//    public List<QueueUserACLInfo> getQueueUserAclInfo() {
+//      return null;
+//    }
+//    @Override
+//    public void addApplication(ApplicationId applicationId,
+//        ApplicationMaster master, String user, String queue, Priority priority,
+//        ApplicationStore store)
+//        throws IOException {
+//    }
+//    @Override
+//    public void addNode(RMNode nodeInfo) {
+//    }
+//    @Override
+//    public void recover(RMState state) throws Exception {
+//    }
+//    @Override
+//    public void reinitialize(Configuration conf,
+//        ContainerTokenSecretManager secretManager, RMContext rmContext)
+//        throws IOException {
+//    }
+//
+//    @Override
+//    public void nodeUpdate(RMNode nodeInfo,
+//        Map<String, List<Container>> containers) {
+//    }
+//
+//    @Override
+//    public Resource getMaximumResourceCapability() {
+//      // TODO Auto-generated method stub
+//      return null;
+//    }
+//
+//    @Override
+//    public Resource getMinimumResourceCapability() {
+//      // TODO Auto-generated method stub
+//      return null;
+//    }
+//  }
+//
+//  @Before
+//  public void setUp() {
+//
+//    asmContext.getDispatcher().register(ApplicationEventType.class,
+//        new ResourceManager.ApplicationEventDispatcher(asmContext));
+//
+//    appID = recordFactory.newRecordInstance(ApplicationId.class);
+//    appID.setClusterTimestamp(System.currentTimeMillis());
+//    appID.setId(1);
+//    Configuration conf = new Configuration();
+//    scheduler = new DummyResourceScheduler();
+//    asmContext.getDispatcher().init(conf);
+//    asmContext.getDispatcher().start();
+//    asmContext.getDispatcher().register(ApplicationTrackerEventType.class, scheduler);
+//    appImpl = new ExtApplicationsManagerImpl(appTokenSecretManager, scheduler, asmContext);
+//
+//    conf.setLong(YarnConfiguration.AM_EXPIRY_INTERVAL, 1000L);
+//    conf.setInt(RMConfig.AM_MAX_RETRIES, maxFailures);
+//    appImpl.init(conf);
+//    appImpl.start();
+//
+//    this.clientRMService = new ClientRMService(asmContext, appImpl
+//        .getAmLivelinessMonitor(), appImpl.getClientToAMSecretManager(),
+//        scheduler);
+//    this.clientRMService.init(conf);
+//  }
+//
+//  @After
+//  public void tearDown() {
+//  }
+//
+//  private void waitForFailed(AppAttempt application, ApplicationState
+//      finalState) throws Exception {
+//    int count = 0;
+//    while(application.getState() != finalState && count < 10) {
+//      Thread.sleep(500);
+//      count++;
+//    }
+//    Assert.assertEquals(finalState, application.getState());
+//  }
+//
+//  @Test
+//  public void testAMRestart() throws Exception {
+//    ApplicationSubmissionContext subContext = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
+//    subContext.setApplicationId(appID);
+//    subContext.setApplicationName("dummyApp");
+////    subContext.command = new ArrayList<String>();
+////    subContext.environment = new HashMap<String, String>();
+////    subContext.fsTokens = new ArrayList<String>();
+//    subContext.setFsTokensTodo(ByteBuffer.wrap(new byte[0]));
+//    SubmitApplicationRequest request = recordFactory
+//        .newRecordInstance(SubmitApplicationRequest.class);
+//    request.setApplicationSubmissionContext(subContext);
+//    clientRMService.submitApplication(request);
+//    AppAttempt application = asmContext.getApplications().get(appID);
+//    synchronized (schedulerNotify) {
+//      while(schedulerNotify.get() == 0) {
+//        schedulerNotify.wait();
+//      }
+//    }
+//    Assert.assertEquals(maxFailures, launcherCleanupCalled);
+//    Assert.assertEquals(maxFailures, launcherLaunchCalled);
+//    Assert.assertEquals(maxFailures, schedulerAddApplication);
+//    Assert.assertEquals(maxFailures, schedulerRemoveApplication);
+//    Assert.assertEquals(maxFailures, application.getFailedCount());
+//    waitForFailed(application, ApplicationState.FAILED);
+//    stop = true;
+//  }
 }

+ 10 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java

@@ -463,7 +463,7 @@ public class TestRMAppTransitions {
     LOG.info("--- START: testUnmanagedAppFailPath ---");
     application = testCreateAppRunning(subContext);
     RMAppEvent event = new RMAppFailedAttemptEvent(
-        application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, "", false);
+        application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, "");
     application.handle(event);
     rmDispatcher.await();
     RMAppAttempt appAttempt = application.getCurrentAppAttempt();
@@ -575,9 +575,7 @@ public class TestRMAppTransitions {
         RMAppEventType.KILL);
     application.handle(event);
     rmDispatcher.await();
-    sendAppUpdateSavedEvent(application);
-    assertKilled(application);
-    assertAppFinalStateSaved(application);
+    assertAppAndAttemptKilled(application);
   }
 
   @Test
@@ -590,9 +588,9 @@ public class TestRMAppTransitions {
     for (int i=1; i < maxAppAttempts; i++) {
       RMAppEvent event = 
           new RMAppFailedAttemptEvent(application.getApplicationId(), 
-              RMAppEventType.ATTEMPT_FAILED, "", false);
+              RMAppEventType.ATTEMPT_FAILED, "");
       application.handle(event);
-      assertAppState(RMAppState.ACCEPTED, application);
+      assertAppState(RMAppState.SUBMITTED, application);
       event = 
           new RMAppEvent(application.getApplicationId(), 
               RMAppEventType.APP_ACCEPTED);
@@ -606,7 +604,7 @@ public class TestRMAppTransitions {
     String message = "Test fail";
     RMAppEvent event = 
         new RMAppFailedAttemptEvent(application.getApplicationId(), 
-            RMAppEventType.ATTEMPT_FAILED, message, false);
+            RMAppEventType.ATTEMPT_FAILED, message);
     application.handle(event);
     rmDispatcher.await();
     sendAppUpdateSavedEvent(application);
@@ -623,16 +621,7 @@ public class TestRMAppTransitions {
         RMAppEventType.KILL);
     application.handle(event);
     rmDispatcher.await();
-
-    assertAppState(RMAppState.KILLING, application);
-    RMAppEvent appAttemptKilled =
-        new RMAppEvent(application.getApplicationId(),
-          RMAppEventType.ATTEMPT_KILLED);
-    application.handle(appAttemptKilled);
-    assertAppState(RMAppState.FINAL_SAVING, application);
-    sendAppUpdateSavedEvent(application);
-    assertKilled(application);
-    assertAppFinalStateSaved(application);
+    assertAppAndAttemptKilled(application);
   }
 
   @Test
@@ -671,10 +660,10 @@ public class TestRMAppTransitions {
     for (int i=1; i<maxAppAttempts; i++) {
       RMAppEvent event = 
           new RMAppFailedAttemptEvent(application.getApplicationId(), 
-              RMAppEventType.ATTEMPT_FAILED, "", false);
+              RMAppEventType.ATTEMPT_FAILED, "");
       application.handle(event);
       rmDispatcher.await();
-      assertAppState(RMAppState.ACCEPTED, application);
+      assertAppState(RMAppState.SUBMITTED, application);
       appAttempt = application.getCurrentAppAttempt();
       Assert.assertEquals(++expectedAttemptId, 
           appAttempt.getAppAttemptId().getAttemptId());
@@ -696,7 +685,7 @@ public class TestRMAppTransitions {
     // after max application attempts
     RMAppEvent event = 
         new RMAppFailedAttemptEvent(application.getApplicationId(), 
-            RMAppEventType.ATTEMPT_FAILED, "", false);
+            RMAppEventType.ATTEMPT_FAILED, "");
     application.handle(event);
     rmDispatcher.await();
     sendAppUpdateSavedEvent(application);
@@ -822,7 +811,7 @@ public class TestRMAppTransitions {
     // KILLED => KILLED event RMAppEventType.ATTEMPT_FAILED
     event = 
         new RMAppFailedAttemptEvent(application.getApplicationId(), 
-            RMAppEventType.ATTEMPT_FAILED, "", false);
+            RMAppEventType.ATTEMPT_FAILED, "");
     application.handle(event);
     rmDispatcher.await();
     assertTimesAtFinish(application);

+ 21 - 90
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -51,7 +51,6 @@ 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.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -69,10 +68,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventT
 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;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
@@ -80,6 +79,7 @@ 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.RMAppAttemptNewSavedEvent;
 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.RMAppAttemptUnregistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
@@ -121,15 +121,14 @@ public class TestRMAppAttemptTransitions {
   private AMLivelinessMonitor amFinishingMonitor;
   private RMStateStore store;
 
-  private RMAppImpl application;
+  private RMApp application;
   private RMAppAttempt applicationAttempt;
 
   private Configuration conf = new Configuration();
   private AMRMTokenSecretManager amRMTokenManager = spy(new AMRMTokenSecretManager(conf));
   private ClientToAMTokenSecretManagerInRM clientToAMTokenManager =
       spy(new ClientToAMTokenSecretManagerInRM());
-  private boolean transferStateFromPreviousAttempt = false;
-
+  
   private final class TestApplicationAttemptEventDispatcher implements
       EventHandler<RMAppAttemptEvent> {
 
@@ -152,11 +151,6 @@ public class TestRMAppAttemptTransitions {
     @Override
     public void handle(RMAppEvent event) {
       assertEquals(application.getApplicationId(), event.getApplicationId());
-      if (event instanceof RMAppFailedAttemptEvent) {
-        transferStateFromPreviousAttempt =
-            ((RMAppFailedAttemptEvent) event)
-              .getTransferStateFromPreviousAttempt();
-      }
       try {
         application.handle(event);
       } catch (Throwable t) {
@@ -261,10 +255,10 @@ public class TestRMAppAttemptTransitions {
 
     unmanagedAM = false;
     
-    application = mock(RMAppImpl.class);
+    application = mock(RMApp.class);
     applicationAttempt =
         new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler,
-          masterService, submissionContext, new Configuration(), false);
+          masterService, submissionContext, new Configuration(), user);
     when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt);
     when(application.getApplicationId()).thenReturn(applicationId);
     
@@ -378,7 +372,6 @@ public class TestRMAppAttemptTransitions {
     assertNull(applicationAttempt.getFinalApplicationStatus());
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
     verifyAttemptFinalStateSaved();
-    assertFalse(transferStateFromPreviousAttempt);
   }
   
   /**
@@ -415,6 +408,9 @@ public class TestRMAppAttemptTransitions {
     assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
     assertEquals(0, applicationAttempt.getRanNodes().size());
     assertNull(applicationAttempt.getFinalApplicationStatus());
+    
+    // Check events
+    verify(application).handle(any(RMAppEvent.class));
   }
 
   /**
@@ -450,7 +446,7 @@ public class TestRMAppAttemptTransitions {
     assertEquals(0, applicationAttempt.getRanNodes().size());
     
     // Check events
-    verify(application, times(1)).handle(any(RMAppFailedAttemptEvent.class));
+    verify(application, times(2)).handle(any(RMAppFailedAttemptEvent.class));
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
     verifyAttemptFinalStateSaved();
   }
@@ -533,7 +529,6 @@ public class TestRMAppAttemptTransitions {
     assertEquals(container, applicationAttempt.getMasterContainer());
     assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
-    assertFalse(transferStateFromPreviousAttempt);
   }
   
   
@@ -549,7 +544,7 @@ public class TestRMAppAttemptTransitions {
     applicationAttempt.handle(
         new RMAppAttemptEvent(
             applicationAttempt.getAppAttemptId(), 
-            RMAppAttemptEventType.ATTEMPT_ADDED));
+            RMAppAttemptEventType.APP_ACCEPTED));
     
     if(unmanagedAM){
       assertEquals(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING, 
@@ -663,7 +658,6 @@ public class TestRMAppAttemptTransitions {
         diagnostics));
     testAppAttemptFinishedState(null, finalStatus, url, diagnostics, 1,
         true);
-    assertFalse(transferStateFromPreviousAttempt);
   }
 
   private void sendAttemptUpdateSavedEvent(RMAppAttempt applicationAttempt) {
@@ -691,21 +685,6 @@ public class TestRMAppAttemptTransitions {
         "Unmanaged AM must register after AM attempt reaches LAUNCHED state.");
   }
 
-  @Test
-  public void testUnmanagedAMContainersCleanup() {
-    unmanagedAM = true;
-    when(submissionContext.getUnmanagedAM()).thenReturn(true);
-    when(submissionContext.getKeepContainersAcrossApplicationAttempts())
-      .thenReturn(true);
-    // submit AM and check it goes to SUBMITTED state
-    submitApplicationAttempt();
-    // launch AM and verify attempt failed
-    applicationAttempt.handle(new RMAppAttemptRegistrationEvent(
-      applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl"));
-    sendAttemptUpdateSavedEvent(applicationAttempt);
-    assertFalse(transferStateFromPreviousAttempt);
-  }
-
   @Test
   public void testNewToKilled() {
     applicationAttempt.handle(
@@ -724,6 +703,16 @@ public class TestRMAppAttemptTransitions {
             RMAppAttemptEventType.RECOVER));
     testAppAttemptRecoveredState();
   }
+  
+  @Test
+  public void testSubmittedToFailed() {
+    submitApplicationAttempt();
+    String message = "Rejected";
+    applicationAttempt.handle(
+        new RMAppAttemptRejectedEvent(
+            applicationAttempt.getAppAttemptId(), message));
+    testAppAttemptSubmittedToFailedState(message);
+  }
 
   @Test
   public void testSubmittedToKilled() {
@@ -1117,64 +1106,6 @@ public class TestRMAppAttemptTransitions {
     Assert.assertNull(token);
   }
 
-  @Test
-  public void testFailedToFailed() {
-    // create a failed attempt.
-    when(submissionContext.getKeepContainersAcrossApplicationAttempts())
-      .thenReturn(true);
-    Container amContainer = allocateApplicationAttempt();
-    launchApplicationAttempt(amContainer);
-    runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
-    ContainerStatus cs1 =
-        ContainerStatus.newInstance(amContainer.getId(),
-          ContainerState.COMPLETE, "some error", 123);
-    ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
-    applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-      appAttemptId, cs1));
-    sendAttemptUpdateSavedEvent(applicationAttempt);
-    assertEquals(RMAppAttemptState.FAILED,
-      applicationAttempt.getAppAttemptState());
-    // should not kill containers when attempt fails.
-    assertTrue(transferStateFromPreviousAttempt);
-
-    // failed attempt captured the container finished event.
-    assertEquals(0, applicationAttempt.getJustFinishedContainers().size());
-    ContainerStatus cs2 =
-        ContainerStatus.newInstance(ContainerId.newInstance(appAttemptId, 2),
-          ContainerState.COMPLETE, "", 0);
-    applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-      appAttemptId, cs2));
-    assertEquals(1, applicationAttempt.getJustFinishedContainers().size());
-    assertEquals(cs2.getContainerId(), applicationAttempt
-      .getJustFinishedContainers().get(0).getContainerId());
-  }
-
-
-  @Test
-  public void testContainersCleanupForLastAttempt() {
-    // create a failed attempt.
-    applicationAttempt =
-        new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), rmContext,
-          scheduler, masterService, submissionContext, new Configuration(),
-          true);
-    when(submissionContext.getKeepContainersAcrossApplicationAttempts())
-      .thenReturn(true);
-    when(submissionContext.getMaxAppAttempts()).thenReturn(1);
-    Container amContainer = allocateApplicationAttempt();
-    launchApplicationAttempt(amContainer);
-    runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
-    ContainerStatus cs1 =
-        ContainerStatus.newInstance(amContainer.getId(),
-          ContainerState.COMPLETE, "some error", 123);
-    ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
-    applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-      appAttemptId, cs1));
-    sendAttemptUpdateSavedEvent(applicationAttempt);
-    assertEquals(RMAppAttemptState.FAILED,
-      applicationAttempt.getAppAttemptState());
-    assertFalse(transferStateFromPreviousAttempt);
-  }
-
   private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) {
     verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId);
     if (UserGroupInformation.isSecurityEnabled()) {

+ 24 - 58
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -66,10 +66,8 @@ public class TestQueueMetrics {
     MetricsSource queueSource= queueSource(ms, queueName);
     AppSchedulingInfo app = mockApp(user);
 
-    metrics.submitApp(user);
+    metrics.submitApp(user, 1);
     MetricsSource userSource = userSource(ms, queueName, user);
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
-    metrics.submitAppAttempt(user);
     checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
 
     metrics.setAvailableResourcesToQueue(Resources.createResource(100*GB, 100));
@@ -78,7 +76,7 @@ public class TestQueueMetrics {
     // configurable cluster/queue resources
     checkResources(queueSource, 0, 0, 0, 0, 0, 100*GB, 100, 15*GB, 15, 5, 0, 0, 0);
 
-    metrics.runAppAttempt(app.getApplicationId(), user);
+    metrics.incrAppsRunning(app, user);
     checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
 
     metrics.allocateResources(user, 3, Resources.createResource(2*GB, 2));
@@ -87,10 +85,7 @@ public class TestQueueMetrics {
     metrics.releaseResources(user, 1, Resources.createResource(2*GB, 2));
     checkResources(queueSource, 4*GB, 4, 2, 3, 1, 100*GB, 100, 9*GB, 9, 2, 0, 0, 0);
 
-    metrics.finishAppAttempt(
-        app.getApplicationId(), app.isPending(), app.getUser());
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
-    metrics.finishApp(user, RMAppState.FINISHED);
+    metrics.finishApp(app, RMAppAttemptState.FINISHED);
     checkApps(queueSource, 1, 0, 0, 1, 0, 0, true);
     assertNull(userSource);
   }
@@ -105,47 +100,39 @@ public class TestQueueMetrics {
     MetricsSource queueSource = queueSource(ms, queueName);
     AppSchedulingInfo app = mockApp(user);
 
-    metrics.submitApp(user);
+    metrics.submitApp(user, 1);
     MetricsSource userSource = userSource(ms, queueName, user);
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
-    metrics.submitAppAttempt(user);
     checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
 
-    metrics.runAppAttempt(app.getApplicationId(), user);
+    metrics.incrAppsRunning(app, user);
     checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
 
-    metrics.finishAppAttempt(
-        app.getApplicationId(), app.isPending(), app.getUser());
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+    metrics.finishApp(app, RMAppAttemptState.FAILED);
+    checkApps(queueSource, 1, 0, 0, 0, 1, 0, true);
 
     // As the application has failed, framework retries the same application
     // based on configuration
-    metrics.submitAppAttempt(user);
+    metrics.submitApp(user, 2);
     checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
 
-    metrics.runAppAttempt(app.getApplicationId(), user);
+    metrics.incrAppsRunning(app, user);
     checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
 
     // Suppose say application has failed this time as well.
-    metrics.finishAppAttempt(
-        app.getApplicationId(), app.isPending(), app.getUser());
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
+    metrics.finishApp(app, RMAppAttemptState.FAILED);
+    checkApps(queueSource, 1, 0, 0, 0, 1, 0, true);
 
     // As the application has failed, framework retries the same application
     // based on configuration
-    metrics.submitAppAttempt(user);
+    metrics.submitApp(user, 3);
     checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
 
-    metrics.runAppAttempt(app.getApplicationId(), user);
+    metrics.incrAppsRunning(app, user);
     checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
 
-    // Suppose say application has failed, and there's no more retries.
-    metrics.finishAppAttempt(
-        app.getApplicationId(), app.isPending(), app.getUser());
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
-
-    metrics.finishApp(user, RMAppState.FAILED);
-    checkApps(queueSource, 1, 0, 0, 0, 1, 0, true);
+    // Suppose say application has finished.
+    metrics.finishApp(app, RMAppAttemptState.FINISHED);
+    checkApps(queueSource, 1, 0, 0, 1, 0, 0, true);
 
     assertNull(userSource);
   }
@@ -159,13 +146,9 @@ public class TestQueueMetrics {
     MetricsSource queueSource = queueSource(ms, queueName);
     AppSchedulingInfo app = mockApp(user);
 
-    metrics.submitApp(user);
+    metrics.submitApp(user, 1);
     MetricsSource userSource = userSource(ms, queueName, user);
 
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
-    checkApps(userSource, 1, 0, 0, 0, 0, 0, true);
-
-    metrics.submitAppAttempt(user);
     checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
     checkApps(userSource, 1, 1, 0, 0, 0, 0, true);
 
@@ -177,7 +160,7 @@ public class TestQueueMetrics {
     checkResources(queueSource, 0, 0, 0, 0, 0,  100*GB, 100, 15*GB, 15, 5, 0, 0, 0);
     checkResources(userSource, 0, 0, 0, 0, 0, 10*GB, 10, 15*GB, 15, 5, 0, 0, 0);
 
-    metrics.runAppAttempt(app.getApplicationId(), user);
+    metrics.incrAppsRunning(app, user);
     checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
     checkApps(userSource, 1, 0, 1, 0, 0, 0, true);
 
@@ -189,11 +172,7 @@ public class TestQueueMetrics {
     checkResources(queueSource, 4*GB, 4, 2, 3, 1, 100*GB, 100, 9*GB, 9, 2, 0, 0, 0);
     checkResources(userSource, 4*GB, 4, 2, 3, 1, 10*GB, 10, 9*GB, 9, 2, 0, 0, 0);
 
-    metrics.finishAppAttempt(
-        app.getApplicationId(), app.isPending(), app.getUser());
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
-    checkApps(userSource, 1, 0, 0, 0, 0, 0, true);
-    metrics.finishApp(user, RMAppState.FINISHED);
+    metrics.finishApp(app, RMAppAttemptState.FINISHED);
     checkApps(queueSource, 1, 0, 0, 1, 0, 0, true);
     checkApps(userSource, 1, 0, 0, 1, 0, 0, true);
   }
@@ -213,16 +192,10 @@ public class TestQueueMetrics {
     MetricsSource queueSource = queueSource(ms, leafQueueName);
     AppSchedulingInfo app = mockApp(user);
 
-    metrics.submitApp(user);
+    metrics.submitApp(user, 1);
     MetricsSource userSource = userSource(ms, leafQueueName, user);
     MetricsSource parentUserSource = userSource(ms, parentQueueName, user);
 
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
-    checkApps(parentQueueSource, 1, 0, 0, 0, 0, 0, true);
-    checkApps(userSource, 1, 0, 0, 0, 0, 0, true);
-    checkApps(parentUserSource, 1, 0, 0, 0, 0, 0, true);
-
-    metrics.submitAppAttempt(user);
     checkApps(queueSource, 1, 1, 0, 0, 0, 0, true);
     checkApps(parentQueueSource, 1, 1, 0, 0, 0, 0, true);
     checkApps(userSource, 1, 1, 0, 0, 0, 0, true);
@@ -238,7 +211,7 @@ public class TestQueueMetrics {
     checkResources(userSource, 0, 0, 0, 0, 0, 10*GB, 10, 15*GB, 15, 5, 0, 0, 0);
     checkResources(parentUserSource, 0, 0, 0, 0, 0, 10*GB, 10, 15*GB, 15, 5, 0, 0, 0);
 
-    metrics.runAppAttempt(app.getApplicationId(), user);
+    metrics.incrAppsRunning(app, user);
     checkApps(queueSource, 1, 0, 1, 0, 0, 0, true);
     checkApps(userSource, 1, 0, 1, 0, 0, 0, true);
 
@@ -258,14 +231,7 @@ public class TestQueueMetrics {
     checkResources(userSource, 4*GB, 4, 2, 3, 1, 10*GB, 10, 9*GB, 9, 2, 0, 0, 0);
     checkResources(parentUserSource, 4*GB, 4, 2, 3, 1, 10*GB, 10, 9*GB, 9, 2, 0, 0, 0);
 
-    metrics.finishAppAttempt(
-        app.getApplicationId(), app.isPending(), app.getUser());
-    checkApps(queueSource, 1, 0, 0, 0, 0, 0, true);
-    checkApps(parentQueueSource, 1, 0, 0, 0, 0, 0, true);
-    checkApps(userSource, 1, 0, 0, 0, 0, 0, true);
-    checkApps(parentUserSource, 1, 0, 0, 0, 0, 0, true);
-
-    metrics.finishApp(user, RMAppState.FINISHED);
+    metrics.finishApp(app, RMAppAttemptState.FINISHED);
     checkApps(queueSource, 1, 0, 0, 1, 0, 0, true);
     checkApps(parentQueueSource, 1, 0, 0, 1, 0, 0, true);
     checkApps(userSource, 1, 0, 0, 1, 0, 0, true);
@@ -342,7 +308,7 @@ public class TestQueueMetrics {
     assertGauge("AppsPending", pending, rb);
     assertGauge("AppsRunning", running, rb);
     assertCounter("AppsCompleted", completed, rb);
-    assertCounter("AppsFailed", failed, rb);
+    assertGauge("AppsFailed", failed, rb);
     assertCounter("AppsKilled", killed, rb);
   }
 

+ 0 - 25
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java

@@ -51,7 +51,6 @@ import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -59,12 +58,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
 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.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@@ -383,24 +378,4 @@ public class TestSchedulerUtils {
           ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
     Assert.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus());
   }
-
-  public static <T> SchedulerApplication verifyAppAddedAndRemovedFromScheduler(
-      final Map<ApplicationId, SchedulerApplication> applications,
-      EventHandler<SchedulerEvent> handler, String queueName) throws Exception {
-    ApplicationId appId =
-        ApplicationId.newInstance(System.currentTimeMillis(), 1);
-    AppAddedSchedulerEvent appAddedEvent =
-        new AppAddedSchedulerEvent(appId, queueName, "user");
-    handler.handle(appAddedEvent);
-    SchedulerApplication app = applications.get(appId);
-    // verify application is added.
-    Assert.assertNotNull(app);
-    Assert.assertEquals("user", app.getUser());
-
-    AppRemovedSchedulerEvent appRemoveEvent =
-        new AppRemovedSchedulerEvent(appId, RMAppState.FINISHED);
-    handler.handle(appRemoveEvent);
-    Assert.assertNull(applications.get(appId));
-    return app;
-  }
 }

+ 19 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java

@@ -304,7 +304,7 @@ public class TestApplicationLimits {
     int APPLICATION_ID = 0;
     // Submit first application
     FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0);
-    queue.submitApplicationAttempt(app_0, user_0);
+    queue.submitApplication(app_0, user_0, A);
     assertEquals(1, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
     assertEquals(1, queue.getNumActiveApplications(user_0));
@@ -312,7 +312,7 @@ public class TestApplicationLimits {
 
     // Submit second application
     FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0);
-    queue.submitApplicationAttempt(app_1, user_0);
+    queue.submitApplication(app_1, user_0, A);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -320,14 +320,14 @@ public class TestApplicationLimits {
     
     // Submit third application, should remain pending
     FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0);
-    queue.submitApplicationAttempt(app_2, user_0);
+    queue.submitApplication(app_2, user_0, A);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
     assertEquals(1, queue.getNumPendingApplications(user_0));
     
     // Finish one application, app_2 should be activated
-    queue.finishApplicationAttempt(app_0, A);
+    queue.finishApplication(app_0, A);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -335,7 +335,7 @@ public class TestApplicationLimits {
     
     // Submit another one for user_0
     FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0);
-    queue.submitApplicationAttempt(app_3, user_0);
+    queue.submitApplication(app_3, user_0, A);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -346,7 +346,7 @@ public class TestApplicationLimits {
     
     // Submit first app for user_1
     FiCaSchedulerApp app_4 = getMockApplication(APPLICATION_ID++, user_1);
-    queue.submitApplicationAttempt(app_4, user_1);
+    queue.submitApplication(app_4, user_1, A);
     assertEquals(3, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -356,7 +356,7 @@ public class TestApplicationLimits {
 
     // Submit second app for user_1, should block due to queue-limit
     FiCaSchedulerApp app_5 = getMockApplication(APPLICATION_ID++, user_1);
-    queue.submitApplicationAttempt(app_5, user_1);
+    queue.submitApplication(app_5, user_1, A);
     assertEquals(3, queue.getNumActiveApplications());
     assertEquals(2, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -365,7 +365,7 @@ public class TestApplicationLimits {
     assertEquals(1, queue.getNumPendingApplications(user_1));
 
     // Now finish one app of user_1 so app_5 should be activated
-    queue.finishApplicationAttempt(app_4, A);
+    queue.finishApplication(app_4, A);
     assertEquals(3, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -385,7 +385,7 @@ public class TestApplicationLimits {
 
     // Submit first application
     FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0);
-    queue.submitApplicationAttempt(app_0, user_0);
+    queue.submitApplication(app_0, user_0, A);
     assertEquals(1, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
     assertEquals(1, queue.getNumActiveApplications(user_0));
@@ -394,7 +394,7 @@ public class TestApplicationLimits {
 
     // Submit second application
     FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0);
-    queue.submitApplicationAttempt(app_1, user_0);
+    queue.submitApplication(app_1, user_0, A);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -403,7 +403,7 @@ public class TestApplicationLimits {
 
     // Submit third application, should remain pending
     FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0);
-    queue.submitApplicationAttempt(app_2, user_0);
+    queue.submitApplication(app_2, user_0, A);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -412,7 +412,7 @@ public class TestApplicationLimits {
 
     // Submit fourth application, should remain pending
     FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0);
-    queue.submitApplicationAttempt(app_3, user_0);
+    queue.submitApplication(app_3, user_0, A);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(2, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -420,7 +420,7 @@ public class TestApplicationLimits {
     assertTrue(queue.pendingApplications.contains(app_3));
 
     // Kill 3rd pending application
-    queue.finishApplicationAttempt(app_2, A);
+    queue.finishApplication(app_2, A);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -429,7 +429,7 @@ public class TestApplicationLimits {
     assertFalse(queue.activeApplications.contains(app_2));
 
     // Finish 1st application, app_3 should become active
-    queue.finishApplicationAttempt(app_0, A);
+    queue.finishApplication(app_0, A);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
@@ -439,7 +439,7 @@ public class TestApplicationLimits {
     assertFalse(queue.activeApplications.contains(app_0));
 
     // Finish 2nd application
-    queue.finishApplicationAttempt(app_1, A);
+    queue.finishApplication(app_1, A);
     assertEquals(1, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
     assertEquals(1, queue.getNumActiveApplications(user_0));
@@ -447,7 +447,7 @@ public class TestApplicationLimits {
     assertFalse(queue.activeApplications.contains(app_1));
 
     // Finish 4th application
-    queue.finishApplicationAttempt(app_3, A);
+    queue.finishApplication(app_3, A);
     assertEquals(0, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
     assertEquals(0, queue.getNumActiveApplications(user_0));
@@ -507,7 +507,7 @@ public class TestApplicationLimits {
     FiCaSchedulerApp app_0_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_0_0, user_0, queue, 
             queue.getActiveUsersManager(), rmContext));
-    queue.submitApplicationAttempt(app_0_0, user_0);
+    queue.submitApplication(app_0_0, user_0, A);
 
     List<ResourceRequest> app_0_0_requests = new ArrayList<ResourceRequest>();
     app_0_0_requests.add(
@@ -526,7 +526,7 @@ public class TestApplicationLimits {
     FiCaSchedulerApp app_0_1 = 
         spy(new FiCaSchedulerApp(appAttemptId_0_1, user_0, queue, 
             queue.getActiveUsersManager(), rmContext));
-    queue.submitApplicationAttempt(app_0_1, user_0);
+    queue.submitApplication(app_0_1, user_0, A);
     
     List<ResourceRequest> app_0_1_requests = new ArrayList<ResourceRequest>();
     app_0_1_requests.add(
@@ -545,7 +545,7 @@ public class TestApplicationLimits {
     FiCaSchedulerApp app_1_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_1_0, user_1, queue, 
             queue.getActiveUsersManager(), rmContext));
-    queue.submitApplicationAttempt(app_1_0, user_1);
+    queue.submitApplication(app_1_0, user_1, A);
 
     List<ResourceRequest> app_1_0_requests = new ArrayList<ResourceRequest>();
     app_1_0_requests.add(

+ 66 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -64,10 +64,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@@ -558,22 +555,19 @@ public class TestCapacityScheduler {
     ApplicationId appId = BuilderUtils.newApplicationId(100, 1);
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
         appId, 1);
-    SchedulerEvent addAppEvent =
-        new AppAddedSchedulerEvent(appId, "default", "user");
-    cs.handle(addAppEvent);
-    SchedulerEvent addAttemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    cs.handle(addAttemptEvent);
+    SchedulerEvent event =
+        new AppAttemptAddedSchedulerEvent(appAttemptId, "default", "user");
+    cs.handle(event);
 
     // Verify the blacklist can be updated independent of requesting containers
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null);
-    Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertTrue(cs.getApplication(appAttemptId).isBlacklisted(host));
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host));
-    Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertFalse(cs.getApplication(appAttemptId).isBlacklisted(host));
     rm.stop();
   }
 
@@ -597,6 +591,66 @@ public class TestCapacityScheduler {
       assertTrue(appComparator.compare(app1, app3) < 0);
       assertTrue(appComparator.compare(app2, app3) < 0);
     }
+
+    @Test
+    public void testConcurrentAccessOnApplications() throws Exception {
+      CapacityScheduler cs = new CapacityScheduler();
+      verifyConcurrentAccessOnApplications(
+          cs.applications, FiCaSchedulerApp.class, Queue.class);
+    }
+
+    public static <T extends SchedulerApplication, Q extends Queue>
+        void verifyConcurrentAccessOnApplications(
+            final Map<ApplicationAttemptId, T> applications, Class<T> appClazz,
+            final Class<Q> queueClazz)
+                throws Exception {
+      final int size = 10000;
+      final ApplicationId appId = ApplicationId.newInstance(0, 0);
+      final Constructor<T> ctor = appClazz.getDeclaredConstructor(
+          ApplicationAttemptId.class, String.class, queueClazz,
+          ActiveUsersManager.class, RMContext.class);
+
+      ApplicationAttemptId appAttemptId0
+          = ApplicationAttemptId.newInstance(appId, 0);
+      applications.put(appAttemptId0, ctor.newInstance(
+              appAttemptId0, null, mock(queueClazz), null, null));
+      assertNotNull(applications.get(appAttemptId0));
+
+      // Imitating the thread of scheduler that will add and remove apps
+      final AtomicBoolean finished = new AtomicBoolean(false);
+      final AtomicBoolean failed = new AtomicBoolean(false);
+      Thread t = new Thread() {
+
+        @Override
+        public void run() {
+          for (int i = 1; i <= size; ++i) {
+            ApplicationAttemptId appAttemptId
+                = ApplicationAttemptId.newInstance(appId, i);
+            try {
+              applications.put(appAttemptId, ctor.newInstance(
+                  appAttemptId, null, mock(queueClazz), null, null));
+            } catch (Exception e) {
+              failed.set(true);
+              finished.set(true);
+              return;
+            }
+          }
+          for (int i = 1; i <= size; ++i) {
+            ApplicationAttemptId appAttemptId
+                = ApplicationAttemptId.newInstance(appId, i);
+            applications.remove(appAttemptId);
+          }
+          finished.set(true);
+        }
+      };
+      t.start();
+
+      // Imitating the thread of rmappattempt that will get the app
+      while (!finished.get()) {
+        assertNotNull(applications.get(appAttemptId0));
+      }
+      assertFalse(failed.get());
+    }
     
     @Test
     public void testGetAppsInQueue() throws Exception {
@@ -628,21 +682,4 @@ public class TestCapacityScheduler {
       Assert.assertNull(scheduler.getAppsInQueue("nonexistentqueue"));
     }
 
-  @Test
-  public void testAddAndRemoveAppFromCapacityScheduler() throws Exception {
-
-    AsyncDispatcher rmDispatcher = new AsyncDispatcher();
-    CapacityScheduler cs = new CapacityScheduler();
-    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
-    setupQueueConfiguration(conf);
-    cs.reinitialize(conf, new RMContextImpl(rmDispatcher, null, null, null,
-      null, null, new RMContainerTokenSecretManager(conf),
-      new NMTokenSecretManagerInRM(conf),
-      new ClientToAMTokenSecretManagerInRM()));
-
-    SchedulerApplication app =
-        TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(
-          cs.getSchedulerApplications(), cs, "a1");
-    Assert.assertEquals("a1", app.getQueue().getQueueName());
-  }
- }
+}

+ 41 - 52
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java

@@ -55,7 +55,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@@ -64,10 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -275,14 +271,14 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, B);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_1, user_0);  // same user
+    a.submitApplication(app_1, user_0, B);  // same user
 
     
     // Setup some nodes
@@ -324,14 +320,14 @@ public class TestLeafQueue {
         .getMockApplicationAttemptId(0, 1);
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_d, d, null,
         rmContext);
-    d.submitApplicationAttempt(app_0, user_d);
+    d.submitApplication(app_0, user_d, D);
 
     // Attempt the same application again
     final ApplicationAttemptId appAttemptId_1 = TestUtils
         .getMockApplicationAttemptId(0, 2);
     FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_d, d, null,
         rmContext);
-    d.submitApplicationAttempt(app_1, user_d); // same user
+    d.submitApplication(app_1, user_d, D); // same user
   }
 
 
@@ -349,37 +345,30 @@ public class TestLeafQueue {
         .getMockApplicationAttemptId(0, 1);
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a, null,
         rmContext);
-    AppAddedSchedulerEvent addAppEvent =
-        new AppAddedSchedulerEvent(appAttemptId_0.getApplicationId(),
-          a.getQueueName(), user_0);
-    cs.handle(addAppEvent);
-    AppAttemptAddedSchedulerEvent addAttemptEvent = 
-        new AppAttemptAddedSchedulerEvent(appAttemptId_0, false);
-    cs.handle(addAttemptEvent);
-
+    a.submitApplication(app_0, user_0, B);
+    
+    when(cs.getApplication(appAttemptId_0)).thenReturn(app_0);
     AppAttemptRemovedSchedulerEvent event = new AppAttemptRemovedSchedulerEvent(
-        appAttemptId_0, RMAppAttemptState.FAILED, false);
+        appAttemptId_0, RMAppAttemptState.FAILED);
     cs.handle(event);
     
     assertEquals(0, a.getMetrics().getAppsPending());
-    assertEquals(0, a.getMetrics().getAppsFailed());
+    assertEquals(1, a.getMetrics().getAppsFailed());
 
     // Attempt the same application again
     final ApplicationAttemptId appAttemptId_1 = TestUtils
         .getMockApplicationAttemptId(0, 2);
     FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a, null,
         rmContext);
-    a.submitApplicationAttempt(app_1, user_0); // same user
+    a.submitApplication(app_1, user_0, B); // same user
 
     assertEquals(1, a.getMetrics().getAppsSubmitted());
     assertEquals(1, a.getMetrics().getAppsPending());
     
+    when(cs.getApplication(appAttemptId_1)).thenReturn(app_0);
     event = new AppAttemptRemovedSchedulerEvent(appAttemptId_0,
-        RMAppAttemptState.FINISHED, false);
+        RMAppAttemptState.FINISHED);
     cs.handle(event);
-    AppRemovedSchedulerEvent rEvent = new AppRemovedSchedulerEvent(
-        appAttemptId_0.getApplicationId(), RMAppState.FINISHED);
-    cs.handle(rEvent);
     
     assertEquals(1, a.getMetrics().getAppsSubmitted());
     assertEquals(0, a.getMetrics().getAppsPending());
@@ -407,14 +396,14 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_1, user_0);  // same user
+    a.submitApplication(app_1, user_0, A);  // same user
 
     
     // Setup some nodes
@@ -535,21 +524,21 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_1, user_0);  // same user
+    a.submitApplication(app_1, user_0, A);  // same user
 
     final ApplicationAttemptId appAttemptId_2 = 
         TestUtils.getMockApplicationAttemptId(2, 0); 
     FiCaSchedulerApp app_2 = 
         new FiCaSchedulerApp(appAttemptId_2, user_1, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_2, user_1);
+    a.submitApplication(app_2, user_1, A);
 
     // Setup some nodes
     String host_0 = "127.0.0.1";
@@ -629,21 +618,21 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_1, user_0);  // same user
+    a.submitApplication(app_1, user_0, A);  // same user
 
     final ApplicationAttemptId appAttemptId_2 = 
         TestUtils.getMockApplicationAttemptId(2, 0); 
     FiCaSchedulerApp app_2 = 
         new FiCaSchedulerApp(appAttemptId_2, user_1, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_2, user_1);
+    a.submitApplication(app_2, user_1, A);
 
     // Setup some nodes
     String host_0 = "127.0.0.1";
@@ -740,28 +729,28 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_1, user_0);  // same user
+    a.submitApplication(app_1, user_0, A);  // same user
 
     final ApplicationAttemptId appAttemptId_2 = 
         TestUtils.getMockApplicationAttemptId(2, 0); 
     FiCaSchedulerApp app_2 = 
         new FiCaSchedulerApp(appAttemptId_2, user_1, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_2, user_1);
+    a.submitApplication(app_2, user_1, A);
 
     final ApplicationAttemptId appAttemptId_3 = 
         TestUtils.getMockApplicationAttemptId(3, 0); 
     FiCaSchedulerApp app_3 = 
         new FiCaSchedulerApp(appAttemptId_3, user_2, a, 
             a.getActiveUsersManager(), rmContext);
-    a.submitApplicationAttempt(app_3, user_2);
+    a.submitApplication(app_3, user_2, A);
     
     // Setup some nodes
     String host_0 = "127.0.0.1";
@@ -916,14 +905,14 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_1, a, 
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_1, user_1);  
+    a.submitApplication(app_1, user_1, A);  
 
     // Setup some nodes
     String host_0 = "127.0.0.1";
@@ -1018,14 +1007,14 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 =
         new FiCaSchedulerApp(appAttemptId_0, user_0, a,
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
 
     final ApplicationAttemptId appAttemptId_1 =
         TestUtils.getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 =
         new FiCaSchedulerApp(appAttemptId_1, user_1, a,
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_1, user_1);
+    a.submitApplication(app_1, user_1, A);
 
     // Setup some nodes
     String host_0 = "127.0.0.1";
@@ -1122,14 +1111,14 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_1, a, 
             mock(ActiveUsersManager.class), rmContext);
-    a.submitApplicationAttempt(app_1, user_1);  
+    a.submitApplication(app_1, user_1, A);  
 
     // Setup some nodes
     String host_0 = "127.0.0.1";
@@ -1243,7 +1232,7 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             mock(ActiveUsersManager.class), rmContext));
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
     
     // Setup some nodes and racks
     String host_0 = "127.0.0.1";
@@ -1384,7 +1373,7 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             mock(ActiveUsersManager.class), rmContext));
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
     
     // Setup some nodes and racks
     String host_0 = "127.0.0.1";
@@ -1515,7 +1504,7 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             mock(ActiveUsersManager.class), rmContext));
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
     
     // Setup some nodes and racks
     String host_0_0 = "127.0.0.1";
@@ -1618,21 +1607,21 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 =
         new FiCaSchedulerApp(appAttemptId_0, user_e, e,
             mock(ActiveUsersManager.class), rmContext);
-    e.submitApplicationAttempt(app_0, user_e);
+    e.submitApplication(app_0, user_e, E);
 
     final ApplicationAttemptId appAttemptId_1 =
         TestUtils.getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 =
         new FiCaSchedulerApp(appAttemptId_1, user_e, e,
             mock(ActiveUsersManager.class), rmContext);
-    e.submitApplicationAttempt(app_1, user_e);  // same user
+    e.submitApplication(app_1, user_e, E);  // same user
 
     final ApplicationAttemptId appAttemptId_2 =
         TestUtils.getMockApplicationAttemptId(2, 0);
     FiCaSchedulerApp app_2 =
         new FiCaSchedulerApp(appAttemptId_2, user_e, e,
             mock(ActiveUsersManager.class), rmContext);
-    e.submitApplicationAttempt(app_2, user_e);  // same user
+    e.submitApplication(app_2, user_e, E);  // same user
 
     // before reinitialization
     assertEquals(2, e.activeApplications.size());
@@ -1696,21 +1685,21 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 =
         new FiCaSchedulerApp(appAttemptId_0, user_e, e,
             mock(ActiveUsersManager.class), rmContext);
-    e.submitApplicationAttempt(app_0, user_e);
+    e.submitApplication(app_0, user_e, E);
 
     final ApplicationAttemptId appAttemptId_1 =
         TestUtils.getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 =
         new FiCaSchedulerApp(appAttemptId_1, user_e, e,
             mock(ActiveUsersManager.class), rmContext);
-    e.submitApplicationAttempt(app_1, user_e);  // same user
+    e.submitApplication(app_1, user_e, E);  // same user
 
     final ApplicationAttemptId appAttemptId_2 =
         TestUtils.getMockApplicationAttemptId(2, 0);
     FiCaSchedulerApp app_2 =
         new FiCaSchedulerApp(appAttemptId_2, user_e, e,
             mock(ActiveUsersManager.class), rmContext);
-    e.submitApplicationAttempt(app_2, user_e);  // same user
+    e.submitApplication(app_2, user_e, E);  // same user
 
     // before updating cluster resource
     assertEquals(2, e.activeApplications.size());
@@ -1773,14 +1762,14 @@ public class TestLeafQueue {
     FiCaSchedulerApp app_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
             mock(ActiveUsersManager.class), rmContext));
-    a.submitApplicationAttempt(app_0, user_0);
+    a.submitApplication(app_0, user_0, A);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         spy(new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
             mock(ActiveUsersManager.class), rmContext));
-    a.submitApplicationAttempt(app_1, user_0);
+    a.submitApplication(app_1, user_0, A);
 
     // Setup some nodes and racks
     String host_0_0 = "127.0.0.1";

+ 112 - 132
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -78,8 +78,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
@@ -256,12 +255,7 @@ public class TestFairScheduler {
   private ApplicationAttemptId createSchedulingRequest(int memory, int vcores,
       String queueId, String userId, int numContainers, int priority) {
     ApplicationAttemptId id = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
-    scheduler.addApplication(id.getApplicationId(), queueId, userId);
-    // This conditional is for testAclSubmitApplication where app is rejected
-    // and no app is added.
-    if (scheduler.getSchedulerApplications().containsKey(id.getApplicationId())) {
-      scheduler.addApplicationAttempt(id, false);
-    }
+    scheduler.addApplicationAttempt(id, queueId, userId);
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     ResourceRequest request = createResourceRequest(memory, vcores, ResourceRequest.ANY,
         priority, numContainers, true);
@@ -589,7 +583,7 @@ public class TestFairScheduler {
     // Make sure queue 2 is waiting with a reservation
     assertEquals(0, scheduler.getQueueManager().getQueue("queue2").
       getResourceUsage().getMemory());
-    assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
+    assertEquals(1024, scheduler.applications.get(attId).getCurrentReservation().getMemory());
 
     // Now another node checks in with capacity
     RMNode node2 =
@@ -605,10 +599,10 @@ public class TestFairScheduler {
         getResourceUsage().getMemory());
 
     // The old reservation should still be there...
-    assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
+    assertEquals(1024, scheduler.applications.get(attId).getCurrentReservation().getMemory());
     // ... but it should disappear when we update the first node.
     scheduler.handle(updateEvent);
-    assertEquals(0, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
+    assertEquals(0, scheduler.applications.get(attId).getCurrentReservation().getMemory());
 
   }
 
@@ -624,13 +618,9 @@ public class TestFairScheduler {
             null, null, null, false, false, 0, null, null), null, null, 0, null);
     appsMap.put(appAttemptId.getApplicationId(), rmApp);
     
-    AppAddedSchedulerEvent appAddedEvent =
-        new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "default",
-          "user1");
+    AppAttemptAddedSchedulerEvent appAddedEvent =
+        new AppAttemptAddedSchedulerEvent(appAttemptId, "default", "user1");
     scheduler.handle(appAddedEvent);
-    AppAttemptAddedSchedulerEvent attempAddedEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    scheduler.handle(attempAddedEvent);
     assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
         .getRunnableAppSchedulables().size());
     assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true)
@@ -649,14 +639,10 @@ public class TestFairScheduler {
         null, null, null, ApplicationSubmissionContext.newInstance(null, null,
             null, null, null, false, false, 0, null, null), null, null, 0, null);
     appsMap.put(appAttemptId.getApplicationId(), rmApp);
-
-    AppAddedSchedulerEvent appAddedEvent =
-        new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "default",
-          "user2");
-    scheduler.handle(appAddedEvent);
-    AppAttemptAddedSchedulerEvent attempAddedEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    scheduler.handle(attempAddedEvent);
+    
+    AppAttemptAddedSchedulerEvent appAddedEvent2 =
+        new AppAttemptAddedSchedulerEvent(appAttemptId, "default", "user2");
+    scheduler.handle(appAddedEvent2);
     assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true)
         .getRunnableAppSchedulables().size());
     assertEquals(1, scheduler.getQueueManager().getLeafQueue("default", true)
@@ -674,8 +660,8 @@ public class TestFairScheduler {
 
     // submit app with empty queue
     ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1);
-    AppAddedSchedulerEvent appAddedEvent =
-        new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "", "user1");
+    AppAttemptAddedSchedulerEvent appAddedEvent =
+        new AppAttemptAddedSchedulerEvent(appAttemptId, "", "user1");
     scheduler.handle(appAddedEvent);
 
     // submission rejected
@@ -709,6 +695,7 @@ public class TestFairScheduler {
     scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     ApplicationAttemptId appId;
+    Map<ApplicationAttemptId, FSSchedulerApp> apps = scheduler.applications;
 
     List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
     rules.add(new QueuePlacementRule.Specified().initialize(true, null));
@@ -721,17 +708,17 @@ public class TestFairScheduler {
     scheduler.getAllocationConfiguration().placementPolicy =
         new QueuePlacementPolicy(rules, queues, conf);
     appId = createSchedulingRequest(1024, "somequeue", "user1");
-    assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
+    assertEquals("root.somequeue", apps.get(appId).getQueueName());
     appId = createSchedulingRequest(1024, "default", "user1");
-    assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
+    assertEquals("root.user1", apps.get(appId).getQueueName());
     appId = createSchedulingRequest(1024, "default", "user3");
-    assertEquals("root.user3group", scheduler.getSchedulerApp(appId).getQueueName());
+    assertEquals("root.user3group", apps.get(appId).getQueueName());
     appId = createSchedulingRequest(1024, "default", "user4");
-    assertEquals("root.user4subgroup1", scheduler.getSchedulerApp(appId).getQueueName());
+    assertEquals("root.user4subgroup1", apps.get(appId).getQueueName());
     appId = createSchedulingRequest(1024, "default", "user5");
-    assertEquals("root.user5subgroup2", scheduler.getSchedulerApp(appId).getQueueName());
+    assertEquals("root.user5subgroup2", apps.get(appId).getQueueName());
     appId = createSchedulingRequest(1024, "default", "otheruser");
-    assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
+    assertEquals("root.default", apps.get(appId).getQueueName());
     
     // test without specified as first rule
     rules = new ArrayList<QueuePlacementRule>();
@@ -741,11 +728,11 @@ public class TestFairScheduler {
     scheduler.getAllocationConfiguration().placementPolicy =
         new QueuePlacementPolicy(rules, queues, conf);
     appId = createSchedulingRequest(1024, "somequeue", "user1");
-    assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
+    assertEquals("root.user1", apps.get(appId).getQueueName());
     appId = createSchedulingRequest(1024, "somequeue", "otheruser");
-    assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
+    assertEquals("root.somequeue", apps.get(appId).getQueueName());
     appId = createSchedulingRequest(1024, "default", "otheruser");
-    assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
+    assertEquals("root.default", apps.get(appId).getQueueName());
   }
 
   @Test
@@ -799,14 +786,11 @@ public class TestFairScheduler {
     scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     ApplicationAttemptId id11 = createAppAttemptId(1, 1);
-    scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1");
-    scheduler.addApplicationAttempt(id11, false);
+    scheduler.addApplicationAttempt(id11, "root.queue1", "user1");
     ApplicationAttemptId id21 = createAppAttemptId(2, 1);
-    scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1");
-    scheduler.addApplicationAttempt(id21, false);
+    scheduler.addApplicationAttempt(id21, "root.queue2", "user1");
     ApplicationAttemptId id22 = createAppAttemptId(2, 2);
-    scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1");
-    scheduler.addApplicationAttempt(id22, false);
+    scheduler.addApplicationAttempt(id22, "root.queue2", "user1");
 
     int minReqSize = 
         FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
@@ -847,13 +831,11 @@ public class TestFairScheduler {
   @Test
   public void testAppAdditionAndRemoval() throws Exception {
     scheduler.reinitialize(conf, resourceManager.getRMContext());
-    ApplicationAttemptId attemptId =createAppAttemptId(1, 1);
-    AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent(attemptId.getApplicationId(), "default",
-      "user1");
-    scheduler.handle(appAddedEvent);
-    AppAttemptAddedSchedulerEvent attemptAddedEvent =
-        new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false);
-    scheduler.handle(attemptAddedEvent);
+
+    AppAttemptAddedSchedulerEvent appAddedEvent1 =
+        new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), "default",
+          "user1");
+    scheduler.handle(appAddedEvent1);
 
     // Scheduler should have two queues (the default and the one created for user1)
     assertEquals(2, scheduler.getQueueManager().getLeafQueues().size());
@@ -863,7 +845,7 @@ public class TestFairScheduler {
         .getRunnableAppSchedulables().size());
 
     AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent(
-        createAppAttemptId(1, 1), RMAppAttemptState.FINISHED, false);
+        createAppAttemptId(1, 1), RMAppAttemptState.FINISHED);
 
     // Now remove app
     scheduler.handle(appRemovedEvent1);
@@ -1136,12 +1118,12 @@ public class TestFairScheduler {
       scheduler.handle(nodeUpdate3);
     }
 
-    assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app3).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app5).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app6).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app2).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app3).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app5).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app6).getLiveContainers().size());
 
     // Now new requests arrive from queues C and D
     ApplicationAttemptId app7 =
@@ -1164,16 +1146,16 @@ public class TestFairScheduler {
     // Make sure it is lowest priority container.
     scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
         Resources.createResource(2 * 1024));
-    assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app5).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app2).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app5).getLiveContainers().size());
     
     // First verify we are adding containers to preemption list for the application
-    assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app3).getLiveContainers(),
-                                     scheduler.getSchedulerApp(app3).getPreemptionContainers()));
-    assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app6).getLiveContainers(),
-                                     scheduler.getSchedulerApp(app6).getPreemptionContainers()));
+    assertTrue(!Collections.disjoint(scheduler.applications.get(app3).getLiveContainers(),
+                                     scheduler.applications.get(app3).getPreemptionContainers()));
+    assertTrue(!Collections.disjoint(scheduler.applications.get(app6).getLiveContainers(),
+                                     scheduler.applications.get(app6).getPreemptionContainers()));
 
     // Pretend 15 seconds have passed
     clock.tick(15);
@@ -1183,8 +1165,8 @@ public class TestFairScheduler {
         Resources.createResource(2 * 1024));
 
     // At this point the containers should have been killed (since we are not simulating AM)
-    assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
-    assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app3).getLiveContainers().size());
 
     // Trigger a kill by insisting we want containers back
     scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
@@ -1198,22 +1180,22 @@ public class TestFairScheduler {
     scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
         Resources.createResource(2 * 1024));
     
-    assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
-    assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size());
-    assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
-    assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size());
-    assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app2).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app3).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app5).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size());
 
     // Now A and B are below fair share, so preemption shouldn't do anything
     scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
         Resources.createResource(2 * 1024));
-    assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
-    assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size());
-    assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
-    assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
-    assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size());
-    assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app2).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app3).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app5).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size());
   }
 
   @Test (timeout = 5000)
@@ -1372,9 +1354,9 @@ public class TestFairScheduler {
     
     // One container should get reservation and the other should get nothing
     assertEquals(1024,
-        scheduler.getSchedulerApp(attId1).getCurrentReservation().getMemory());
+        scheduler.applications.get(attId1).getCurrentReservation().getMemory());
     assertEquals(0,
-        scheduler.getSchedulerApp(attId2).getCurrentReservation().getMemory());
+        scheduler.applications.get(attId2).getCurrentReservation().getMemory());
   }
 
   @Test (timeout = 5000)
@@ -1409,7 +1391,7 @@ public class TestFairScheduler {
     scheduler.handle(updateEvent);
     
     // App 1 should be running
-    assertEquals(1, scheduler.getSchedulerApp(attId1).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(attId1).getLiveContainers().size());
     
     ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
         "user1", 1);
@@ -1418,7 +1400,7 @@ public class TestFairScheduler {
     scheduler.handle(updateEvent);
     
     // App 2 should not be running
-    assertEquals(0, scheduler.getSchedulerApp(attId2).getLiveContainers().size());
+    assertEquals(0, scheduler.applications.get(attId2).getLiveContainers().size());
     
     // Request another container for app 1
     createSchedulingRequestExistingApplication(1024, 1, attId1);
@@ -1427,7 +1409,7 @@ public class TestFairScheduler {
     scheduler.handle(updateEvent);
     
     // Request should be fulfilled
-    assertEquals(2, scheduler.getSchedulerApp(attId1).getLiveContainers().size());
+    assertEquals(2, scheduler.applications.get(attId1).getLiveContainers().size());
   }
   
   @Test (timeout = 5000)
@@ -1447,10 +1429,10 @@ public class TestFairScheduler {
     NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(updateEvent);
     
-    FSSchedulerApp app = scheduler.getSchedulerApp(attId);
+    FSSchedulerApp app = scheduler.applications.get(attId);
     assertEquals(1, app.getLiveContainers().size());
     
-    ContainerId containerId = scheduler.getSchedulerApp(attId)
+    ContainerId containerId = scheduler.applications.get(attId)
         .getLiveContainers().iterator().next().getContainerId();
 
     // Cause reservation to be created
@@ -1519,9 +1501,9 @@ public class TestFairScheduler {
     ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
         "norealuserhasthisname2", 1);
 
-    FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
+    FSSchedulerApp app1 = scheduler.applications.get(attId1);
     assertNotNull("The application was not allowed", app1);
-    FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
+    FSSchedulerApp app2 = scheduler.applications.get(attId2);
     assertNull("The application was allowed", app2);
   }
   
@@ -1544,8 +1526,7 @@ public class TestFairScheduler {
     scheduler.handle(nodeEvent2);
     
     ApplicationAttemptId appId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
-    scheduler.addApplication(appId.getApplicationId(), "queue1", "user1");
-    scheduler.addApplicationAttempt(appId, false);
+    scheduler.addApplicationAttempt(appId, "queue1", "user1");
     
     // 1 request with 2 nodes on the same rack. another request with 1 node on
     // a different rack
@@ -1564,14 +1545,14 @@ public class TestFairScheduler {
     NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(updateEvent1);
     // should assign node local
-    assertEquals(1, scheduler.getSchedulerApp(appId).getLiveContainers().size());
+    assertEquals(1, scheduler.applications.get(appId).getLiveContainers().size());
 
     // node 2 checks in
     scheduler.update();
     NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
     scheduler.handle(updateEvent2);
     // should assign rack local
-    assertEquals(2, scheduler.getSchedulerApp(appId).getLiveContainers().size());
+    assertEquals(2, scheduler.applications.get(appId).getLiveContainers().size());
   }
   
   @Test (timeout = 5000)
@@ -1590,8 +1571,8 @@ public class TestFairScheduler {
         "user1", 2);
     ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
         "user1", 2);
-    FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
-    FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
+    FSSchedulerApp app1 = scheduler.applications.get(attId1);
+    FSSchedulerApp app2 = scheduler.applications.get(attId2);
     
     FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true);
     queue1.setPolicy(new FifoPolicy());
@@ -1631,7 +1612,7 @@ public class TestFairScheduler {
 
     ApplicationAttemptId attId =
         createSchedulingRequest(1024, "root.default", "user", 8);
-    FSSchedulerApp app = scheduler.getSchedulerApp(attId);
+    FSSchedulerApp app = scheduler.applications.get(attId);
 
     // set maxAssign to 2: only 2 containers should be allocated
     scheduler.maxAssign = 2;
@@ -1693,10 +1674,10 @@ public class TestFairScheduler {
     ApplicationAttemptId attId4 =
         createSchedulingRequest(1024, fifoQueue, user, 4);
 
-    FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
-    FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
-    FSSchedulerApp app3 = scheduler.getSchedulerApp(attId3);
-    FSSchedulerApp app4 = scheduler.getSchedulerApp(attId4);
+    FSSchedulerApp app1 = scheduler.applications.get(attId1);
+    FSSchedulerApp app2 = scheduler.applications.get(attId2);
+    FSSchedulerApp app3 = scheduler.applications.get(attId3);
+    FSSchedulerApp app4 = scheduler.applications.get(attId4);
 
     scheduler.getQueueManager().getLeafQueue(fifoQueue, true)
         .setPolicy(SchedulingPolicy.parse("fifo"));
@@ -1783,7 +1764,7 @@ public class TestFairScheduler {
 
     ApplicationAttemptId attId =
         ApplicationAttemptId.newInstance(applicationId, this.ATTEMPT_ID++);
-    scheduler.addApplication(attId.getApplicationId(), queue, user);
+    scheduler.addApplicationAttempt(attId, queue, user);
 
     numTries = 0;
     while (application.getFinishTime() == 0 && numTries < MAX_TRIES) {
@@ -1811,7 +1792,7 @@ public class TestFairScheduler {
     NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(updateEvent);
     
-    FSSchedulerApp app = scheduler.getSchedulerApp(attId);
+    FSSchedulerApp app = scheduler.applications.get(attId);
     assertEquals(0, app.getLiveContainers().size());
     assertEquals(0, app.getReservedContainers().size());
     
@@ -1880,7 +1861,7 @@ public class TestFairScheduler {
     NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2);
 
     // no matter how many heartbeats, node2 should never get a container
-    FSSchedulerApp app = scheduler.getSchedulerApp(attId1);
+    FSSchedulerApp app = scheduler.applications.get(attId1);
     for (int i = 0; i < 10; i++) {
       scheduler.handle(node2UpdateEvent);
       assertEquals(0, app.getLiveContainers().size());
@@ -1919,7 +1900,7 @@ public class TestFairScheduler {
     NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2);
 
     // no matter how many heartbeats, node2 should never get a container
-    FSSchedulerApp app = scheduler.getSchedulerApp(attId1);
+    FSSchedulerApp app = scheduler.applications.get(attId1);
     for (int i = 0; i < 10; i++) {
       scheduler.handle(node2UpdateEvent);
       assertEquals(0, app.getLiveContainers().size());
@@ -1952,7 +1933,7 @@ public class TestFairScheduler {
 
     ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
         "user1", 0);
-    FSSchedulerApp app = scheduler.getSchedulerApp(attId);
+    FSSchedulerApp app = scheduler.applications.get(attId);
     
     ResourceRequest nodeRequest = createResourceRequest(1024, node2.getHostName(), 1, 2, true);
     ResourceRequest rackRequest = createResourceRequest(1024, "rack1", 1, 2, true);
@@ -1992,7 +1973,7 @@ public class TestFairScheduler {
     
     ApplicationAttemptId attId = createSchedulingRequest(1024, 1, "default",
         "user1", 2);
-    FSSchedulerApp app = scheduler.getSchedulerApp(attId);
+    FSSchedulerApp app = scheduler.applications.get(attId);
     scheduler.update();
 
     NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
@@ -2012,10 +1993,10 @@ public class TestFairScheduler {
 
     ApplicationAttemptId appAttId1 = createSchedulingRequest(2048, 1, "queue1",
         "user1", 2);
-    FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
+    FSSchedulerApp app1 = scheduler.applications.get(appAttId1);
     ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 2, "queue1",
         "user1", 2);
-    FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
+    FSSchedulerApp app2 = scheduler.applications.get(appAttId2);
 
     DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
     drfPolicy.initialize(scheduler.getClusterCapacity());
@@ -2053,13 +2034,13 @@ public class TestFairScheduler {
 
     ApplicationAttemptId appAttId1 = createSchedulingRequest(3072, 1, "queue1",
         "user1", 2);
-    FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
+    FSSchedulerApp app1 = scheduler.applications.get(appAttId1);
     ApplicationAttemptId appAttId2 = createSchedulingRequest(2048, 2, "queue1",
         "user1", 2);
-    FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
+    FSSchedulerApp app2 = scheduler.applications.get(appAttId2);
     ApplicationAttemptId appAttId3 = createSchedulingRequest(1024, 2, "queue2",
         "user1", 2);
-    FSSchedulerApp app3 = scheduler.getSchedulerApp(appAttId3);
+    FSSchedulerApp app3 = scheduler.applications.get(appAttId3);
     
     DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
     drfPolicy.initialize(scheduler.getClusterCapacity());
@@ -2090,19 +2071,19 @@ public class TestFairScheduler {
     ApplicationAttemptId appAttId1 = createSchedulingRequest(3074, 1, "queue1.subqueue1",
         "user1", 2);
     Thread.sleep(3); // so that start times will be different
-    FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
+    FSSchedulerApp app1 = scheduler.applications.get(appAttId1);
     ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 3, "queue1.subqueue1",
         "user1", 2);
     Thread.sleep(3); // so that start times will be different
-    FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
+    FSSchedulerApp app2 = scheduler.applications.get(appAttId2);
     ApplicationAttemptId appAttId3 = createSchedulingRequest(2048, 2, "queue1.subqueue2",
         "user1", 2);
     Thread.sleep(3); // so that start times will be different
-    FSSchedulerApp app3 = scheduler.getSchedulerApp(appAttId3);
+    FSSchedulerApp app3 = scheduler.applications.get(appAttId3);
     ApplicationAttemptId appAttId4 = createSchedulingRequest(1024, 2, "queue2",
         "user1", 2);
     Thread.sleep(3); // so that start times will be different
-    FSSchedulerApp app4 = scheduler.getSchedulerApp(appAttId4);
+    FSSchedulerApp app4 = scheduler.applications.get(appAttId4);
     
     DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
     drfPolicy.initialize(scheduler.getClusterCapacity());
@@ -2182,7 +2163,7 @@ public class TestFairScheduler {
         NodeUpdateSchedulerEvent(node2);
 
     // no matter how many heartbeats, node2 should never get a container  
-    FSSchedulerApp app = scheduler.getSchedulerApp(attId1);
+    FSSchedulerApp app = scheduler.applications.get(attId1);
     for (int i = 0; i < 10; i++) {
       scheduler.handle(node2UpdateEvent);
       assertEquals(0, app.getLiveContainers().size());
@@ -2193,8 +2174,16 @@ public class TestFairScheduler {
     assertEquals(1, app.getLiveContainers().size());
   }
 
+  @Test
+  public void testConcurrentAccessOnApplications() throws Exception {
+    FairScheduler fs = new FairScheduler();
+    TestCapacityScheduler.verifyConcurrentAccessOnApplications(
+        fs.applications, FSSchedulerApp.class, FSLeafQueue.class);
+  }
+  
+  
   private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) {
-    FSSchedulerApp app = scheduler.getSchedulerApp(attId);
+    FSSchedulerApp app = scheduler.applications.get(attId);
     FSLeafQueue queue = app.getQueue();
     Collection<AppSchedulable> runnableApps =
         queue.getRunnableAppSchedulables();
@@ -2250,7 +2239,7 @@ public class TestFairScheduler {
     
     // Remove app 1 and both app 2 and app 4 should becomes runnable in its place
     AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
-        new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED, false);
+        new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED);
     scheduler.handle(appRemovedEvent1);
     verifyAppRunnable(attId2, true);
     verifyQueueNumRunnable("queue2", 1, 0);
@@ -2314,7 +2303,7 @@ public class TestFairScheduler {
     // Even though the app was removed from sub3, the app from sub2 gets to go
     // because it came in first
     AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
-        new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED, false);
+        new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED);
     scheduler.handle(appRemovedEvent1);
     verifyAppRunnable(attId4, true);
     verifyQueueNumRunnable("queue1.sub2", 2, 0);
@@ -2323,7 +2312,7 @@ public class TestFairScheduler {
 
     // Now test removal of a non-runnable app
     AppAttemptRemovedSchedulerEvent appRemovedEvent2 =
-        new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED, true);
+        new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED);
     scheduler.handle(appRemovedEvent2);
     assertEquals(0, scheduler.maxRunningEnforcer.usersNonRunnableApps
         .get("user1").size());
@@ -2331,7 +2320,7 @@ public class TestFairScheduler {
     verifyQueueNumRunnable("queue1.sub3", 0, 0);
     // verify it doesn't become runnable when there would be space for it
     AppAttemptRemovedSchedulerEvent appRemovedEvent3 =
-        new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED, true);
+        new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED);
     scheduler.handle(appRemovedEvent3);
     verifyQueueNumRunnable("queue1.sub2", 1, 0);
     verifyQueueNumRunnable("queue1.sub3", 0, 0);
@@ -2367,8 +2356,7 @@ public class TestFairScheduler {
     // send application request
     ApplicationAttemptId appAttemptId =
             createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
-    fs.addApplication(appAttemptId.getApplicationId(), "queue11", "user11");
-    fs.addApplicationAttempt(appAttemptId, false);
+    fs.addApplicationAttempt(appAttemptId, "queue11", "user11");
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     ResourceRequest request =
             createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
@@ -2379,7 +2367,7 @@ public class TestFairScheduler {
     // at least one pass
     Thread.sleep(fs.getConf().getContinuousSchedulingSleepMs() + 500);
 
-    FSSchedulerApp app = fs.getSchedulerApp(appAttemptId);
+    FSSchedulerApp app = fs.applications.get(appAttemptId);
     // Wait until app gets resources.
     while (app.getCurrentConsumption().equals(Resources.none())) { }
 
@@ -2467,7 +2455,7 @@ public class TestFairScheduler {
 
     ApplicationAttemptId appAttemptId =
         createSchedulingRequest(GB, "root.default", "user", 1);
-    FSSchedulerApp app = scheduler.getSchedulerApp(appAttemptId);
+    FSSchedulerApp app = scheduler.applications.get(appAttemptId);
 
     // Verify the blacklist can be updated independent of requesting containers
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
@@ -2477,7 +2465,7 @@ public class TestFairScheduler {
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host));
-    assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host));
+    assertFalse(scheduler.applications.get(appAttemptId).isBlacklisted(host));
 
     List<ResourceRequest> update = Arrays.asList(
         createResourceRequest(GB, node.getHostName(), 1, 0, true));
@@ -2539,12 +2527,4 @@ public class TestFairScheduler {
     assertTrue(appAttIds.contains(appAttId1));
     assertTrue(appAttIds.contains(appAttId2));
   }
-
-  @Test
-  public void testAddAndRemoveAppFromFairScheduler() throws Exception {
-    FairScheduler scheduler =
-        (FairScheduler) resourceManager.getResourceScheduler();
-    TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(
-      scheduler.getSchedulerApplications(), scheduler, "default");
-  }
 }

+ 24 - 44
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

@@ -61,11 +61,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
@@ -152,16 +150,14 @@ public class TestFifoScheduler {
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
         appId, 1);
 
-    SchedulerEvent appEvent = new AppAddedSchedulerEvent(appId, "queue", "user");
-    schedular.handle(appEvent);
-    SchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    schedular.handle(attemptEvent);
+    SchedulerEvent event =
+        new AppAttemptAddedSchedulerEvent(appAttemptId, "queue", "user");
+    schedular.handle(event);
 
     appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2);
-    SchedulerEvent attemptEvent2 =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    schedular.handle(attemptEvent2);
+
+    event = new AppAttemptAddedSchedulerEvent(appAttemptId, "queue", "user");
+    schedular.handle(event);
 
     int afterAppsSubmitted = metrics.getAppsSubmitted();
     Assert.assertEquals(1, afterAppsSubmitted - beforeAppsSubmitted);
@@ -192,13 +188,9 @@ public class TestFifoScheduler {
     int _appAttemptId = 1;
     ApplicationAttemptId appAttemptId = createAppAttemptId(_appId,
         _appAttemptId);
-    AppAddedSchedulerEvent appEvent =
-        new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "queue1",
-          "user1");
-    scheduler.handle(appEvent);
-    AppAttemptAddedSchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    scheduler.handle(attemptEvent);
+    AppAttemptAddedSchedulerEvent appEvent1 =
+        new AppAttemptAddedSchedulerEvent(appAttemptId, "queue1", "user1");
+    scheduler.handle(appEvent1);
 
     int memory = 64;
     int nConts = 3;
@@ -282,13 +274,9 @@ public class TestFifoScheduler {
     int _appAttemptId = 1;
     ApplicationAttemptId appAttemptId = createAppAttemptId(_appId,
         _appAttemptId);
-    AppAddedSchedulerEvent appEvent =
-        new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "queue1",
-          "user1");
-    scheduler.handle(appEvent);
-    AppAttemptAddedSchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    scheduler.handle(attemptEvent);
+    AppAttemptAddedSchedulerEvent appEvent1 =
+        new AppAttemptAddedSchedulerEvent(appAttemptId, "queue1", "user1");
+    scheduler.handle(appEvent1);
 
     int memory = 1024;
     int priority = 1;
@@ -528,6 +516,13 @@ public class TestFifoScheduler {
     LOG.info("--- END: testFifoScheduler ---");
   }
 
+  @Test
+  public void testConcurrentAccessOnApplications() throws Exception {
+    FifoScheduler fs = new FifoScheduler();
+    TestCapacityScheduler.verifyConcurrentAccessOnApplications(
+        fs.applications, FiCaSchedulerApp.class, Queue.class);
+  }
+
   @SuppressWarnings("resource")
   @Test
   public void testBlackListNodes() throws Exception {
@@ -546,23 +541,19 @@ public class TestFifoScheduler {
     ApplicationId appId = BuilderUtils.newApplicationId(100, 1);
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
         appId, 1);
-    SchedulerEvent appEvent =
-        new AppAddedSchedulerEvent(appId, "default",
-          "user");
-    fs.handle(appEvent);
-    SchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    fs.handle(attemptEvent);
+    SchedulerEvent event =
+        new AppAttemptAddedSchedulerEvent(appAttemptId, "default", "user");
+    fs.handle(event);
 
     // Verify the blacklist can be updated independent of requesting containers
     fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null);
-    Assert.assertTrue(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertTrue(fs.getApplication(appAttemptId).isBlacklisted(host));
     fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host));
-    Assert.assertFalse(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertFalse(fs.getApplication(appAttemptId).isBlacklisted(host));
     rm.stop();
   }
   
@@ -584,17 +575,6 @@ public class TestFifoScheduler {
     Assert.assertNull(scheduler.getAppsInQueue("someotherqueue"));
   }
 
-  @Test
-  public void testAddAndRemoveAppFromFiFoScheduler() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
-        ResourceScheduler.class);
-    MockRM rm = new MockRM(conf);
-    FifoScheduler fs = (FifoScheduler)rm.getResourceScheduler();
-    TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(
-      fs.getSchedulerApplications(), fs, "queue");
-  }
-
   private void checkApplicationResourceUsage(int expected, 
       Application application) {
     Assert.assertEquals(expected, application.getUsedResources().getMemory());

+ 13 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java

@@ -41,9 +41,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 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;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
 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.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
@@ -1384,29 +1386,24 @@ public class TestRMWebServicesApps extends JerseyTest {
     rm.stop();
   }
 
-  @Test (timeout = 20000)
+  @Test
   public void testMultipleAppAttempts() throws JSONException, Exception {
     rm.start();
-    MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 8192);
+    MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
     RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1");
-    MockAM am = MockRM.launchAndRegisterAM(app1, rm, amNodeManager);
+    amNodeManager.nodeHeartbeat(true);
     int maxAppAttempts = rm.getConfig().getInt(
         YarnConfiguration.RM_AM_MAX_ATTEMPTS,
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     assertTrue(maxAppAttempts > 1);
-    int numAttempt = 1;
-    while (true) {
-      // fail the AM by sending CONTAINER_FINISHED event without registering.
-      amNodeManager.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
-      am.waitForState(RMAppAttemptState.FAILED);
-      if (numAttempt == maxAppAttempts) {
-        rm.waitForState(app1.getApplicationId(), RMAppState.FAILED);
-        break;
-      }
-      // wait for app to start a new attempt.
+    int retriesLeft = maxAppAttempts;
+    while (--retriesLeft > 0) {
+      RMAppEvent event =
+          new RMAppFailedAttemptEvent(app1.getApplicationId(),
+              RMAppEventType.ATTEMPT_FAILED, "");
+      app1.handle(event);
       rm.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
-      am = MockRM.launchAndRegisterAM(app1, rm, amNodeManager);
-      numAttempt++;
+      amNodeManager.nodeHeartbeat(true);
     }
     assertEquals("incorrect number of attempts", maxAppAttempts,
         app1.getAppAttempts().values().size());