Jelajahi Sumber

YARN-415. Capture aggregate memory allocation at the app-level for chargeback. Contributed by Eric Payne & Andrey Klochkov
(cherry picked from commit 83be3ad44484bf8a24cb90de4b9c26ab59d226a8)

Jian He 10 tahun lalu
induk
melakukan
ceae7be6b2
38 mengubah file dengan 1045 tambahan dan 97 penghapusan
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 40 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java
  3. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  4. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
  5. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
  6. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationResourceUsageReportPBImpl.java
  7. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
  8. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
  9. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
  10. 7 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
  11. 19 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
  12. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
  13. 30 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java
  14. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationAttemptStateDataPBImpl.java
  15. 15 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  16. 14 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppMetrics.java
  17. 60 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AggregateAppResourceUsage.java
  18. 13 3
      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
  19. 34 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java
  20. 18 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
  21. 37 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
  22. 3 0
      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
  23. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
  24. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
  25. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
  26. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/yarn_server_resourcemanager_recovery.proto
  27. 54 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  28. 401 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
  29. 18 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
  30. 16 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
  31. 14 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
  32. 67 5
      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
  33. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
  34. 52 37
      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
  35. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
  36. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
  37. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
  38. 19 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm

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

@@ -44,6 +44,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2394. FairScheduler: Configure fairSharePreemptionThreshold per queue.
     (Wei Yan via kasha)
 
