浏览代码

YARN-3543. ApplicationReport should be able to tell whether the
Application is AM managed or not. Contributed by Rohith Sharma K S

(cherry picked from commit 0306d902f53582320aa5895ca9f5c31f64aaaff6)

Xuan 9 年之前
父节点
当前提交
8126fde884
共有 21 个文件被更改,包括 136 次插入25 次删除
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 16 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
  3. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  4. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
  5. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
  6. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
  7. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
  8. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicatonReport.java
  9. 11 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
  10. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
  11. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java
  12. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
  13. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
  14. 8 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java
  15. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
  16. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  17. 7 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
  18. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
  19. 10 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
  20. 10 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
  21. 16 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md

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

@@ -311,6 +311,9 @@ Release 2.8.0 - UNRELEASED
 
 
     YARN-3965. Add startup timestamp to nodemanager UI (Hong Zhiguo via jlowe)
     YARN-3965. Add startup timestamp to nodemanager UI (Hong Zhiguo via jlowe)
 
 
+    YARN-3543. ApplicationReport should be able to tell whether the Application
+    is AM managed or not. (Rohith Sharma K S via xgong)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

+ 16 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java

@@ -92,13 +92,14 @@ public abstract class ApplicationReport {
       long startTime, long finishTime, FinalApplicationStatus finalStatus,
       long startTime, long finishTime, FinalApplicationStatus finalStatus,
       ApplicationResourceUsageReport appResources, String origTrackingUrl,
       ApplicationResourceUsageReport appResources, String origTrackingUrl,
       float progress, String applicationType, Token amRmToken,
       float progress, String applicationType, Token amRmToken,
-      Set<String> tags) {
+      Set<String> tags, boolean unmanagedApplication) {
     ApplicationReport report =
     ApplicationReport report =
         newInstance(applicationId, applicationAttemptId, user, queue, name,
         newInstance(applicationId, applicationAttemptId, user, queue, name,
           host, rpcPort, clientToAMToken, state, diagnostics, url, startTime,
           host, rpcPort, clientToAMToken, state, diagnostics, url, startTime,
           finishTime, finalStatus, appResources, origTrackingUrl, progress,
           finishTime, finalStatus, appResources, origTrackingUrl, progress,
           applicationType, amRmToken);
           applicationType, amRmToken);
     report.setApplicationTags(tags);
     report.setApplicationTags(tags);
+    report.setUnmanagedApp(unmanagedApplication);
     return report;
     return report;
   }
   }
 
 
@@ -393,4 +394,18 @@ public abstract class ApplicationReport {
   @Unstable
   @Unstable
   public abstract void setLogAggregationStatus(
   public abstract void setLogAggregationStatus(
       LogAggregationStatus logAggregationStatus);
       LogAggregationStatus logAggregationStatus);
+
+  /**
+   * @return true if the AM is not managed by the RM
+   */
+  @Public
+  @Unstable
+  public abstract boolean isUnmanagedApp();
+
+  /**
+   * @param value true if RM should not manage the AM
+   */
+  @Public
+  @Unstable
+  public abstract void setUnmanagedApp(boolean unmanagedApplication);
 }
 }

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