+    YARN-415. Capture aggregate memory allocation at the app-level for chargeback.
+    (Eric Payne & Andrey Klochkov via jianhe)
+
   IMPROVEMENTS
 
     YARN-2242. Improve exception information on AM launch crashes. (Li Lu 

+ 40 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java

@@ -35,7 +35,8 @@ public abstract class ApplicationResourceUsageReport {
   @Unstable
   public static ApplicationResourceUsageReport newInstance(
       int numUsedContainers, int numReservedContainers, Resource usedResources,
-      Resource reservedResources, Resource neededResources) {
+      Resource reservedResources, Resource neededResources, long memorySeconds,
+      long vcoreSeconds) {
     ApplicationResourceUsageReport report =
         Records.newRecord(ApplicationResourceUsageReport.class);
     report.setNumUsedContainers(numUsedContainers);
@@ -43,6 +44,8 @@ public abstract class ApplicationResourceUsageReport {
     report.setUsedResources(usedResources);
     report.setReservedResources(reservedResources);
     report.setNeededResources(neededResources);
+    report.setMemorySeconds(memorySeconds);
+    report.setVcoreSeconds(vcoreSeconds);
     return report;
   }
 
@@ -113,4 +116,40 @@ public abstract class ApplicationResourceUsageReport {
   @Private
   @Unstable
   public abstract void setNeededResources(Resource needed_resources);
+
+  /**
+   * Set the aggregated amount of memory (in megabytes) the application has
+   * allocated times the number of seconds the application has been running.
+   * @param memory_seconds the aggregated amount of memory seconds
+   */
+  @Private
+  @Unstable
+  public abstract void setMemorySeconds(long memory_seconds);
+
+  /**
+   * Get the aggregated amount of memory (in megabytes) the application has
+   * allocated times the number of seconds the application has been running.
+   * @return the aggregated amount of memory seconds
+   */
+  @Public
+  @Unstable
+  public abstract long getMemorySeconds();
+
+  /**
+   * Set the aggregated number of vcores that the application has allocated
+   * times the number of seconds the application has been running.
+   * @param vcore_seconds the aggregated number of vcore seconds
+   */
+  @Private
+  @Unstable
+  public abstract void setVcoreSeconds(long vcore_seconds);
+
+  /**
+   * Get the aggregated number of vcores that the application has allocated
+   * times the number of seconds the application has been running.
+   * @return the aggregated number of vcore seconds
+   */
+  @Public
+  @Unstable
+  public abstract long getVcoreSeconds();
 }

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

@@ -167,6 +167,8 @@ message ApplicationResourceUsageReportProto {
   optional ResourceProto used_resources = 3;
   optional ResourceProto reserved_resources = 4;
   optional ResourceProto needed_resources = 5;
+  optional int64 memory_seconds = 6;
+  optional int64 vcore_seconds = 7;
 }
 
 message ApplicationReportProto {

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
@@ -460,6 +461,11 @@ public class ApplicationCLI extends YarnCLI {
       appReportStr.println(appReport.getRpcPort());
       appReportStr.print("\tAM Host : ");
       appReportStr.println(appReport.getHost());
+      appReportStr.print("\tAggregate Resource Allocation : ");
+
+      ApplicationResourceUsageReport usageReport = appReport.getApplicationResourceUsageReport();
+      appReportStr.print(usageReport.getMemorySeconds() + " MB-seconds, ");
+      appReportStr.println(usageReport.getVcoreSeconds() + " vcore-seconds");
       appReportStr.print("\tDiagnostics : ");
       appReportStr.print(appReport.getDiagnostics());
     } else {

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -87,11 +88,15 @@ public class TestYarnCLI {
   public void testGetApplicationReport() throws Exception {
     ApplicationCLI cli = createAndGetAppCLI();
     ApplicationId applicationId = ApplicationId.newInstance(1234, 5);
+    ApplicationResourceUsageReport usageReport = 
+        ApplicationResourceUsageReport.newInstance(
+            2, 0, null, null, null, 123456, 4567);
     ApplicationReport newApplicationReport = ApplicationReport.newInstance(
         applicationId, ApplicationAttemptId.newInstance(applicationId, 1),
         "user", "queue", "appname", "host", 124, null,
         YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
-        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f, "YARN", null);
+        FinalApplicationStatus.SUCCEEDED, usageReport, "N/A", 0.53789f, "YARN",
+        null);
     when(client.getApplicationReport(any(ApplicationId.class))).thenReturn(
         newApplicationReport);
     int result = cli.run(new String[] { "application", "-status", applicationId.toString() });
@@ -113,6 +118,7 @@ public class TestYarnCLI {
     pw.println("\tTracking-URL : N/A");
     pw.println("\tRPC Port : 124");
     pw.println("\tAM Host : host");
+    pw.println("\tAggregate Resource Allocation : 123456 MB-seconds, 4567 vcore-seconds");
     pw.println("\tDiagnostics : diagnostics");
     pw.close();
     String appReportStr = baos.toString("UTF-8");

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

@@ -200,6 +200,30 @@ extends ApplicationResourceUsageReport {
     this.neededResources = reserved_resources;
   }
 
+  @Override
+  public synchronized void setMemorySeconds(long memory_seconds) {
+    maybeInitBuilder();
+    builder.setMemorySeconds(memory_seconds);
+  }
+  
+  @Override
+  public synchronized long getMemorySeconds() {
+    ApplicationResourceUsageReportProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getMemorySeconds();
+  }
+
+  @Override
+  public synchronized void setVcoreSeconds(long vcore_seconds) {
+    maybeInitBuilder();
+    builder.setVcoreSeconds(vcore_seconds);
+  }
+
+  @Override
+  public synchronized long getVcoreSeconds() {
+    ApplicationResourceUsageReportProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getVcoreSeconds());
+  }
+  
   private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
     return new ResourcePBImpl(p);
   }

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java

@@ -370,7 +370,8 @@ public class BuilderUtils {
   
   public static ApplicationResourceUsageReport newApplicationResourceUsageReport(
       int numUsedContainers, int numReservedContainers, Resource usedResources,
-      Resource reservedResources, Resource neededResources) {
+      Resource reservedResources, Resource neededResources, long memorySeconds, 
+      long vcoreSeconds) {
     ApplicationResourceUsageReport report =
         recordFactory.newRecordInstance(ApplicationResourceUsageReport.class);
     report.setNumUsedContainers(numUsedContainers);
@@ -378,6 +379,8 @@ public class BuilderUtils {
     report.setUsedResources(usedResources);
     report.setReservedResources(reservedResources);
     report.setNeededResources(neededResources);
+    report.setMemorySeconds(memorySeconds);
+    report.setVcoreSeconds(vcoreSeconds);
     return report;
   }
 

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

@@ -236,5 +236,5 @@ public class RMServerUtils {
     DUMMY_APPLICATION_RESOURCE_USAGE_REPORT =
       BuilderUtils.newApplicationResourceUsageReport(-1, -1,
           Resources.createResource(-1, -1), Resources.createResource(-1, -1),
-          Resources.createResource(-1, -1));
+          Resources.createResource(-1, -1), 0, 0);
 }

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

@@ -280,7 +280,9 @@ public class FileSystemRMStateStore extends RMStateStore {
                   attemptStateData.getFinalTrackingUrl(),
                   attemptStateData.getDiagnostics(),
                   attemptStateData.getFinalApplicationStatus(),
-                  attemptStateData.getAMContainerExitStatus());
+                  attemptStateData.getAMContainerExitStatus(),
+                  attemptStateData.getMemorySeconds(),
+                  attemptStateData.getVcoreSeconds());
 
             // assert child node name is same as application attempt id
             assert attemptId.equals(attemptState.getAttemptId());

+ 7 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java

@@ -138,7 +138,10 @@ public class MemoryRMStateStore extends RMStateStore {
     ApplicationAttemptState attemptState =
         new ApplicationAttemptState(appAttemptId,
           attemptStateData.getMasterContainer(), credentials,
-          attemptStateData.getStartTime());
+          attemptStateData.getStartTime(),
+          attemptStateData.getMemorySeconds(),
+          attemptStateData.getVcoreSeconds());
+
 
     ApplicationState appState = state.getApplicationState().get(
         attemptState.getAttemptId().getApplicationId());
@@ -167,7 +170,9 @@ public class MemoryRMStateStore extends RMStateStore {
           attemptStateData.getFinalTrackingUrl(),
           attemptStateData.getDiagnostics(),
           attemptStateData.getFinalApplicationStatus(),
-          attemptStateData.getAMContainerExitStatus());
+          attemptStateData.getAMContainerExitStatus(),
+          attemptStateData.getMemorySeconds(),
+          attemptStateData.getVcoreSeconds());
 
     ApplicationState appState =
         state.getApplicationState().get(

+ 19 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java

@@ -55,6 +55,7 @@ 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.AggregateAppResourceUsage;
 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;
@@ -265,19 +266,21 @@ public abstract class RMStateStore extends AbstractService {
     String diagnostics;
     int exitStatus = ContainerExitStatus.INVALID;
     FinalApplicationStatus amUnregisteredFinalStatus;
+    long memorySeconds;
+    long vcoreSeconds;
 
     public ApplicationAttemptState(ApplicationAttemptId attemptId,
         Container masterContainer, Credentials appAttemptCredentials,
-        long startTime) {
+        long startTime, long memorySeconds, long vcoreSeconds) {
       this(attemptId, masterContainer, appAttemptCredentials, startTime, null,
-        null, "", null, ContainerExitStatus.INVALID);
+        null, "", null, ContainerExitStatus.INVALID, memorySeconds, vcoreSeconds);
     }
 
     public ApplicationAttemptState(ApplicationAttemptId attemptId,
         Container masterContainer, Credentials appAttemptCredentials,
         long startTime, RMAppAttemptState state, String finalTrackingUrl,
         String diagnostics, FinalApplicationStatus amUnregisteredFinalStatus,
-        int exitStatus) {
+        int exitStatus, long memorySeconds, long vcoreSeconds) {
       this.attemptId = attemptId;
       this.masterContainer = masterContainer;
       this.appAttemptCredentials = appAttemptCredentials;
@@ -287,6 +290,8 @@ public abstract class RMStateStore extends AbstractService {
       this.diagnostics = diagnostics == null ? "" : diagnostics;
       this.amUnregisteredFinalStatus = amUnregisteredFinalStatus;
       this.exitStatus = exitStatus;
+      this.memorySeconds = memorySeconds;
+      this.vcoreSeconds = vcoreSeconds;
     }
 
     public Container getMasterContainer() {
@@ -316,6 +321,12 @@ public abstract class RMStateStore extends AbstractService {
     public int getAMContainerExitStatus(){
       return this.exitStatus;
     }
+    public long getMemorySeconds() {
+      return memorySeconds;
+    }
+    public long getVcoreSeconds() {
+      return vcoreSeconds;
+    }
   }
   
   /**
@@ -587,10 +598,13 @@ public abstract class RMStateStore extends AbstractService {
   public synchronized void storeNewApplicationAttempt(RMAppAttempt appAttempt) {
     Credentials credentials = getCredentialsFromAppAttempt(appAttempt);
 
+    AggregateAppResourceUsage resUsage =
+        appAttempt.getRMAppAttemptMetrics().getAggregateAppResourceUsage();
     ApplicationAttemptState attemptState =
         new ApplicationAttemptState(appAttempt.getAppAttemptId(),
           appAttempt.getMasterContainer(), credentials,
-          appAttempt.getStartTime());
+          appAttempt.getStartTime(), resUsage.getMemorySeconds(),
+          resUsage.getVcoreSeconds());
 
     dispatcher.getEventHandler().handle(
       new RMStateStoreAppAttemptEvent(attemptState));
@@ -746,7 +760,7 @@ public abstract class RMStateStore extends AbstractService {
       ApplicationAttemptState attemptState =
           new ApplicationAttemptState(appAttempt.getAppAttemptId(),
             appAttempt.getMasterContainer(), credentials,
-            appAttempt.getStartTime());
+            appAttempt.getStartTime(), 0, 0);
       appState.attempts.put(attemptState.getAttemptId(), attemptState);
     }
     

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

@@ -603,7 +603,9 @@ public class ZKRMStateStore extends RMStateStore {
               attemptStateData.getFinalTrackingUrl(),
               attemptStateData.getDiagnostics(),
               attemptStateData.getFinalApplicationStatus(),
-              attemptStateData.getAMContainerExitStatus());
+              attemptStateData.getAMContainerExitStatus(),
+              attemptStateData.getMemorySeconds(),
+              attemptStateData.getVcoreSeconds());
 
         appState.attempts.put(attemptState.getAttemptId(), attemptState);
       }

+ 30 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java

@@ -43,7 +43,8 @@ public abstract class ApplicationAttemptStateData {
       ApplicationAttemptId attemptId, Container container,
       ByteBuffer attemptTokens, long startTime, RMAppAttemptState finalState,
       String finalTrackingUrl, String diagnostics,
-      FinalApplicationStatus amUnregisteredFinalStatus, int exitStatus) {
+      FinalApplicationStatus amUnregisteredFinalStatus, int exitStatus,
+      long memorySeconds, long vcoreSeconds) {
     ApplicationAttemptStateData attemptStateData =
         Records.newRecord(ApplicationAttemptStateData.class);
     attemptStateData.setAttemptId(attemptId);
@@ -55,6 +56,8 @@ public abstract class ApplicationAttemptStateData {
     attemptStateData.setStartTime(startTime);
     attemptStateData.setFinalApplicationStatus(amUnregisteredFinalStatus);
     attemptStateData.setAMContainerExitStatus(exitStatus);
+    attemptStateData.setMemorySeconds(memorySeconds);
+    attemptStateData.setVcoreSeconds(vcoreSeconds);
     return attemptStateData;
   }
 
@@ -72,7 +75,8 @@ public abstract class ApplicationAttemptStateData {
       attemptState.getStartTime(), attemptState.getState(),
       attemptState.getFinalTrackingUrl(), attemptState.getDiagnostics(),
       attemptState.getFinalApplicationStatus(),
-      attemptState.getAMContainerExitStatus());
+      attemptState.getAMContainerExitStatus(),
+      attemptState.getMemorySeconds(), attemptState.getVcoreSeconds());
   }
 
   public abstract ApplicationAttemptStateDataProto getProto();
@@ -157,4 +161,28 @@ public abstract class ApplicationAttemptStateData {
   public abstract int getAMContainerExitStatus();
 
   public abstract void setAMContainerExitStatus(int exitStatus);
+
+  /**
+   * Get the <em>memory seconds</em> (in MB seconds) of the application.
+   * @return <em>memory seconds</em> (in MB seconds) of the application
+   */
+  @Public
+  @Unstable
+  public abstract long getMemorySeconds();
+
+  @Public
+  @Unstable
+  public abstract void setMemorySeconds(long memorySeconds);
+
+  /**
+   * Get the <em>vcore seconds</em> of the application.
+   * @return <em>vcore seconds</em> of the application
+   */
+  @Public
+  @Unstable
+  public abstract long getVcoreSeconds();
+
+  @Public
+  @Unstable
+  public abstract void setVcoreSeconds(long vcoreSeconds);
 }

+ 24 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationAttemptStateDataPBImpl.java

@@ -228,6 +228,30 @@ public class ApplicationAttemptStateDataPBImpl extends
     builder.setStartTime(startTime);
   }
 
+  @Override
+  public long getMemorySeconds() {
+    ApplicationAttemptStateDataProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getMemorySeconds();
+  }
+ 
+  @Override
+  public long getVcoreSeconds() {
+    ApplicationAttemptStateDataProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getVcoreSeconds();
+  }
+
+  @Override
+  public void setMemorySeconds(long memorySeconds) {
+    maybeInitBuilder();
+    builder.setMemorySeconds(memorySeconds);
+  }
+ 
+  @Override
+  public void setVcoreSeconds(long vcoreSeconds) {
+    maybeInitBuilder();
+    builder.setVcoreSeconds(vcoreSeconds);
+  }
+
   @Override
   public FinalApplicationStatus getFinalApplicationStatus() {
     ApplicationAttemptStateDataProtoOrBuilder p = viaProto ? proto : builder;

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

@@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.Appli
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppNodeUpdateEvent.RMAppNodeUpdateType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AggregateAppResourceUsage;
 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;
@@ -561,6 +562,10 @@ public class RMAppImpl implements RMApp, Recoverable {
             }
           }
         }
+
+        RMAppMetrics rmAppMetrics = getRMAppMetrics();
+        appUsageReport.setMemorySeconds(rmAppMetrics.getMemorySeconds());
+        appUsageReport.setVcoreSeconds(rmAppMetrics.getVcoreSeconds());
       }
 
       if (currentApplicationAttemptId == null) {
@@ -1117,7 +1122,6 @@ public class RMAppImpl implements RMApp, Recoverable {
 
     @Override
     public RMAppState transition(RMAppImpl app, RMAppEvent event) {
-
       if (!app.submissionContext.getUnmanagedAM()
           && app.getNumFailedAppAttempts() < app.maxAppAttempts) {
         boolean transferStateFromPreviousAttempt = false;
@@ -1199,6 +1203,8 @@ public class RMAppImpl implements RMApp, Recoverable {
     Resource resourcePreempted = Resource.newInstance(0, 0);
     int numAMContainerPreempted = 0;
     int numNonAMContainerPreempted = 0;
+    long memorySeconds = 0;
+    long vcoreSeconds = 0;
     for (RMAppAttempt attempt : attempts.values()) {
       if (null != attempt) {
         RMAppAttemptMetrics attemptMetrics =
@@ -1208,10 +1214,17 @@ public class RMAppImpl implements RMApp, Recoverable {
         numAMContainerPreempted += attemptMetrics.getIsPreempted() ? 1 : 0;
         numNonAMContainerPreempted +=
             attemptMetrics.getNumNonAMContainersPreempted();
+        // getAggregateAppResourceUsage() will calculate resource usage stats
+        // for both running and finished containers.
+        AggregateAppResourceUsage resUsage =
+            attempt.getRMAppAttemptMetrics().getAggregateAppResourceUsage();
+        memorySeconds += resUsage.getMemorySeconds();
+        vcoreSeconds += resUsage.getVcoreSeconds();
       }
     }
 
     return new RMAppMetrics(resourcePreempted,
-        numNonAMContainerPreempted, numAMContainerPreempted);
+        numNonAMContainerPreempted, numAMContainerPreempted,
+        memorySeconds, vcoreSeconds);
   }
 }

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

@@ -24,12 +24,17 @@ public class RMAppMetrics {
   final Resource resourcePreempted;
   final int numNonAMContainersPreempted;
   final int numAMContainersPreempted;
+  final long memorySeconds;
+  final long vcoreSeconds;
 
   public RMAppMetrics(Resource resourcePreempted,
-      int numNonAMContainersPreempted, int numAMContainersPreempted) {
+      int numNonAMContainersPreempted, int numAMContainersPreempted,
+      long memorySeconds, long vcoreSeconds) {
     this.resourcePreempted = resourcePreempted;
     this.numNonAMContainersPreempted = numNonAMContainersPreempted;
     this.numAMContainersPreempted = numAMContainersPreempted;
+    this.memorySeconds = memorySeconds;
+    this.vcoreSeconds = vcoreSeconds;
   }
 
   public Resource getResourcePreempted() {
@@ -43,4 +48,12 @@ public class RMAppMetrics {
   public int getNumAMContainersPreempted() {
     return numAMContainersPreempted;
   }
+
+  public long getMemorySeconds() {
+    return memorySeconds;
+  }
+
+  public long getVcoreSeconds() {
+    return vcoreSeconds;
+  }
 }

+ 60 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AggregateAppResourceUsage.java

@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+
+@Private
+public class AggregateAppResourceUsage {
+  long memorySeconds;
+  long vcoreSeconds;
+
+  public AggregateAppResourceUsage(long memorySeconds, long vcoreSeconds) {
+    this.memorySeconds = memorySeconds;
+    this.vcoreSeconds = vcoreSeconds;
+  }
+
+  /**
+   * @return the memorySeconds
+   */
+  public long getMemorySeconds() {
+    return memorySeconds;
+  }
+
+  /**
+   * @param memorySeconds the memorySeconds to set
+   */
+  public void setMemorySeconds(long memorySeconds) {
+    this.memorySeconds = memorySeconds;
+  }
+
+  /**
+   * @return the vcoreSeconds
+   */
+  public long getVcoreSeconds() {
+    return vcoreSeconds;
+  }
+
+  /**
+   * @param vcoreSeconds the vcoreSeconds to set
+   */
+  public void setVcoreSeconds(long vcoreSeconds) {
+    this.vcoreSeconds = vcoreSeconds;
+  }
+}

+ 13 - 3
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

@@ -85,6 +85,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
@@ -430,7 +431,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     this.proxiedTrackingUrl = generateProxyUriWithScheme(null);
     this.maybeLastAttempt = maybeLastAttempt;
     this.stateMachine = stateMachineFactory.make(this);
-    this.attemptMetrics = new RMAppAttemptMetrics(applicationAttemptId);
+    this.attemptMetrics =
+        new RMAppAttemptMetrics(applicationAttemptId, rmContext);
   }
 
   @Override
@@ -704,6 +706,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       if (report == null) {
         report = RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT;
       }
+      AggregateAppResourceUsage resUsage =
+          this.attemptMetrics.getAggregateAppResourceUsage();
+      report.setMemorySeconds(resUsage.getMemorySeconds());
+      report.setVcoreSeconds(resUsage.getVcoreSeconds());
       return report;
     } finally {
       this.readLock.unlock();
@@ -733,6 +739,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     this.proxiedTrackingUrl = generateProxyUriWithScheme(originalTrackingUrl);
     this.finalStatus = attemptState.getFinalApplicationStatus();
     this.startTime = attemptState.getStartTime();
+    this.attemptMetrics.updateAggregateAppResourceUsage(
+        attemptState.getMemorySeconds(),attemptState.getVcoreSeconds());
   }
 
   public void transferStateFromPreviousAttempt(RMAppAttempt attempt) {
@@ -1017,12 +1025,14 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     default:
       break;
     }
-
+    AggregateAppResourceUsage resUsage =
+        this.attemptMetrics.getAggregateAppResourceUsage();
     RMStateStore rmStore = rmContext.getStateStore();
     ApplicationAttemptState attemptState =
         new ApplicationAttemptState(applicationAttemptId, getMasterContainer(),
           rmStore.getCredentialsFromAppAttempt(this), startTime,
-          stateToBeStored, finalTrackingUrl, diags, finalStatus, exitStatus);
+          stateToBeStored, finalTrackingUrl, diags, finalStatus, exitStatus,
+          resUsage.getMemorySeconds(), resUsage.getVcoreSeconds());
     LOG.info("Updating application attempt " + applicationAttemptId
         + " with final state: " + targetedFinalState + ", and exit status: "
         + exitStatus);

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -27,7 +28,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -42,12 +45,17 @@ public class RMAppAttemptMetrics {
   
   private ReadLock readLock;
   private WriteLock writeLock;
-  
-  public RMAppAttemptMetrics(ApplicationAttemptId attemptId) {
+  private AtomicLong finishedMemorySeconds = new AtomicLong(0);
+  private AtomicLong finishedVcoreSeconds = new AtomicLong(0);
+  private RMContext rmContext;
+
+  public RMAppAttemptMetrics(ApplicationAttemptId attemptId,
+      RMContext rmContext) {
     this.attemptId = attemptId;
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     this.readLock = lock.readLock();
     this.writeLock = lock.writeLock();
+    this.rmContext = rmContext;
   }
   
   public void updatePreemptionInfo(Resource resource, RMContainer container) {
@@ -94,4 +102,28 @@ public class RMAppAttemptMetrics {
   public boolean getIsPreempted() {
     return this.isPreempted.get();
   }
+
+  public AggregateAppResourceUsage getAggregateAppResourceUsage() {
+    long memorySeconds = finishedMemorySeconds.get();
+    long vcoreSeconds = finishedVcoreSeconds.get();
+
+    // Only add in the running containers if this is the active attempt.
+    RMAppAttempt currentAttempt = rmContext.getRMApps()
+                   .get(attemptId.getApplicationId()).getCurrentAppAttempt();
+    if (currentAttempt.getAppAttemptId().equals(attemptId)) {
+      ApplicationResourceUsageReport appResUsageReport = rmContext
+            .getScheduler().getAppResourceUsageReport(attemptId);
+      if (appResUsageReport != null) {
+        memorySeconds += appResUsageReport.getMemorySeconds();
+        vcoreSeconds += appResUsageReport.getVcoreSeconds();
+      }
+    }
+    return new AggregateAppResourceUsage(memorySeconds, vcoreSeconds);
+  }
+
+  public void updateAggregateAppResourceUsage(long finishedMemorySeconds,
+                                        long finishedVcoreSeconds) {
+    this.finishedMemorySeconds.addAndGet(finishedMemorySeconds);
+    this.finishedVcoreSeconds.addAndGet(finishedVcoreSeconds);
+  }
 }

+ 18 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java

@@ -24,6 +24,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
+import org.apache.commons.lang.time.DateUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -40,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
@@ -488,7 +490,7 @@ public class RMContainerImpl implements RMContainer {
       // Inform AppAttempt
       // container.getContainer() can return null when a RMContainer is a
       // reserved container
-      updateMetricsIfPreempted(container);
+      updateAttemptMetrics(container);
 
       container.eventHandler.handle(new RMAppAttemptContainerFinishedEvent(
         container.appAttemptId, finishedEvent.getRemoteContainerStatus()));
@@ -497,19 +499,27 @@ public class RMContainerImpl implements RMContainer {
         container);
     }
 
-    private static void updateMetricsIfPreempted(RMContainerImpl container) {
+    private static void updateAttemptMetrics(RMContainerImpl container) {
       // If this is a preempted container, update preemption metrics
+      Resource resource = container.getContainer().getResource();
+      RMAppAttempt rmAttempt = container.rmContext.getRMApps()
+          .get(container.getApplicationAttemptId().getApplicationId())
+          .getCurrentAppAttempt();
       if (ContainerExitStatus.PREEMPTED == container.finishedStatus
         .getExitStatus()) {
-
-        Resource resource = container.getContainer().getResource();
-        RMAppAttempt rmAttempt =
-            container.rmContext.getRMApps()
-              .get(container.getApplicationAttemptId().getApplicationId())
-              .getCurrentAppAttempt();
         rmAttempt.getRMAppAttemptMetrics().updatePreemptionInfo(resource,
           container);
       }
+
+      if (rmAttempt != null) {
+        long usedMillis = container.finishTime - container.creationTime;
+        long memorySeconds = resource.getMemory()
+                              * usedMillis / DateUtils.MILLIS_PER_SECOND;
+        long vcoreSeconds = resource.getVirtualCores()
+                             * usedMillis / DateUtils.MILLIS_PER_SECOND;
+        rmAttempt.getRMAppAttemptMetrics()
+                  .updateAggregateAppResourceUsage(memorySeconds,vcoreSeconds);
+      }
     }
   }
 

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

@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.commons.lang.time.DateUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -43,6 +44,7 @@ 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.AggregateAppResourceUsage;
 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;
@@ -69,6 +71,11 @@ public class SchedulerApplicationAttempt {
   private static final Log LOG = LogFactory
     .getLog(SchedulerApplicationAttempt.class);
 
+  private static final long MEM_AGGREGATE_ALLOCATION_CACHE_MSECS = 3000;
+  protected long lastMemoryAggregateAllocationUpdateTime = 0;
+  private long lastMemorySeconds = 0;
+  private long lastVcoreSeconds = 0;
+
   protected final AppSchedulingInfo appSchedulingInfo;
   
   protected Map<ContainerId, RMContainer> liveContainers =
@@ -505,12 +512,38 @@ public class SchedulerApplicationAttempt {
     lastScheduledContainer.put(priority, currentTimeMs);
     schedulingOpportunities.setCount(priority, 0);
   }
-  
+
+  synchronized AggregateAppResourceUsage getRunningAggregateAppResourceUsage() {
+    long currentTimeMillis = System.currentTimeMillis();
+    // Don't walk the whole container list if the resources were computed
+    // recently.
+    if ((currentTimeMillis - lastMemoryAggregateAllocationUpdateTime)
+        > MEM_AGGREGATE_ALLOCATION_CACHE_MSECS) {
+      long memorySeconds = 0;
+      long vcoreSeconds = 0;
+      for (RMContainer rmContainer : this.liveContainers.values()) {
+        long usedMillis = currentTimeMillis - rmContainer.getCreationTime();
+        Resource resource = rmContainer.getContainer().getResource();
+        memorySeconds += resource.getMemory() * usedMillis /  
+            DateUtils.MILLIS_PER_SECOND;
+        vcoreSeconds += resource.getVirtualCores() * usedMillis  
+            / DateUtils.MILLIS_PER_SECOND;
+      }
+
+      lastMemoryAggregateAllocationUpdateTime = currentTimeMillis;
+      lastMemorySeconds = memorySeconds;
+      lastVcoreSeconds = vcoreSeconds;
+    }
+    return new AggregateAppResourceUsage(lastMemorySeconds, lastVcoreSeconds);
+  }
+
   public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
+    AggregateAppResourceUsage resUsage = getRunningAggregateAppResourceUsage();
     return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
-        reservedContainers.size(), Resources.clone(currentConsumption),
-        Resources.clone(currentReservation),
-        Resources.add(currentConsumption, currentReservation));
+               reservedContainers.size(), Resources.clone(currentConsumption),
+               Resources.clone(currentReservation),
+               Resources.add(currentConsumption, currentReservation),
+               resUsage.getMemorySeconds(), resUsage.getVcoreSeconds());
   }
 
   public synchronized Map<ContainerId, RMContainer> getLiveContainersMap() {

+ 3 - 0
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

@@ -106,6 +106,9 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     queue.getMetrics().releaseResources(getUser(), 1, containerResource);
     Resources.subtractFrom(currentConsumption, containerResource);
 
+    // Clear resource utilization metrics cache.
+    lastMemoryAggregateAllocationUpdateTime = -1;
+
     return true;
   }
 

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

@@ -146,6 +146,9 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
     // remove from preemption map if it is completed
     preemptionMap.remove(rmContainer);
+
+    // Clear resource utilization metrics cache.
+    lastMemoryAggregateAllocationUpdateTime = -1;
   }
 
   private synchronized void unreserveInternal(

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java

@@ -146,7 +146,10 @@ public class AppBlock extends HtmlBlock {
           attemptMetrics.getResourcePreempted())
         ._("Number of Non-AM Containers Preempted from Current Attempt:",
           String.valueOf(attemptMetrics
-            .getNumNonAMContainersPreempted()));
+            .getNumNonAMContainersPreempted()))
+        ._("Aggregate Resource Allocation:",
+            String.format("%d MB-seconds, %d vcore-seconds", 
+                appMerics.getMemorySeconds(), appMerics.getVcoreSeconds()));
     pdiv._();
 
     Collection<RMAppAttempt> attempts = rmApp.getAppAttempts().values();

+ 12 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java

@@ -79,6 +79,8 @@ public class AppInfo {
   protected int allocatedMB;
   protected int allocatedVCores;
   protected int runningContainers;
+  protected long memorySeconds;
+  protected long vcoreSeconds;
   
   // preemption info fields
   protected int preemptedResourceMB;
@@ -165,6 +167,8 @@ public class AppInfo {
           appMetrics.getNumNonAMContainersPreempted();
       preemptedResourceVCores =
           appMetrics.getResourcePreempted().getVirtualCores();
+      memorySeconds = appMetrics.getMemorySeconds();
+      vcoreSeconds = appMetrics.getVcoreSeconds();
     }
   }
 
@@ -287,4 +291,12 @@ public class AppInfo {
   public int getNumAMContainersPreempted() {
     return numAMContainerPreempted;
   }
+ 
+  public long getMemorySeconds() {
+    return memorySeconds;
+  }
+
+  public long getVcoreSeconds() {
+    return vcoreSeconds;
+  }
 }

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/proto/yarn_server_resourcemanager_recovery.proto

@@ -78,6 +78,8 @@ message ApplicationAttemptStateDataProto {
     optional int64 start_time = 7;
     optional FinalApplicationStatusProto final_application_status = 8;
     optional int32 am_container_exit_status = 9 [default = -1000];
+    optional int64 memory_seconds = 10;
+    optional int64 vcore_seconds = 11;
 }
 
 message EpochProto {
@@ -87,4 +89,4 @@ message EpochProto {
 message AMRMTokenSecretManagerStateProto {
   optional MasterKeyProto current_master_key = 1;
   optional MasterKeyProto next_master_key = 2;
-}
+}

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

@@ -60,6 +60,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRes
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -79,6 +80,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -223,7 +225,7 @@ public class TestClientRMService {
   }
   
   @Test
-  public void testGetApplicationReport() throws YarnException {
+  public void testNonExistingApplicationReport() throws YarnException {
     RMContext rmContext = mock(RMContext.class);
     when(rmContext.getRMApps()).thenReturn(
         new ConcurrentHashMap<ApplicationId, RMApp>());
@@ -242,6 +244,38 @@ public class TestClientRMService {
               + "' doesn't exist in RM.");
     }
   }
+
+   @Test
+  public void testGetApplicationReport() throws Exception {
+    YarnScheduler yarnScheduler = mock(YarnScheduler.class);
+    RMContext rmContext = mock(RMContext.class);
+    mockRMContext(yarnScheduler, rmContext);
+
+    ApplicationId appId1 = getApplicationId(1);
+
+    ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class);
+    when(
+        mockAclsManager.checkAccess(UserGroupInformation.getCurrentUser(),
+            ApplicationAccessType.VIEW_APP, null, appId1)).thenReturn(true);
+
+    ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler,
+        null, mockAclsManager, null, null);
+    try {
+      RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+      GetApplicationReportRequest request = recordFactory
+          .newRecordInstance(GetApplicationReportRequest.class);
+      request.setApplicationId(appId1);
+      GetApplicationReportResponse response = 
+          rmService.getApplicationReport(request);
+      ApplicationReport report = response.getApplicationReport();
+      ApplicationResourceUsageReport usageReport = 
+          report.getApplicationResourceUsageReport();
+      Assert.assertEquals(10, usageReport.getMemorySeconds());
+      Assert.assertEquals(3, usageReport.getVcoreSeconds());
+    } finally {
+      rmService.close();
+    }
+  }
   
   @Test
   public void testGetApplicationAttemptReport() throws YarnException,
@@ -1065,11 +1099,11 @@ public class TestClientRMService {
     ApplicationId applicationId3 = getApplicationId(3);
     YarnConfiguration config = new YarnConfiguration();
     apps.put(applicationId1, getRMApp(rmContext, yarnScheduler, applicationId1,
-        config, "testqueue"));
+        config, "testqueue", 10, 3));
     apps.put(applicationId2, getRMApp(rmContext, yarnScheduler, applicationId2,
-        config, "a"));
+        config, "a", 20, 2));
     apps.put(applicationId3, getRMApp(rmContext, yarnScheduler, applicationId3,
-        config, "testqueue"));
+        config, "testqueue", 40, 5));
     return apps;
   }
   