@@ -195,6 +195,7 @@ message ApplicationReportProto {
   optional hadoop.common.TokenProto am_rm_token = 19;
   optional hadoop.common.TokenProto am_rm_token = 19;
   repeated string applicationTags = 20;
   repeated string applicationTags = 20;
   optional LogAggregationStatusProto log_aggregation_status = 21;
   optional LogAggregationStatusProto log_aggregation_status = 21;
+  optional bool unmanaged_application = 22 [default = false];
 }
 }
 
 
 enum LogAggregationStatusProto {
 enum LogAggregationStatusProto {

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java

@@ -534,7 +534,9 @@ public class ApplicationCLI extends YarnCLI {
       appReportStr.println(appReport.getLogAggregationStatus() == null ? "N/A"
       appReportStr.println(appReport.getLogAggregationStatus() == null ? "N/A"
           : appReport.getLogAggregationStatus());
           : appReport.getLogAggregationStatus());
       appReportStr.print("\tDiagnostics : ");
       appReportStr.print("\tDiagnostics : ");
-      appReportStr.print(appReport.getDiagnostics());
+      appReportStr.println(appReport.getDiagnostics());
+      appReportStr.print("\tUnmanaged Application : ");
+      appReportStr.print(appReport.isUnmanagedApp());
     } else {
     } else {
       appReportStr.print("Application with id '" + applicationId
       appReportStr.print("Application with id '" + applicationId
           + "' doesn't exist in RM.");
           + "' doesn't exist in RM.");

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java

@@ -895,12 +895,12 @@ public class TestYarnClient {
       rmClient.start();
       rmClient.start();
 
 
       ApplicationId appId = createApp(rmClient, false);
       ApplicationId appId = createApp(rmClient, false);
-      waitTillAccepted(rmClient, appId);
+      waitTillAccepted(rmClient, appId, false);
       //managed AMs don't return AMRM token
       //managed AMs don't return AMRM token
       Assert.assertNull(rmClient.getAMRMToken(appId));
       Assert.assertNull(rmClient.getAMRMToken(appId));
 
 
       appId = createApp(rmClient, true);
       appId = createApp(rmClient, true);
-      waitTillAccepted(rmClient, appId);
+      waitTillAccepted(rmClient, appId, true);
       long start = System.currentTimeMillis();
       long start = System.currentTimeMillis();
       while (rmClient.getAMRMToken(appId) == null) {
       while (rmClient.getAMRMToken(appId) == null) {
         if (System.currentTimeMillis() - start > 20 * 1000) {
         if (System.currentTimeMillis() - start > 20 * 1000) {
@@ -921,7 +921,7 @@ public class TestYarnClient {
             rmClient.init(yarnConf);
             rmClient.init(yarnConf);
             rmClient.start();
             rmClient.start();
             ApplicationId appId = createApp(rmClient, true);
             ApplicationId appId = createApp(rmClient, true);
-            waitTillAccepted(rmClient, appId);
+          waitTillAccepted(rmClient, appId, true);
             long start = System.currentTimeMillis();
             long start = System.currentTimeMillis();
             while (rmClient.getAMRMToken(appId) == null) {
             while (rmClient.getAMRMToken(appId) == null) {
               if (System.currentTimeMillis() - start > 20 * 1000) {
               if (System.currentTimeMillis() - start > 20 * 1000) {
@@ -981,7 +981,8 @@ public class TestYarnClient {
     return appId;
     return appId;
   }
   }
   
   
-  private void waitTillAccepted(YarnClient rmClient, ApplicationId appId)
+  private void waitTillAccepted(YarnClient rmClient, ApplicationId appId,
+      boolean unmanagedApplication)
     throws Exception {
     throws Exception {
     try {
     try {
       long start = System.currentTimeMillis();
       long start = System.currentTimeMillis();
@@ -994,6 +995,7 @@ public class TestYarnClient {
         Thread.sleep(200);
         Thread.sleep(200);
         report = rmClient.getApplicationReport(appId);
         report = rmClient.getApplicationReport(appId);
       }
       }
+      Assert.assertEquals(unmanagedApplication, report.isUnmanagedApp());
     } catch (Exception ex) {
     } catch (Exception ex) {
       throw new Exception(ex);
       throw new Exception(ex);
     }
     }

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

@@ -104,7 +104,7 @@ public class TestYarnCLI {
           "user", "queue", "appname", "host", 124, null,
           "user", "queue", "appname", "host", 124, null,
           YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
           YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
           FinalApplicationStatus.SUCCEEDED, usageReport, "N/A", 0.53789f, "YARN",
           FinalApplicationStatus.SUCCEEDED, usageReport, "N/A", 0.53789f, "YARN",
-          null);
+          null, null, false);
       newApplicationReport.setLogAggregationStatus(LogAggregationStatus.SUCCEEDED);
       newApplicationReport.setLogAggregationStatus(LogAggregationStatus.SUCCEEDED);
       when(client.getApplicationReport(any(ApplicationId.class))).thenReturn(
       when(client.getApplicationReport(any(ApplicationId.class))).thenReturn(
           newApplicationReport);
           newApplicationReport);
@@ -131,6 +131,7 @@ public class TestYarnCLI {
           (i == 0 ? "N/A" : "123456 MB-seconds, 4567 vcore-seconds"));
           (i == 0 ? "N/A" : "123456 MB-seconds, 4567 vcore-seconds"));
       pw.println("\tLog Aggregation Status : SUCCEEDED");
       pw.println("\tLog Aggregation Status : SUCCEEDED");
       pw.println("\tDiagnostics : diagnostics");
       pw.println("\tDiagnostics : diagnostics");
+      pw.println("\tUnmanaged Application : false");
       pw.close();
       pw.close();
       String appReportStr = baos.toString("UTF-8");
       String appReportStr = baos.toString("UTF-8");
       Assert.assertEquals(appReportStr, sysOutStream.toString());
       Assert.assertEquals(appReportStr, sysOutStream.toString());

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

@@ -581,4 +581,16 @@ public class ApplicationReportPBImpl extends ApplicationReport {
       convertToProtoFormat(LogAggregationStatus s) {
       convertToProtoFormat(LogAggregationStatus s) {
     return ProtoUtils.convertToProtoFormat(s);
     return ProtoUtils.convertToProtoFormat(s);
   }
   }
+
+  @Override
+  public boolean isUnmanagedApp() {
+    ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getUnmanagedApplication();
+  }
+
+  @Override
+  public void setUnmanagedApp(boolean unmanagedApplication) {
+    maybeInitBuilder();
+    builder.setUnmanagedApplication(unmanagedApplication);
+  }
 }
 }

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicatonReport.java

@@ -58,7 +58,8 @@ public class TestApplicatonReport {
         ApplicationReport.newInstance(appId, appAttemptId, "user", "queue",
         ApplicationReport.newInstance(appId, appAttemptId, "user", "queue",
           "appname", "host", 124, null, YarnApplicationState.FINISHED,
           "appname", "host", 124, null, YarnApplicationState.FINISHED,
           "diagnostics", "url", 0, 0, FinalApplicationStatus.SUCCEEDED, null,
           "diagnostics", "url", 0, 0, FinalApplicationStatus.SUCCEEDED, null,
-          "N/A", 0.53789f, YarnConfiguration.DEFAULT_APPLICATION_TYPE, null);
+          "N/A", 0.53789f, YarnConfiguration.DEFAULT_APPLICATION_TYPE, null,
+          null,false);
     return appReport;
     return appReport;
   }
   }
 
 

+ 11 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java

@@ -247,6 +247,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
     String queue = null;
     String queue = null;
     String name = null;
     String name = null;
     String type = null;
     String type = null;
+    boolean unmanagedApplication = false;
     long createdTime = 0;
     long createdTime = 0;
     long finishedTime = 0;
     long finishedTime = 0;
     float progress = 0.0f;
     float progress = 0.0f;
@@ -277,7 +278,8 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
             ConverterUtils.toApplicationId(entity.getEntityId()),
             ConverterUtils.toApplicationId(entity.getEntityId()),
             latestApplicationAttemptId, user, queue, name, null, -1, null, state,
             latestApplicationAttemptId, user, queue, name, null, -1, null, state,
             diagnosticsInfo, null, createdTime, finishedTime, finalStatus, null,
             diagnosticsInfo, null, createdTime, finishedTime, finalStatus, null,
-            null, progress, type, null, appTags), appViewACLs);
+            null, progress, type, null, appTags,
+            unmanagedApplication), appViewACLs);
       }
       }
       if (entityInfo.containsKey(ApplicationMetricsConstants.QUEUE_ENTITY_INFO)) {
       if (entityInfo.containsKey(ApplicationMetricsConstants.QUEUE_ENTITY_INFO)) {
         queue =
         queue =
@@ -294,6 +296,13 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
             entityInfo.get(ApplicationMetricsConstants.TYPE_ENTITY_INFO)
             entityInfo.get(ApplicationMetricsConstants.TYPE_ENTITY_INFO)
                 .toString();
                 .toString();
       }
       }
+      if (entityInfo
+          .containsKey(ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO)) {
+        unmanagedApplication =
+            Boolean.parseBoolean(entityInfo.get(
+                ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO)
+                .toString());
+      }
       if (entityInfo.containsKey(ApplicationMetricsConstants.APP_CPU_METRICS)) {
       if (entityInfo.containsKey(ApplicationMetricsConstants.APP_CPU_METRICS)) {
         long vcoreSeconds=Long.parseLong(entityInfo.get(
         long vcoreSeconds=Long.parseLong(entityInfo.get(
                 ApplicationMetricsConstants.APP_CPU_METRICS).toString());
                 ApplicationMetricsConstants.APP_CPU_METRICS).toString());
@@ -365,7 +374,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
         ConverterUtils.toApplicationId(entity.getEntityId()),
         ConverterUtils.toApplicationId(entity.getEntityId()),
         latestApplicationAttemptId, user, queue, name, null, -1, null, state,
         latestApplicationAttemptId, user, queue, name, null, -1, null, state,
         diagnosticsInfo, null, createdTime, finishedTime, finalStatus, appResources,
         diagnosticsInfo, null, createdTime, finishedTime, finalStatus, appResources,
-        null, progress, type, null, appTags), appViewACLs);
+        null, progress, type, null, appTags, unmanagedApplication), appViewACLs);
   }
   }
 
 
   private static ApplicationAttemptReport convertToApplicationAttemptReport(
   private static ApplicationAttemptReport convertToApplicationAttemptReport(

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java

@@ -475,6 +475,8 @@ public class TestApplicationHistoryManagerOnTimelineStore {
         "test app type");
         "test app type");
     entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, "user1");
     entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, "user1");
     entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO, "test queue");
     entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO, "test queue");
+    entityInfo.put(
+        ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO, "false");
     entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO,
     entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO,
         Integer.MAX_VALUE + 1L);
         Integer.MAX_VALUE + 1L);
     entityInfo.put(ApplicationMetricsConstants.APP_MEM_METRICS,123);
     entityInfo.put(ApplicationMetricsConstants.APP_MEM_METRICS,123);

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java

@@ -74,4 +74,7 @@ public class ApplicationMetricsConstants {
       "YARN_APPLICATION_LATEST_APP_ATTEMPT";
       "YARN_APPLICATION_LATEST_APP_ATTEMPT";
 
 
   public static final String APP_TAGS_INFO = "YARN_APPLICATION_TAGS";
   public static final String APP_TAGS_INFO = "YARN_APPLICATION_TAGS";
+
+  public static final String UNMANAGED_APPLICATION_ENTITY_INFO =
+      "YARN_APPLICATION_UNMANAGED_APPLICATION";
 }
 }

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

@@ -206,6 +206,7 @@ public class AppBlock extends HtmlBlock {
     }
     }
     overviewTable._("Diagnostics:",
     overviewTable._("Diagnostics:",
         app.getDiagnosticsInfo() == null ? "" : app.getDiagnosticsInfo());
         app.getDiagnosticsInfo() == null ? "" : app.getDiagnosticsInfo());