@@ -1091,12 +1125,26 @@ public class TestClientRMService {
   }
 
   private RMAppImpl getRMApp(RMContext rmContext, YarnScheduler yarnScheduler,
-      ApplicationId applicationId3, YarnConfiguration config, String queueName) {
+      ApplicationId applicationId3, YarnConfiguration config, String queueName,
+      final long memorySeconds, final long vcoreSeconds) {
     ApplicationSubmissionContext asContext = mock(ApplicationSubmissionContext.class);
     when(asContext.getMaxAppAttempts()).thenReturn(1);
     RMAppImpl app = spy(new RMAppImpl(applicationId3, rmContext, config, null,
         null, queueName, asContext, yarnScheduler, null,
-        System.currentTimeMillis(), "YARN", null));
+        System.currentTimeMillis(), "YARN", null) {
+              @Override
+              public ApplicationReport createAndGetApplicationReport(
+                  String clientUserName, boolean allowAccess) {
+                ApplicationReport report = super.createAndGetApplicationReport(
+                    clientUserName, allowAccess);
+                ApplicationResourceUsageReport usageReport = 
+                    report.getApplicationResourceUsageReport();
+                usageReport.setMemorySeconds(memorySeconds);
+                usageReport.setVcoreSeconds(vcoreSeconds);
+                report.setApplicationResourceUsageReport(usageReport);
+                return report;
+              }
+          });
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
         ApplicationId.newInstance(123456, 1), 1);
     RMAppAttemptImpl rmAppAttemptImpl = spy(new RMAppAttemptImpl(attemptId,

+ 401 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java

@@ -0,0 +1,401 @@
+/*
+ * 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;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.commons.lang.time.DateUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+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.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
+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.rmapp.attempt.AggregateAppResourceUsage;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestContainerResourceUsage {
+
+  private YarnConfiguration conf;
+
+  @Before
+  public void setup() throws UnknownHostException {
+    Logger rootLogger = LogManager.getRootLogger();
+    rootLogger.setLevel(Level.DEBUG);
+    conf = new YarnConfiguration();
+    UserGroupInformation.setConfiguration(conf);
+    conf.set(YarnConfiguration.RECOVERY_ENABLED, "true");
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
+  }
+
+  @After
+  public void tearDown() {
+  }
+
+  @Test (timeout = 60000)
+  public void testUsageWithOneAttemptAndOneContainer() throws Exception {
+    MockRM rm = new MockRM(conf);
+    rm.start();
+    
+    MockNM nm =
+        new MockNM("127.0.0.1:1234", 15120, rm.getResourceTrackerService());
+    nm.registerNode();
+
+    RMApp app0 = rm.submitApp(200);
+
+    RMAppMetrics rmAppMetrics = app0.getRMAppMetrics();
+    Assert.assertTrue(
+        "Before app submittion, memory seconds should have been 0 but was "
+                          + rmAppMetrics.getMemorySeconds(),
+        rmAppMetrics.getMemorySeconds() == 0);
+    Assert.assertTrue(
+        "Before app submission, vcore seconds should have been 0 but was "
+                          + rmAppMetrics.getVcoreSeconds(),
+        rmAppMetrics.getVcoreSeconds() == 0);
+
+    RMAppAttempt attempt0 = app0.getCurrentAppAttempt();
+
+    nm.nodeHeartbeat(true);
+    MockAM am0 = rm.sendAMLaunched(attempt0.getAppAttemptId());
+    am0.registerAppAttempt();
+
+    RMContainer rmContainer =
+        rm.getResourceScheduler()
+           .getRMContainer(attempt0.getMasterContainer().getId());
+
+    // Allow metrics to accumulate.
+    Thread.sleep(1000);
+
+    rmAppMetrics = app0.getRMAppMetrics();
+    Assert.assertTrue(
+        "While app is running, memory seconds should be >0 but is "
+            + rmAppMetrics.getMemorySeconds(),
+        rmAppMetrics.getMemorySeconds() > 0);
+    Assert.assertTrue(
+        "While app is running, vcore seconds should be >0 but is "
+            + rmAppMetrics.getVcoreSeconds(),
+        rmAppMetrics.getVcoreSeconds() > 0);
+
+    MockRM.finishAMAndVerifyAppState(app0, rm, nm, am0);
+
+    AggregateAppResourceUsage ru = calculateContainerResourceMetrics(rmContainer);
+    rmAppMetrics = app0.getRMAppMetrics();
+
+    Assert.assertEquals("Unexcpected MemorySeconds value",
+        ru.getMemorySeconds(), rmAppMetrics.getMemorySeconds());
+    Assert.assertEquals("Unexpected VcoreSeconds value",
+        ru.getVcoreSeconds(), rmAppMetrics.getVcoreSeconds());
+
+    rm.stop();
+  }
+
+  @Test (timeout = 60000)
+  public void testUsageWithMultipleContainersAndRMRestart() throws Exception {
+    // Set max attempts to 1 so that when the first attempt fails, the app
+    // won't try to start a new one.
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+
+    MockRM rm0 = new MockRM(conf, memStore);
+    rm0.start();
+    MockNM nm =
+        new MockNM("127.0.0.1:1234", 65536, rm0.getResourceTrackerService());
+    nm.registerNode();
+
+    RMApp app0 = rm0.submitApp(200);
+
+    rm0.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED);
+    RMAppAttempt attempt0 = app0.getCurrentAppAttempt();
+    ApplicationAttemptId attemptId0 = attempt0.getAppAttemptId();
+    rm0.waitForState(attemptId0, RMAppAttemptState.SCHEDULED);
+
+    nm.nodeHeartbeat(true);
+    rm0.waitForState(attemptId0, RMAppAttemptState.ALLOCATED);
+    MockAM am0 = rm0.sendAMLaunched(attempt0.getAppAttemptId());
+    am0.registerAppAttempt();
+
+    int NUM_CONTAINERS = 2;
+    am0.allocate("127.0.0.1" , 1000, NUM_CONTAINERS,
+                  new ArrayList<ContainerId>());
+    nm.nodeHeartbeat(true);
+    List<Container> conts = am0.allocate(new ArrayList<ResourceRequest>(),
+        new ArrayList<ContainerId>()).getAllocatedContainers();
+    while (conts.size() != NUM_CONTAINERS) {
+      nm.nodeHeartbeat(true);
+      conts.addAll(am0.allocate(new ArrayList<ResourceRequest>(),
+          new ArrayList<ContainerId>()).getAllocatedContainers());
+      Thread.sleep(500);
+    }
+
+    // launch the 2nd and 3rd containers.
+    for (Container c : conts) {
+      nm.nodeHeartbeat(attempt0.getAppAttemptId(),
+                       c.getId().getId(), ContainerState.RUNNING);
+      rm0.waitForState(nm, c.getId(), RMContainerState.RUNNING);
+    }
+
+    // Get the RMContainers for all of the live containers, to be used later
+    // for metrics calculations and comparisons.
+    Collection<RMContainer> rmContainers =
+        rm0.scheduler
+            .getSchedulerAppInfo(attempt0.getAppAttemptId())
+              .getLiveContainers();
+
+    // Give the metrics time to accumulate.
+    Thread.sleep(1000);
+
+    // Stop all non-AM containers
+    for (Container c : conts) {
+      if (c.getId().getId() == 1) continue;
+      nm.nodeHeartbeat(attempt0.getAppAttemptId(),
+                       c.getId().getId(), ContainerState.COMPLETE);
+      rm0.waitForState(nm, c.getId(), RMContainerState.COMPLETED);
+    }
+
+    // After all other containers have completed, manually complete the master
+    // container in order to trigger a save to the state store of the resource
+    // usage metrics. This will cause the attempt to fail, and, since the max
+    // attempt retries is 1, the app will also fail. This is intentional so
+    // that all containers will complete prior to saving.
+    ContainerId cId = ContainerId.newInstance(attempt0.getAppAttemptId(), 1);
+    nm.nodeHeartbeat(attempt0.getAppAttemptId(),
+                 cId.getId(), ContainerState.COMPLETE);
+    rm0.waitForState(nm, cId, RMContainerState.COMPLETED);
+
+    // Check that the container metrics match those from the app usage report.
+    long memorySeconds = 0;
+    long vcoreSeconds = 0;
+    for (RMContainer c : rmContainers) {
+      AggregateAppResourceUsage ru = calculateContainerResourceMetrics(c);
+      memorySeconds += ru.getMemorySeconds();
+      vcoreSeconds += ru.getVcoreSeconds();
+    }
+
+    RMAppMetrics metricsBefore = app0.getRMAppMetrics();
+    Assert.assertEquals("Unexcpected MemorySeconds value",
+        memorySeconds, metricsBefore.getMemorySeconds());
+    Assert.assertEquals("Unexpected VcoreSeconds value",
+        vcoreSeconds, metricsBefore.getVcoreSeconds());
+
+    // create new RM to represent RM restart. Load up the state store.
+    MockRM rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    RMApp app0After =
+        rm1.getRMContext().getRMApps().get(app0.getApplicationId());
+
+    // Compare container resource usage metrics from before and after restart.
+    RMAppMetrics metricsAfter = app0After.getRMAppMetrics();
+    Assert.assertEquals("Vcore seconds were not the same after RM Restart",
+        metricsBefore.getVcoreSeconds(), metricsAfter.getVcoreSeconds());
+    Assert.assertEquals("Memory seconds were not the same after RM Restart",
+        metricsBefore.getMemorySeconds(), metricsAfter.getMemorySeconds());
+
+    rm0.stop();
+    rm0.close();
+    rm1.stop();
+    rm1.close();
+  }
+
+  @Test(timeout = 60000)
+  public void testUsageAfterAMRestartWithMultipleContainers() throws Exception {
+    amRestartTests(false);
+  }
+
+  @Test(timeout = 60000)
+  public void testUsageAfterAMRestartKeepContainers() throws Exception {
+    amRestartTests(true);
+  }
+
+  private void amRestartTests(boolean keepRunningContainers)
+      throws Exception {
+    MockRM rm = new MockRM(conf);
+    rm.start();
+
+    RMApp app =
+        rm.submitApp(200, "name", "user",
+          new HashMap<ApplicationAccessType, String>(), false, "default", -1,
+          null, "MAPREDUCE", false, keepRunningContainers);
+    MockNM nm = 
+        new MockNM("127.0.0.1:1234", 10240, rm.getResourceTrackerService());
+    nm.registerNode();
+
+    MockAM am0 = MockRM.launchAndRegisterAM(app, rm, nm);
+    int NUM_CONTAINERS = 1;
+    // allocate NUM_CONTAINERS containers
+    am0.allocate("127.0.0.1", 1024, NUM_CONTAINERS,
+      new ArrayList<ContainerId>());
+    nm.nodeHeartbeat(true);
+
+    // wait for containers to be allocated.
+    List<Container> containers =
+        am0.allocate(new ArrayList<ResourceRequest>(),
+          new ArrayList<ContainerId>()).getAllocatedContainers();
+    while (containers.size() != NUM_CONTAINERS) {
+      nm.nodeHeartbeat(true);
+      containers.addAll(am0.allocate(new ArrayList<ResourceRequest>(),
+        new ArrayList<ContainerId>()).getAllocatedContainers());
+      Thread.sleep(200);
+    }   
+
+    // launch the 2nd container.
+    ContainerId containerId2 =
+        ContainerId.newInstance(am0.getApplicationAttemptId(), 2);
+    nm.nodeHeartbeat(am0.getApplicationAttemptId(),
+                      containerId2.getId(), ContainerState.RUNNING);
+    rm.waitForState(nm, containerId2, RMContainerState.RUNNING);
+
+    // Capture the containers here so the metrics can be calculated after the
+    // app has completed.
+    Collection<RMContainer> rmContainers =
+        rm.scheduler
+            .getSchedulerAppInfo(am0.getApplicationAttemptId())
+              .getLiveContainers();
+
+    // fail the first app attempt by sending CONTAINER_FINISHED event without
+    // registering.
+    ContainerId amContainerId =
+        app.getCurrentAppAttempt().getMasterContainer().getId();
+    nm.nodeHeartbeat(am0.getApplicationAttemptId(),
+                      amContainerId.getId(), ContainerState.COMPLETE);
+    am0.waitForState(RMAppAttemptState.FAILED);
+
+    long memorySeconds = 0;
+    long vcoreSeconds = 0;
+
+    // Calculate container usage metrics for first attempt.
+    if (keepRunningContainers) {
+      // Only calculate the usage for the one container that has completed.
+      for (RMContainer c : rmContainers) {
+        if (c.getContainerId().equals(amContainerId)) {
+          AggregateAppResourceUsage ru = calculateContainerResourceMetrics(c);
+          memorySeconds += ru.getMemorySeconds();
+          vcoreSeconds += ru.getVcoreSeconds();
+        } else {
+          // The remaining container should be RUNNING.
+          Assert.assertTrue("After first attempt failed, remaining container "
+                        + "should still be running. ",
+                        c.getContainerState().equals(ContainerState.RUNNING));
+        }
+      }
+    } else {
+      // If keepRunningContainers is false, all live containers should now
+      // be completed. Calculate the resource usage metrics for all of them.
+      for (RMContainer c : rmContainers) { 
+        AggregateAppResourceUsage ru = calculateContainerResourceMetrics(c);
+        memorySeconds += ru.getMemorySeconds();
+        vcoreSeconds += ru.getVcoreSeconds();
+      }
+    }
+
+    // wait for app to start a new attempt.
+    rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED);
+
+    // assert this is a new AM.
+    RMAppAttempt attempt2 = app.getCurrentAppAttempt();
+    Assert.assertFalse(attempt2.getAppAttemptId()
+                               .equals(am0.getApplicationAttemptId()));
+
+    // launch the new AM
+    nm.nodeHeartbeat(true);
+    MockAM am1 = rm.sendAMLaunched(attempt2.getAppAttemptId());
+    am1.registerAppAttempt();
+    
+    // allocate NUM_CONTAINERS containers
+    am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS,
+      new ArrayList<ContainerId>());
+    nm.nodeHeartbeat(true);
+
+    // wait for containers to be allocated.
+    containers =
+        am1.allocate(new ArrayList<ResourceRequest>(),
+          new ArrayList<ContainerId>()).getAllocatedContainers();
+    while (containers.size() != NUM_CONTAINERS) {
+      nm.nodeHeartbeat(true);
+      containers.addAll(am1.allocate(new ArrayList<ResourceRequest>(),
+        new ArrayList<ContainerId>()).getAllocatedContainers());
+      Thread.sleep(200);
+    }
+
+    rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
+    
+    // Capture running containers for later use by metrics calculations.
+    rmContainers = rm.scheduler.getSchedulerAppInfo(attempt2.getAppAttemptId())
+                               .getLiveContainers();
+      
+    // complete container by sending the container complete event which has
+    // earlier attempt's attemptId
+    amContainerId = app.getCurrentAppAttempt().getMasterContainer().getId();
+    nm.nodeHeartbeat(am0.getApplicationAttemptId(),
+                      amContainerId.getId(), ContainerState.COMPLETE);
+    
+    MockRM.finishAMAndVerifyAppState(app, rm, nm, am1);
+
+    // Calculate container usage metrics for second attempt.
+    for (RMContainer c : rmContainers) {
+      AggregateAppResourceUsage ru = calculateContainerResourceMetrics(c);
+      memorySeconds += ru.getMemorySeconds();
+      vcoreSeconds += ru.getVcoreSeconds();
+    }
+    
+    RMAppMetrics rmAppMetrics = app.getRMAppMetrics();
+
+    Assert.assertEquals("Unexcpected MemorySeconds value",
+        memorySeconds, rmAppMetrics.getMemorySeconds());
+    Assert.assertEquals("Unexpected VcoreSeconds value",
+        vcoreSeconds, rmAppMetrics.getVcoreSeconds());
+
+    rm.stop();
+    return;
+  }
+
+  private AggregateAppResourceUsage calculateContainerResourceMetrics(
+      RMContainer rmContainer) {
+    Resource resource = rmContainer.getContainer().getResource();
+    long usedMillis =
+        rmContainer.getFinishTime() - rmContainer.getCreationTime();
+    long memorySeconds = resource.getMemory()
+                          * usedMillis / DateUtils.MILLIS_PER_SECOND;
+    long vcoreSeconds = resource.getVirtualCores()
+                          * usedMillis / DateUtils.MILLIS_PER_SECOND;
+    return new AggregateAppResourceUsage(memorySeconds, vcoreSeconds);
+  }
+}

+ 18 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.MockApps;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -170,7 +171,7 @@ public abstract class MockAsm extends MockApps {
 
     @Override
     public RMAppMetrics getRMAppMetrics() {
-      return new RMAppMetrics(Resource.newInstance(0, 0), 0, 0);
+      return new RMAppMetrics(Resource.newInstance(0, 0), 0, 0, 0, 0);
     }
   }
 
@@ -259,6 +260,22 @@ public abstract class MockAsm extends MockApps {
       public Set<String> getApplicationTags() {
         return null;
       }
+
+      @Override
+      public ApplicationReport createAndGetApplicationReport(
+          String clientUserName, boolean allowAccess) {
+        ApplicationResourceUsageReport usageReport =
+            ApplicationResourceUsageReport.newInstance(0, 0, null, null, null, 
+            0, 0);
+        ApplicationReport report = ApplicationReport.newInstance(
+            getApplicationId(), appAttemptId, getUser(), getQueue(), 
+            getName(), null, 0, null, null, getDiagnostics().toString(), 
+            getTrackingUrl(), getStartTime(), getFinishTime(), 
+            getFinalApplicationStatus(), usageReport , null, getProgress(),
+            type, null);
+        return report;
+      }
+
     };
   }
 

+ 16 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java

@@ -64,7 +64,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMSta
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.AMRMTokenSecretManagerState;
 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.AggregateAppResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
@@ -152,6 +154,8 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
       SecretKey clientTokenMasterKey, TestDispatcher dispatcher)
       throws Exception {
 
+    RMAppAttemptMetrics mockRmAppAttemptMetrics = 
+        mock(RMAppAttemptMetrics.class);
     Container container = new ContainerPBImpl();
     container.setId(ConverterUtils.toContainerId(containerIdStr));
     RMAppAttempt mockAttempt = mock(RMAppAttempt.class);
@@ -160,6 +164,10 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
     when(mockAttempt.getAMRMToken()).thenReturn(appToken);
     when(mockAttempt.getClientTokenMasterKey())
         .thenReturn(clientTokenMasterKey);
+    when(mockAttempt.getRMAppAttemptMetrics())
+        .thenReturn(mockRmAppAttemptMetrics);
+    when(mockRmAppAttemptMetrics.getAggregateAppResourceUsage())
+        .thenReturn(new AggregateAppResourceUsage(0,0));
     dispatcher.attemptId = attemptId;
     store.storeNewApplicationAttempt(mockAttempt);
     waitNotify(dispatcher);
@@ -224,6 +232,8 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
         "container_1352994193343_0002_01_000001", null, null, dispatcher);
 
     RMApp mockRemovedApp = mock(RMApp.class);
+    RMAppAttemptMetrics mockRmAppAttemptMetrics = 
+        mock(RMAppAttemptMetrics.class);
     HashMap<ApplicationAttemptId, RMAppAttempt> attempts =
                               new HashMap<ApplicationAttemptId, RMAppAttempt>();
     ApplicationSubmissionContext context =
@@ -234,6 +244,10 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
     when(mockRemovedApp.getAppAttempts()).thenReturn(attempts);
     RMAppAttempt mockRemovedAttempt = mock(RMAppAttempt.class);
     when(mockRemovedAttempt.getAppAttemptId()).thenReturn(attemptIdRemoved);
+    when(mockRemovedAttempt.getRMAppAttemptMetrics())
+        .thenReturn(mockRmAppAttemptMetrics);
+    when(mockRmAppAttemptMetrics.getAggregateAppResourceUsage())
+        .thenReturn(new AggregateAppResourceUsage(0,0));
     attempts.put(attemptIdRemoved, mockRemovedAttempt);
     store.removeApplication(mockRemovedApp);
 
@@ -304,7 +318,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
           oldAttemptState.getAppAttemptCredentials(),
           oldAttemptState.getStartTime(), RMAppAttemptState.FINISHED,
           "myTrackingUrl", "attemptDiagnostics",
-          FinalApplicationStatus.SUCCEEDED, 100);
+          FinalApplicationStatus.SUCCEEDED, 100, 0, 0);
     store.updateApplicationAttemptState(newAttemptState);
 
     // test updating the state of an app/attempt whose initial state was not
@@ -327,7 +341,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
           oldAttemptState.getAppAttemptCredentials(),
           oldAttemptState.getStartTime(), RMAppAttemptState.FINISHED,
           "myTrackingUrl", "attemptDiagnostics",
-          FinalApplicationStatus.SUCCEEDED, 111);
+          FinalApplicationStatus.SUCCEEDED, 111, 0, 0);
     store.updateApplicationAttemptState(dummyAttempt);
 
     // let things settle down

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

@@ -19,8 +19,10 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
 import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
@@ -38,6 +40,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.yarn.MockApps;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -61,6 +64,7 @@ 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.rmcontainer.ContainerAllocationExpirer;
+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.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -74,6 +78,7 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.mockito.ArgumentCaptor;
+import org.mockito.Matchers;
 
 
 @RunWith(value = Parameterized.class)
@@ -189,7 +194,7 @@ public class TestRMAppTransitions {
     AMLivelinessMonitor amFinishingMonitor = mock(AMLivelinessMonitor.class);
     store = mock(RMStateStore.class);
     writer = mock(RMApplicationHistoryWriter.class);
-    this.rmContext =
+    RMContext realRMContext = 
         new RMContextImpl(rmDispatcher,
           containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
           null, new AMRMTokenSecretManager(conf, this.rmContext),
@@ -197,7 +202,14 @@ public class TestRMAppTransitions {
           new NMTokenSecretManagerInRM(conf),
           new ClientToAMTokenSecretManagerInRM(),
           writer);
-    ((RMContextImpl)rmContext).setStateStore(store);
+    ((RMContextImpl)realRMContext).setStateStore(store);
+
+    this.rmContext = spy(realRMContext);
+
+    ResourceScheduler resourceScheduler = mock(ResourceScheduler.class);
+    doReturn(null).when(resourceScheduler)
+              .getAppResourceUsageReport((ApplicationAttemptId)Matchers.any());
+    doReturn(resourceScheduler).when(rmContext).getScheduler();
 
     rmDispatcher.register(RMAppAttemptEventType.class,
         new TestApplicationAttemptEventDispatcher(this.rmContext));

+ 67 - 5
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

@@ -25,6 +25,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
 import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -49,6 +50,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.MockApps;
 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.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -87,6 +89,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAlloca
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 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.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
@@ -107,6 +111,8 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.mockito.ArgumentCaptor;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
 
 @RunWith(value = Parameterized.class)
 public class TestRMAppAttemptTransitions {
@@ -120,7 +126,9 @@ public class TestRMAppAttemptTransitions {
   
   private boolean isSecurityEnabled;
   private RMContext rmContext;
+  private RMContext spyRMContext;
   private YarnScheduler scheduler;
+  private ResourceScheduler resourceScheduler;
   private ApplicationMasterService masterService;
   private ApplicationMasterLauncher applicationMasterLauncher;
   private AMLivelinessMonitor amLivelinessMonitor;
@@ -262,7 +270,20 @@ public class TestRMAppAttemptTransitions {
     ApplicationId applicationId = MockApps.newAppID(appId++);
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, 0);
-    
+
+    resourceScheduler = mock(ResourceScheduler.class);
+
+    ApplicationResourceUsageReport appResUsgRpt =
+        mock(ApplicationResourceUsageReport.class);
+    when(appResUsgRpt.getMemorySeconds()).thenReturn(0L);
+    when(appResUsgRpt.getVcoreSeconds()).thenReturn(0L);
+    when(resourceScheduler
+        .getAppResourceUsageReport((ApplicationAttemptId)Matchers.any()))
+     .thenReturn(appResUsgRpt);
+    spyRMContext = spy(rmContext);
+    Mockito.doReturn(resourceScheduler).when(spyRMContext).getScheduler();
+
+
     final String user = MockApps.newUserName();
     final String queue = MockApps.newQueue();
     submissionContext = mock(ApplicationSubmissionContext.class);
@@ -278,17 +299,18 @@ public class TestRMAppAttemptTransitions {
     
     application = mock(RMAppImpl.class);
     applicationAttempt =
-        new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler,
+        new RMAppAttemptImpl(applicationAttemptId, spyRMContext, scheduler,
           masterService, submissionContext, new Configuration(), false);
     when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt);
     when(application.getApplicationId()).thenReturn(applicationId);
-    
+    spyRMContext.getRMApps().put(application.getApplicationId(), application);
+
     testAppAttemptNewState();
   }
 
   @After
   public void tearDown() throws Exception {
-    ((AsyncDispatcher)this.rmContext.getDispatcher()).stop();
+    ((AsyncDispatcher)this.spyRMContext.getDispatcher()).stop();
   }
   
 
@@ -698,6 +720,46 @@ public class TestRMAppAttemptTransitions {
           RMAppAttemptEventType.ATTEMPT_UPDATE_SAVED));
   }
 
+  @Test
+  public void testUsageReport() {
+    // scheduler has info on running apps
+    ApplicationAttemptId attemptId = applicationAttempt.getAppAttemptId();
+    ApplicationResourceUsageReport appResUsgRpt =
+            mock(ApplicationResourceUsageReport.class);
+    when(appResUsgRpt.getMemorySeconds()).thenReturn(123456L);
+    when(appResUsgRpt.getVcoreSeconds()).thenReturn(55544L);
+    when(scheduler.getAppResourceUsageReport(any(ApplicationAttemptId.class)))
+    .thenReturn(appResUsgRpt);
+
+    // start and finish the attempt
+    Container amContainer = allocateApplicationAttempt();
+    launchApplicationAttempt(amContainer);
+    runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
+    applicationAttempt.handle(new RMAppAttemptUnregistrationEvent(attemptId,
+        "", FinalApplicationStatus.SUCCEEDED, ""));
+
+    // expect usage stats to come from the scheduler report
+    ApplicationResourceUsageReport report = 
+        applicationAttempt.getApplicationResourceUsageReport();
+    Assert.assertEquals(123456L, report.getMemorySeconds());
+    Assert.assertEquals(55544L, report.getVcoreSeconds());
+
+    // finish app attempt and remove it from scheduler 
+    when(appResUsgRpt.getMemorySeconds()).thenReturn(223456L);
+    when(appResUsgRpt.getVcoreSeconds()).thenReturn(75544L);
+    sendAttemptUpdateSavedEvent(applicationAttempt);
+    applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
+        attemptId, 
+        ContainerStatus.newInstance(
+            amContainer.getId(), ContainerState.COMPLETE, "", 0)));
+
+    when(scheduler.getSchedulerAppInfo(eq(attemptId))).thenReturn(null);
+
+    report = applicationAttempt.getApplicationResourceUsageReport();
+    Assert.assertEquals(223456, report.getMemorySeconds());
+    Assert.assertEquals(75544, report.getVcoreSeconds());
+  }
+
   @Test
   public void testUnmanagedAMUnexpectedRegistration() {
     unmanagedAM = true;
@@ -1243,7 +1305,7 @@ public class TestRMAppAttemptTransitions {
   public void testContainersCleanupForLastAttempt() {
     // create a failed attempt.
     applicationAttempt =
-        new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), rmContext,
+        new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), spyRMContext,
           scheduler, masterService, submissionContext, new Configuration(),
           true);
     when(submissionContext.getKeepContainersAcrossApplicationAttempts())

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java

@@ -23,10 +23,13 @@ import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -59,6 +62,8 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
 public class TestRMContainerImpl {
@@ -86,12 +91,18 @@ public class TestRMContainerImpl {
 
     Container container = BuilderUtils.newContainer(containerId, nodeId,
         "host:3465", resource, priority, null);
+    ConcurrentMap<ApplicationId, RMApp> rmApps =
+        spy(new ConcurrentHashMap<ApplicationId, RMApp>());
+    RMApp rmApp = mock(RMApp.class);
+    when(rmApp.getRMAppAttempt((ApplicationAttemptId)Matchers.any())).thenReturn(null);
+    Mockito.doReturn(rmApp).when(rmApps).get((ApplicationId)Matchers.any());
 
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
     RMContext rmContext = mock(RMContext.class);
     when(rmContext.getDispatcher()).thenReturn(drainDispatcher);
     when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer);
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
+    when(rmContext.getRMApps()).thenReturn(rmApps);
     RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
         nodeId, "user", rmContext);
 

+ 52 - 37
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

@@ -39,12 +39,15 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.junit.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -59,6 +62,7 @@ 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.RMApp;
 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;
@@ -79,6 +83,8 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -90,6 +96,7 @@ public class TestLeafQueue {
       RecordFactoryProvider.getRecordFactory(null);
 
   RMContext rmContext;
+  RMContext spyRMContext;
   CapacityScheduler cs;
   CapacitySchedulerConfiguration csConf;
   CapacitySchedulerContext csContext;
@@ -107,6 +114,14 @@ public class TestLeafQueue {
     CapacityScheduler spyCs = new CapacityScheduler();
     cs = spy(spyCs);
     rmContext = TestUtils.getMockRMContext();
+    spyRMContext = spy(rmContext);
+
+    ConcurrentMap<ApplicationId, RMApp> spyApps = 
+        spy(new ConcurrentHashMap<ApplicationId, RMApp>());
+    RMApp rmApp = mock(RMApp.class);
+    when(rmApp.getRMAppAttempt((ApplicationAttemptId)Matchers.any())).thenReturn(null);
+    Mockito.doReturn(rmApp).when(spyApps).get((ApplicationId)Matchers.any());
+    when(spyRMContext.getRMApps()).thenReturn(spyApps);
     
     csConf = 
         new CapacitySchedulerConfiguration();
@@ -143,7 +158,7 @@ public class TestLeafQueue {
             queues, queues, 
             TestUtils.spyHook);
 
-    cs.setRMContext(rmContext);
+    cs.setRMContext(spyRMContext);
     cs.init(csConf);
     cs.start();
   }
@@ -280,14 +295,14 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0);  // same user
 
     
@@ -329,14 +344,14 @@ public class TestLeafQueue {
     final ApplicationAttemptId appAttemptId_0 = TestUtils
         .getMockApplicationAttemptId(0, 1);
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_d, d, null,
-        rmContext);
+        spyRMContext);
     d.submitApplicationAttempt(app_0, user_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);
+        spyRMContext);
     d.submitApplicationAttempt(app_1, user_d); // same user
   }
 
@@ -373,7 +388,7 @@ public class TestLeafQueue {
     final ApplicationAttemptId appAttemptId_1 = TestUtils
         .getMockApplicationAttemptId(0, 2);
     FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a, null,
-        rmContext);
+        spyRMContext);
     a.submitApplicationAttempt(app_1, user_0); // same user
 
     assertEquals(1, a.getMetrics().getAppsSubmitted());
@@ -411,14 +426,14 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0);  // same user
 
     
@@ -545,21 +560,21 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0);  // same user
 
     final ApplicationAttemptId appAttemptId_2 = 
         TestUtils.getMockApplicationAttemptId(2, 0); 
     FiCaSchedulerApp app_2 = 
         new FiCaSchedulerApp(appAttemptId_2, user_1, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_2, user_1);
 
     // Setup some nodes
@@ -639,21 +654,21 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0);  // same user
 
     final ApplicationAttemptId appAttemptId_2 = 
         TestUtils.getMockApplicationAttemptId(2, 0); 
     FiCaSchedulerApp app_2 = 
         new FiCaSchedulerApp(appAttemptId_2, user_1, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_2, user_1);
 
     // Setup some nodes
@@ -750,28 +765,28 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0);  // same user
 
     final ApplicationAttemptId appAttemptId_2 = 
         TestUtils.getMockApplicationAttemptId(2, 0); 
     FiCaSchedulerApp app_2 = 
         new FiCaSchedulerApp(appAttemptId_2, user_1, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_2, user_1);
 
     final ApplicationAttemptId appAttemptId_3 = 
         TestUtils.getMockApplicationAttemptId(3, 0); 
     FiCaSchedulerApp app_3 = 
         new FiCaSchedulerApp(appAttemptId_3, user_2, a, 
-            a.getActiveUsersManager(), rmContext);
+            a.getActiveUsersManager(), spyRMContext);
     a.submitApplicationAttempt(app_3, user_2);
     
     // Setup some nodes
@@ -935,14 +950,14 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_1, a, 
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_1);  
 
     // Setup some nodes
@@ -1043,14 +1058,14 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 =
         new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 =
         TestUtils.getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 =
         new FiCaSchedulerApp(appAttemptId_1, user_1, a,
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_1);
 
     // Setup some nodes
@@ -1150,14 +1165,14 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         new FiCaSchedulerApp(appAttemptId_1, user_1, a, 
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_1);  
 
     // Setup some nodes
@@ -1277,7 +1292,7 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext));
+            mock(ActiveUsersManager.class), spyRMContext));
     a.submitApplicationAttempt(app_0, user_0);
     
     // Setup some nodes and racks
@@ -1418,7 +1433,7 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext));
+            mock(ActiveUsersManager.class), spyRMContext));
     a.submitApplicationAttempt(app_0, user_0);
     
     // Setup some nodes and racks
@@ -1549,7 +1564,7 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext));
+            mock(ActiveUsersManager.class), spyRMContext));
     a.submitApplicationAttempt(app_0, user_0);
     
     // Setup some nodes and racks
@@ -1652,21 +1667,21 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 =
         new FiCaSchedulerApp(appAttemptId_0, user_e, e,
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     e.submitApplicationAttempt(app_0, user_e);
 
     final ApplicationAttemptId appAttemptId_1 =
         TestUtils.getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 =
         new FiCaSchedulerApp(appAttemptId_1, user_e, e,
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     e.submitApplicationAttempt(app_1, user_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);
+            mock(ActiveUsersManager.class), spyRMContext);
     e.submitApplicationAttempt(app_2, user_e);  // same user
 
     // before reinitialization
@@ -1730,21 +1745,21 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 =
         new FiCaSchedulerApp(appAttemptId_0, user_e, e,
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     e.submitApplicationAttempt(app_0, user_e);
 
     final ApplicationAttemptId appAttemptId_1 =
         TestUtils.getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 =
         new FiCaSchedulerApp(appAttemptId_1, user_e, e,
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     e.submitApplicationAttempt(app_1, user_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);
+            mock(ActiveUsersManager.class), spyRMContext);
     e.submitApplicationAttempt(app_2, user_e);  // same user
 
     // before updating cluster resource
@@ -1807,14 +1822,14 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0); 
     FiCaSchedulerApp app_0 = 
         spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext));
+            mock(ActiveUsersManager.class), spyRMContext));
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
     FiCaSchedulerApp app_1 = 
         spy(new FiCaSchedulerApp(appAttemptId_1, user_0, a, 
-            mock(ActiveUsersManager.class), rmContext));
+            mock(ActiveUsersManager.class), spyRMContext));
     a.submitApplicationAttempt(app_1, user_0);
 
     // Setup some nodes and racks
@@ -2062,14 +2077,14 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 =
         new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_0, user_0);
 
     final ApplicationAttemptId appAttemptId_1 =
         TestUtils.getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 =
         new FiCaSchedulerApp(appAttemptId_1, user_0, a,
-            mock(ActiveUsersManager.class), rmContext);
+            mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0); // same user
 
     // Setup some nodes

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

@@ -162,7 +162,7 @@ public class FairSchedulerTestBase {
     RMAppAttempt rmAppAttempt = mock(RMAppAttempt.class);
     when(rmApp.getCurrentAppAttempt()).thenReturn(rmAppAttempt);
     when(rmAppAttempt.getRMAppAttemptMetrics()).thenReturn(
-        new RMAppAttemptMetrics(id));
+        new RMAppAttemptMetrics(id, resourceManager.getRMContext()));
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
     return id;
@@ -183,7 +183,7 @@ public class FairSchedulerTestBase {
     RMAppAttempt rmAppAttempt = mock(RMAppAttempt.class);
     when(rmApp.getCurrentAppAttempt()).thenReturn(rmAppAttempt);
     when(rmAppAttempt.getRMAppAttemptMetrics()).thenReturn(
-        new RMAppAttemptMetrics(id));
+        new RMAppAttemptMetrics(id,resourceManager.getRMContext()));
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
     return id;

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

@@ -136,7 +136,7 @@ public class TestRMWebAppFairScheduler {
       MockRMApp app = new MockRMApp(i, i, state) {
         @Override
         public RMAppMetrics getRMAppMetrics() {
-          return new RMAppMetrics(Resource.newInstance(0, 0), 0, 0);
+          return new RMAppMetrics(Resource.newInstance(0, 0), 0, 0, 0, 0);
         }
         @Override
         public YarnApplicationState createApplicationState() {

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

@@ -1322,7 +1322,7 @@ public class TestRMWebServicesApps extends JerseyTest {
       Exception {
 
     // 28 because trackingUrl not assigned yet
-    assertEquals("incorrect number of elements", 24, info.length());
+    assertEquals("incorrect number of elements", 26, info.length());
 
     verifyAppInfoGeneric(app, info.getString("id"), info.getString("user"),
         info.getString("name"), info.getString("applicationType"),

+ 19 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm

@@ -1197,7 +1197,9 @@ ResourceManager REST API's.
           "queue" : "default",
           "allocatedMB" : 0,
           "allocatedVCores" : 0,
-          "runningContainers" : 0
+          "runningContainers" : 0,
+          "memorySeconds" : 151730,
+          "vcoreSeconds" : 103
        },
        {
           "finishedTime" : 1326815789546,
@@ -1218,7 +1220,9 @@ ResourceManager REST API's.
           "queue" : "default",
           "allocatedMB" : 0,
           "allocatedVCores" : 0,
-          "runningContainers" : 1
+          "runningContainers" : 1,
+          "memorySeconds" : 640064,
+          "vcoreSeconds" : 442
        } 
     ]
   }
@@ -1271,6 +1275,8 @@ _01_000001</amContainerLogs>
     <allocatedMB>0</allocatedMB>
     <allocatedVCores>0</allocatedVCores>
     <runningContainers>0</runningContainers>
+    <memorySeconds>151730</memorySeconds>
+    <vcoreSeconds>103</vcoreSeconds>
   </app>
   <app>
     <id>application_1326815542473_0002</id>
@@ -1293,6 +1299,8 @@ _01_000001</amContainerLogs>
     <allocatedMB>0</allocatedMB>
     <allocatedVCores>0</allocatedVCores>
     <runningContainers>0</runningContainers>
+    <memorySeconds>640064</memorySeconds>
+    <vcoreSeconds>442</vcoreSeconds>
   </app>
 </apps>
 +---+
@@ -1491,6 +1499,10 @@ _01_000001</amContainerLogs>
 +---------------------------------------------------------------+
 | runningContainers | int | The number of containers currently running for the application |
 +---------------------------------------------------------------+
+| memorySeconds | long | The amount of memory the application has allocated (megabyte-seconds) |
+*---------------+--------------+--------------------------------+
+| vcoreSeconds | long | The amount of CPU resources the application has allocated (virtual core-seconds) |
+*---------------+--------------+--------------------------------+
 
 ** Response Examples
 
@@ -1532,7 +1544,9 @@ _01_000001</amContainerLogs>
       "elapsedTime" : 446748,
       "diagnostics" : "",
       "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326821518301_0005/jobhistory/job/job_1326821518301_5_5",
-      "queue" : "a1"
+      "queue" : "a1",
+      "memorySeconds" : 151730,
+      "vcoreSeconds" : 103
    }
 }
 +---+
@@ -1576,6 +1590,8 @@ _01_000001</amContainerLogs>
   <elapsedTime>446748</elapsedTime>
   <amContainerLogs>http://host.domain.com:8042/node/containerlogs/container_1326821518301_0005_01_000001</amContainerLogs>
   <amHostHttpAddress>host.domain.com:8042</amHostHttpAddress>
+  <memorySeconds>151730</memorySeconds>
+  <vcoreSeconds>103</vcoreSeconds>
 </app>
 +---+