+    overviewTable._("Unmanaged Application:", app.isUnmanagedApp());
 
 
     Collection<ApplicationAttemptReport> attempts;
     Collection<ApplicationAttemptReport> attempts;
     try {
     try {

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

@@ -60,6 +60,7 @@ public class AppInfo {
   protected String applicationTags;
   protected String applicationTags;
   private int allocatedCpuVcores;
   private int allocatedCpuVcores;
   private int allocatedMemoryMB;
   private int allocatedMemoryMB;
+  protected boolean unmanagedApplication;
 
 
   public AppInfo() {
   public AppInfo() {
     // JAXB needs this
     // JAXB needs this
@@ -99,6 +100,7 @@ public class AppInfo {
     if (app.getApplicationTags() != null && !app.getApplicationTags().isEmpty()) {
     if (app.getApplicationTags() != null && !app.getApplicationTags().isEmpty()) {
       this.applicationTags = CSV_JOINER.join(app.getApplicationTags());
       this.applicationTags = CSV_JOINER.join(app.getApplicationTags());
     }
     }
+    unmanagedApplication = app.isUnmanagedApp();
   }
   }
 
 
   public String getAppId() {
   public String getAppId() {
@@ -188,4 +190,8 @@ public class AppInfo {
   public String getApplicationTags() {
   public String getApplicationTags() {
     return applicationTags;
     return applicationTags;
   }
   }
+
+  public boolean isUnmanagedApp() {
+    return unmanagedApplication;
+  }
 }
 }

+ 8 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java

@@ -32,6 +32,7 @@ public class ApplicationCreatedEvent extends
   private String queue;
   private String queue;
   private long submittedTime;
   private long submittedTime;
   private Set<String> appTags;
   private Set<String> appTags;
+  private boolean unmanagedApplication;
 
 
   public ApplicationCreatedEvent(ApplicationId appId,
   public ApplicationCreatedEvent(ApplicationId appId,
       String name,
       String name,
@@ -40,7 +41,8 @@ public class ApplicationCreatedEvent extends
       String queue,
       String queue,
       long submittedTime,
       long submittedTime,
       long createdTime,
       long createdTime,
-      Set<String> appTags) {
+      Set<String> appTags,
+      boolean unmanagedApplication) {
     super(SystemMetricsEventType.APP_CREATED, createdTime);
     super(SystemMetricsEventType.APP_CREATED, createdTime);
     this.appId = appId;
     this.appId = appId;
     this.name = name;
     this.name = name;
@@ -49,6 +51,7 @@ public class ApplicationCreatedEvent extends
     this.queue = queue;
     this.queue = queue;
     this.submittedTime = submittedTime;
     this.submittedTime = submittedTime;
     this.appTags = appTags;
     this.appTags = appTags;
+    this.unmanagedApplication = unmanagedApplication;
   }
   }
 
 
   @Override
   @Override
@@ -83,4 +86,8 @@ public class ApplicationCreatedEvent extends
   public Set<String> getAppTags() {
   public Set<String> getAppTags() {
     return appTags;
     return appTags;
   }
   }
+
+  public boolean isUnmanagedApp() {
+    return unmanagedApplication;
+  }
 }
 }

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java

@@ -106,7 +106,8 @@ public class SystemMetricsPublisher extends CompositeService {
               app.getUser(),
               app.getUser(),
               app.getQueue(),
               app.getQueue(),
               app.getSubmitTime(),
               app.getSubmitTime(),
-              createdTime, app.getApplicationTags()));
+              createdTime, app.getApplicationTags(),
+              app.getApplicationSubmissionContext().getUnmanagedAM()));
     }
     }
   }
   }
 
 
@@ -253,6 +254,9 @@ public class SystemMetricsPublisher extends CompositeService {
         event.getSubmittedTime());
         event.getSubmittedTime());
     entityInfo.put(ApplicationMetricsConstants.APP_TAGS_INFO,
     entityInfo.put(ApplicationMetricsConstants.APP_TAGS_INFO,
         event.getAppTags());
         event.getAppTags());
+    entityInfo.put(
+        ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO,
+        event.isUnmanagedApp());
     entity.setOtherInfo(entityInfo);
     entity.setOtherInfo(entityInfo);
     TimelineEvent tEvent = new TimelineEvent();
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(
     tEvent.setEventType(

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

@@ -663,8 +663,9 @@ public class RMAppImpl implements RMApp, Recoverable {
           createApplicationState(), diags,
           createApplicationState(), diags,
           trackingUrl, this.startTime, this.finishTime, finishState,
           trackingUrl, this.startTime, this.finishTime, finishState,
           appUsageReport, origTrackingUrl, progress, this.applicationType, 
           appUsageReport, origTrackingUrl, progress, this.applicationType, 
-          amrmToken, applicationTags);
+              amrmToken, applicationTags);
       report.setLogAggregationStatus(logAggregationStatus);
       report.setLogAggregationStatus(logAggregationStatus);
+      report.setUnmanagedApp(submissionContext.getUnmanagedAM());
       return report;
       return report;
     } finally {
     } finally {
       this.readLock.unlock();
       this.readLock.unlock();

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

@@ -96,6 +96,7 @@ public class AppInfo {
   protected List<ResourceRequest> resourceRequests;
   protected List<ResourceRequest> resourceRequests;
 
 
   protected LogAggregationStatus logAggregationStatus;
   protected LogAggregationStatus logAggregationStatus;
+  protected boolean unmanagedApplication;
 
 
   public AppInfo() {
   public AppInfo() {
   } // JAXB needs this
   } // JAXB needs this
@@ -183,6 +184,8 @@ public class AppInfo {
           appMetrics.getResourcePreempted().getVirtualCores();
           appMetrics.getResourcePreempted().getVirtualCores();
       memorySeconds = appMetrics.getMemorySeconds();
       memorySeconds = appMetrics.getMemorySeconds();
       vcoreSeconds = appMetrics.getVcoreSeconds();
       vcoreSeconds = appMetrics.getVcoreSeconds();
+      unmanagedApplication =
+          app.getApplicationSubmissionContext().getUnmanagedAM();
     }
     }
   }
   }
 
 
@@ -321,4 +324,8 @@ public class AppInfo {
   public LogAggregationStatus getLogAggregationStatus() {
   public LogAggregationStatus getLogAggregationStatus() {
     return this.logAggregationStatus;
     return this.logAggregationStatus;
   }
   }
+
+  public boolean isUnmanagedApp() {
+    return unmanagedApplication;
+  }
 }
 }

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java

@@ -29,6 +29,7 @@ import java.util.Set;
 
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -134,6 +135,12 @@ public class TestSystemMetricsPublisher {
       Assert.assertEquals(app.getQueue(),
       Assert.assertEquals(app.getQueue(),
           entity.getOtherInfo()
           entity.getOtherInfo()
               .get(ApplicationMetricsConstants.QUEUE_ENTITY_INFO));
               .get(ApplicationMetricsConstants.QUEUE_ENTITY_INFO));
+
+      Assert.assertEquals(
+          app.getApplicationSubmissionContext().getUnmanagedAM(),
+          entity.getOtherInfo().get(
+              ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO));
+
       Assert
       Assert
           .assertEquals(
           .assertEquals(
               app.getUser(),
               app.getUser(),
@@ -362,6 +369,9 @@ public class TestSystemMetricsPublisher {
     appTags.add("test");
     appTags.add("test");
     appTags.add("tags");
     appTags.add("tags");
     when(app.getApplicationTags()).thenReturn(appTags);
     when(app.getApplicationTags()).thenReturn(appTags);
+    ApplicationSubmissionContext asc = mock(ApplicationSubmissionContext.class);
+    when(asc.getUnmanagedAM()).thenReturn(false);
+    when(app.getApplicationSubmissionContext()).thenReturn(asc);
     return app;
     return app;
   }
   }
 
 

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

@@ -1308,14 +1308,15 @@ public class TestRMWebServicesApps extends JerseyTestBase {
           WebServicesTestUtils.getXmlInt(element, "preemptedResourceVCores"),
           WebServicesTestUtils.getXmlInt(element, "preemptedResourceVCores"),
           WebServicesTestUtils.getXmlInt(element, "numNonAMContainerPreempted"),
           WebServicesTestUtils.getXmlInt(element, "numNonAMContainerPreempted"),
           WebServicesTestUtils.getXmlInt(element, "numAMContainerPreempted"),
           WebServicesTestUtils.getXmlInt(element, "numAMContainerPreempted"),
-          WebServicesTestUtils.getXmlString(element, "logAggregationStatus"));
+          WebServicesTestUtils.getXmlString(element, "logAggregationStatus"),
+          WebServicesTestUtils.getXmlBoolean(element, "unmanagedApplication"));
     }
     }
   }
   }
 
 
   public void verifyAppInfo(JSONObject info, RMApp app) throws JSONException,
   public void verifyAppInfo(JSONObject info, RMApp app) throws JSONException,
       Exception {
       Exception {
 
 
-    assertEquals("incorrect number of elements", 28, info.length());
+    assertEquals("incorrect number of elements", 29, info.length());
 
 
     verifyAppInfoGeneric(app, info.getString("id"), info.getString("user"),
     verifyAppInfoGeneric(app, info.getString("id"), info.getString("user"),
         info.getString("name"), info.getString("applicationType"),
         info.getString("name"), info.getString("applicationType"),
@@ -1331,7 +1332,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         info.getInt("preemptedResourceVCores"),
         info.getInt("preemptedResourceVCores"),
         info.getInt("numNonAMContainerPreempted"),
         info.getInt("numNonAMContainerPreempted"),
         info.getInt("numAMContainerPreempted"),
         info.getInt("numAMContainerPreempted"),
-        info.getString("logAggregationStatus"));
+        info.getString("logAggregationStatus"),
+        info.getBoolean("unmanagedApplication"));
   }
   }
 
 
   public void verifyAppInfoGeneric(RMApp app, String id, String user,
   public void verifyAppInfoGeneric(RMApp app, String id, String user,
@@ -1342,7 +1344,8 @@ public class TestRMWebServicesApps extends JerseyTestBase {
       int allocatedMB, int allocatedVCores, int numContainers,
       int allocatedMB, int allocatedVCores, int numContainers,
       int preemptedResourceMB, int preemptedResourceVCores,
       int preemptedResourceMB, int preemptedResourceVCores,
       int numNonAMContainerPreempted, int numAMContainerPreempted,
       int numNonAMContainerPreempted, int numAMContainerPreempted,
-      String logAggregationStatus) throws JSONException,
+      String logAggregationStatus, boolean unmanagedApplication)
+      throws JSONException,
       Exception {
       Exception {
 
 
     WebServicesTestUtils.checkStringMatch("id", app.getApplicationId()
     WebServicesTestUtils.checkStringMatch("id", app.getApplicationId()
@@ -1392,6 +1395,9 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     assertEquals("Log aggregation Status doesn't match", app
     assertEquals("Log aggregation Status doesn't match", app
         .getLogAggregationStatusForAppReport().toString(),
         .getLogAggregationStatusForAppReport().toString(),
         logAggregationStatus);
         logAggregationStatus);
+    assertEquals("unmanagedApplication doesn't match", app
+        .getApplicationSubmissionContext().getUnmanagedAM(),
+        unmanagedApplication);
   }
   }
 
 
   @Test
   @Test

+ 10 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md

@@ -1380,7 +1380,8 @@ Response Body:
 	  "applicationType" : "MAPREDUCE",
 	  "applicationType" : "MAPREDUCE",
 	  "applicationTags" : ""
 	  "applicationTags" : ""
           "memorySeconds" : 151730,
           "memorySeconds" : 151730,
-          "vcoreSeconds" : 103
+          "vcoreSeconds" : 103,
+          "unmanagedApplication":"false"
        },
        },
        {
        {
           "finishedTime" : 1326815789546,
           "finishedTime" : 1326815789546,
@@ -1406,7 +1407,8 @@ Response Body:
 	  "applicationTags" : "tag1"
 	  "applicationTags" : "tag1"
           "memorySeconds" : 640064,
           "memorySeconds" : 640064,
           "vcoreSeconds" : 442
           "vcoreSeconds" : 442
-       } 
+          "unmanagedApplication":"false"
+       }
     ]
     ]
   }
   }
 }
 }
@@ -1455,6 +1457,7 @@ Response Body:
     <runningContainers>0</runningContainers>
     <runningContainers>0</runningContainers>
     <memorySeconds>151730</memorySeconds>
     <memorySeconds>151730</memorySeconds>
     <vcoreSeconds>103</vcoreSeconds>
     <vcoreSeconds>103</vcoreSeconds>
+    <unmanagedApplication>false</unmanagedApplication>
   </app>
   </app>
   <app>
   <app>
     <id>application_1326815542473_0002</id>
     <id>application_1326815542473_0002</id>
@@ -1480,6 +1483,7 @@ Response Body:
     <runningContainers>0</runningContainers>
     <runningContainers>0</runningContainers>
     <memorySeconds>640064</memorySeconds>
     <memorySeconds>640064</memorySeconds>
     <vcoreSeconds>442</vcoreSeconds>
     <vcoreSeconds>442</vcoreSeconds>
+    <unmanagedApplication>false</unmanagedApplication>
   </app>
   </app>
 </apps>
 </apps>
 ```
 ```
@@ -1639,6 +1643,7 @@ Note that depending on security settings a user might not be able to see all the
 | runningContainers | int | The number of containers currently running for the application |
 | runningContainers | int | The number of containers currently running for the application |
 | memorySeconds | long | The amount of memory the application has allocated (megabyte-seconds) |
 | 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) |
 | vcoreSeconds | long | The amount of CPU resources the application has allocated (virtual core-seconds) |
+| unmanagedApplication | boolean | Is the application unmanaged. |
 
 
 ### Response Examples
 ### Response Examples
 
 
@@ -1678,7 +1683,8 @@ Response Body:
       "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326821518301_0005/jobhistory/job/job_1326821518301_5_5",
       "trackingUrl" : "http://host.domain.com:8088/proxy/application_1326821518301_0005/jobhistory/job/job_1326821518301_5_5",
       "queue" : "a1",
       "queue" : "a1",
       "memorySeconds" : 151730,
       "memorySeconds" : 151730,
-      "vcoreSeconds" : 103
+      "vcoreSeconds" : 103,
+      "unmanagedApplication":"false"
    }
    }
 }
 }
 ```
 ```
@@ -1720,6 +1726,7 @@ Response Body:
   <amHostHttpAddress>host.domain.com:8042</amHostHttpAddress>
   <amHostHttpAddress>host.domain.com:8042</amHostHttpAddress>
   <memorySeconds>151730</memorySeconds>
   <memorySeconds>151730</memorySeconds>
   <vcoreSeconds>103</vcoreSeconds>
   <vcoreSeconds>103</vcoreSeconds>
+  <unmanagedApplication>false</unmanagedApplication>
 </app>
 </app>
 ```
 ```
 
 

+ 16 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md

@@ -1087,7 +1087,8 @@ Response Body:
           "submittedTime":1430425001004,
           "submittedTime":1430425001004,
           "startedTime":1430425001004,
           "startedTime":1430425001004,
           "finishedTime":1430425008861,
           "finishedTime":1430425008861,
-          "elapsedTime":7857},
+          "elapsedTime":7857,
+          "unmanagedApplication":"false"},
           {
           {
           "appId":"application_1430424020775_0003",
           "appId":"application_1430424020775_0003",
           "currentAppAttemptId":"appattempt_1430424020775_0003_000001",
           "currentAppAttemptId":"appattempt_1430424020775_0003_000001",
@@ -1106,7 +1107,8 @@ Response Body:
           "submittedTime":1430424956650,
           "submittedTime":1430424956650,
           "startedTime":1430424956650,
           "startedTime":1430424956650,
           "finishedTime":1430424963907,
           "finishedTime":1430424963907,
-          "elapsedTime":7257},
+          "elapsedTime":7257,
+          "unmanagedApplication":"false"},
           {
           {
           "appId":"application_1430424020775_0002",
           "appId":"application_1430424020775_0002",
           "currentAppAttemptId":"appattempt_1430424020775_0002_000001",
           "currentAppAttemptId":"appattempt_1430424020775_0002_000001",
@@ -1125,7 +1127,8 @@ Response Body:
           "submittedTime":1430424769395,
           "submittedTime":1430424769395,
           "startedTime":1430424769395,
           "startedTime":1430424769395,
           "finishedTime":1430424776594,
           "finishedTime":1430424776594,
-          "elapsedTime":7199},
+          "elapsedTime":7199,
+          "unmanagedApplication":"false"},
           {
           {
           "appId":"application_1430424020775_0001",
           "appId":"application_1430424020775_0001",
           "currentAppAttemptId":"appattempt_1430424020775_0001_000001",
           "currentAppAttemptId":"appattempt_1430424020775_0001_000001",
@@ -1145,7 +1148,8 @@ Response Body:
           "startedTime":1430424072153,
           "startedTime":1430424072153,
           "finishedTime":1430424776594,
           "finishedTime":1430424776594,
           "elapsedTime":18344,
           "elapsedTime":18344,
-          "applicationTags":"mrapplication,ta-example"
+          "applicationTags":"mrapplication,ta-example",
+          "unmanagedApplication":"false"
           }
           }
       ]
       ]
     }
     }
@@ -1187,6 +1191,7 @@ Response Body:
         <startedTime>1430425001004</startedTime>
         <startedTime>1430425001004</startedTime>
         <finishedTime>1430425008861</finishedTime>
         <finishedTime>1430425008861</finishedTime>
         <elapsedTime>7857</elapsedTime>
         <elapsedTime>7857</elapsedTime>
+        <unmanagedApplication>false</unmanagedApplication>
       </app>
       </app>
       <app>
       <app>
         <appId>application_1430424020775_0003</appId>
         <appId>application_1430424020775_0003</appId>
@@ -1207,6 +1212,7 @@ Response Body:
         <startedTime>1430424956650</startedTime>
         <startedTime>1430424956650</startedTime>
         <finishedTime>1430424963907</finishedTime>
         <finishedTime>1430424963907</finishedTime>
         <elapsedTime>7257</elapsedTime>
         <elapsedTime>7257</elapsedTime>
+        <unmanagedApplication>false</unmanagedApplication>
       </app>
       </app>
       <app>
       <app>
         <appId>application_1430424020775_0002</appId>
         <appId>application_1430424020775_0002</appId>
@@ -1227,6 +1233,7 @@ Response Body:
         <startedTime>1430424769395</startedTime>
         <startedTime>1430424769395</startedTime>
         <finishedTime>1430424776594</finishedTime>
         <finishedTime>1430424776594</finishedTime>
         <elapsedTime>7199</elapsedTime>
         <elapsedTime>7199</elapsedTime>
+        <unmanagedApplication>false</unmanagedApplication>
       </app>
       </app>
       <app>
       <app>
         <appId>application_1430424020775_0001</appId>
         <appId>application_1430424020775_0001</appId>
@@ -1248,6 +1255,7 @@ Response Body:
         <finishedTime>1430424072153</finishedTime>
         <finishedTime>1430424072153</finishedTime>
         <elapsedTime>18344</elapsedTime>
         <elapsedTime>18344</elapsedTime>
         <applicationTags>mrapplication,ta-example</applicationTags>
         <applicationTags>mrapplication,ta-example</applicationTags>
+        <unmanagedApplication>false</unmanagedApplication>
       </app>
       </app>
     </apps>
     </apps>
 
 
@@ -1298,6 +1306,7 @@ None
 | `host` | string | The host of the ApplicationMaster |
 | `host` | string | The host of the ApplicationMaster |
 | `rpcPort` | int | The RPC port of the ApplicationMaster; zero if no IPC service declared |
 | `rpcPort` | int | The RPC port of the ApplicationMaster; zero if no IPC service declared |
 | `applicationTags` | string | The application tags. |
 | `applicationTags` | string | The application tags. |
+| `unmanagedApplication` | boolean | Is the application unmanaged. |
 
 
 ### Response Examples:
 ### Response Examples:
 
 
@@ -1334,7 +1343,8 @@ Response Body:
       "startedTime": 1430424053809,
       "startedTime": 1430424053809,
       "finishedTime": 1430424072153,
       "finishedTime": 1430424072153,
       "elapsedTime": 18344,
       "elapsedTime": 18344,
-      "applicationTags": mrapplication,tag-example
+      "applicationTags": mrapplication,tag-example,
+      "unmanagedApplication":"false"
     }
     }
 
 
 #### XML response
 #### XML response
@@ -1373,6 +1383,7 @@ Response Body:
        <finishedTime>1430424072153</finishedTime>
        <finishedTime>1430424072153</finishedTime>
        <elapsedTime>18344</elapsedTime>
        <elapsedTime>18344</elapsedTime>
        <applicationTags>mrapplication,ta-example</applicationTags>
        <applicationTags>mrapplication,ta-example</applicationTags>
+       <unmanagedApplication>false</unmanagedApplication>
      </app>
      </app>
 
 
 ## <a name="REST_API_APPLICATION_ATTEMPT_LIST"></a>Application Attempt List
 ## <a name="REST_API_APPLICATION_ATTEMPT_LIST"></a>Application Attempt List