Browse Source

YARN-3700. Made generic history service load a number of latest applications according to the parameter or the configuration. Contributed by Xuan Gong.

Zhijie Shen 10 years ago
parent
commit
54504133f4
13 changed files with 83 additions and 31 deletions
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  3. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
  4. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  5. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
  6. 8 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java
  7. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
  8. 9 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
  9. 23 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
  10. 7 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
  11. 6 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
  12. 5 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
  13. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md

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

@@ -274,6 +274,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3581. Deprecate -directlyAccessNodeLabelStore in RMAdminCLI. 
     YARN-3581. Deprecate -directlyAccessNodeLabelStore in RMAdminCLI. 
     (Naganarasimha G R via wangda)
     (Naganarasimha G R via wangda)
 
 
+    YARN-3700. Made generic history service load a number of latest applications
+    according to the parameter or the configuration. (Xuan Gong via zjshen)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -1459,6 +1459,15 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS =
   public static final String DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS =
       "0.0.0.0:" + DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_PORT;
       "0.0.0.0:" + DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_PORT;
 
 
+  /**
+   * Defines the max number of applications could be fetched using
+   * REST API or application history protocol and shown in timeline
+   * server web ui.
+   */
+  public static final String APPLICATION_HISTORY_PREFIX_MAX_APPS =
+      APPLICATION_HISTORY_PREFIX + "max-applications";
+  public static final long DEFAULT_APPLICATION_HISTORY_PREFIX_MAX_APPS = 10000;
+
   /** Timeline service store class */
   /** Timeline service store class */
   public static final String TIMELINE_SERVICE_STORE =
   public static final String TIMELINE_SERVICE_STORE =
       TIMELINE_SERVICE_PREFIX + "store-class";
       TIMELINE_SERVICE_PREFIX + "store-class";

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

@@ -33,6 +33,7 @@ public interface YarnWebParams {
   String ENTITY_STRING = "entity.string";
   String ENTITY_STRING = "entity.string";
   String APP_OWNER = "app.owner";
   String APP_OWNER = "app.owner";
   String APP_STATE = "app.state";
   String APP_STATE = "app.state";
+  String APPS_NUM = "apps.num";
   String QUEUE_NAME = "queue.name";
   String QUEUE_NAME = "queue.name";
   String NODE_STATE = "node.state";
   String NODE_STATE = "node.state";
   String NODE_LABEL = "node.label";
   String NODE_LABEL = "node.label";

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -1381,6 +1381,15 @@
     <value></value>
     <value></value>
   </property>
   </property>
 
 
+  <property>
+    <description>
+      Defines the max number of applications could be fetched using REST API or
+      application history protocol and shown in timeline server web ui.
+    </description>
+    <name>yarn.timeline-service.generic-application-history.max-applications</name>
+    <value>10000</value>
+  </property>
+
   <property>
   <property>
     <description>Store class name for timeline store.</description>
     <description>Store class name for timeline store.</description>
     <name>yarn.timeline-service.store-class</name>
     <name>yarn.timeline-service.store-class</name>

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java

@@ -186,7 +186,7 @@ public class ApplicationHistoryClientService extends AbstractService implements
           IOException {
           IOException {
     GetApplicationsResponse response =
     GetApplicationsResponse response =
         GetApplicationsResponse.newInstance(new ArrayList<ApplicationReport>(
         GetApplicationsResponse.newInstance(new ArrayList<ApplicationReport>(
-          history.getAllApplications().values()));
+          history.getApplications(request.getLimit()).values()));
     return response;
     return response;
   }
   }
 
 

+ 8 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java

@@ -51,16 +51,20 @@ public interface ApplicationHistoryManager {
       IOException;
       IOException;
 
 
   /**
   /**
-   * This method returns all Application {@link ApplicationReport}s
-   * 
+   * This method returns the given number of Application
+   * {@link ApplicationReport}s.
+   *
+   * @param appsNum
+   *
    * @return map of {@link ApplicationId} to {@link ApplicationReport}s.
    * @return map of {@link ApplicationId} to {@link ApplicationReport}s.
    * @throws YarnException
    * @throws YarnException
    * @throws IOException
    * @throws IOException
    */
    */
   @Public
   @Public
   @Unstable
   @Unstable
-  Map<ApplicationId, ApplicationReport> getAllApplications()
-      throws YarnException, IOException;
+  Map<ApplicationId, ApplicationReport>
+      getApplications(long appsNum) throws YarnException,
+          IOException;
 
 
   /**
   /**
    * Application can have multiple application attempts
    * Application can have multiple application attempts

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java

@@ -98,7 +98,7 @@ public class ApplicationHistoryManagerImpl extends AbstractService implements
   }
   }
 
 
   @Override
   @Override
-  public Map<ApplicationId, ApplicationReport> getAllApplications()
+  public Map<ApplicationId, ApplicationReport> getApplications(long appsNum)
       throws IOException {
       throws IOException {
     Map<ApplicationId, ApplicationHistoryData> histData =
     Map<ApplicationId, ApplicationHistoryData> histData =
         historyStore.getAllApplications();
         historyStore.getAllApplications();

+ 9 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
@@ -78,6 +79,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
   private TimelineDataManager timelineDataManager;
   private TimelineDataManager timelineDataManager;
   private ApplicationACLsManager aclsManager;
   private ApplicationACLsManager aclsManager;
   private String serverHttpAddress;
   private String serverHttpAddress;
+  private long maxLoadedApplications;
 
 
   public ApplicationHistoryManagerOnTimelineStore(
   public ApplicationHistoryManagerOnTimelineStore(
       TimelineDataManager timelineDataManager,
       TimelineDataManager timelineDataManager,
@@ -91,6 +93,9 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
   protected void serviceInit(Configuration conf) throws Exception {
   protected void serviceInit(Configuration conf) throws Exception {
     serverHttpAddress = WebAppUtils.getHttpSchemePrefix(conf) +
     serverHttpAddress = WebAppUtils.getHttpSchemePrefix(conf) +
         WebAppUtils.getAHSWebAppURLWithoutScheme(conf);
         WebAppUtils.getAHSWebAppURLWithoutScheme(conf);
+    maxLoadedApplications =
+        conf.getLong(YarnConfiguration.APPLICATION_HISTORY_PREFIX_MAX_APPS,
+          YarnConfiguration.DEFAULT_APPLICATION_HISTORY_PREFIX_MAX_APPS);
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }
 
 
@@ -101,12 +106,12 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
   }
   }
 
 
   @Override
   @Override
-  public Map<ApplicationId, ApplicationReport> getAllApplications()
+  public Map<ApplicationId, ApplicationReport> getApplications(long appsNum)
       throws YarnException, IOException {
       throws YarnException, IOException {
     TimelineEntities entities = timelineDataManager.getEntities(
     TimelineEntities entities = timelineDataManager.getEntities(
-        ApplicationMetricsConstants.ENTITY_TYPE, null, null, null, null,
-        null, null, Long.MAX_VALUE, EnumSet.allOf(Field.class),
-        UserGroupInformation.getLoginUser());
+        ApplicationMetricsConstants.ENTITY_TYPE, null, null, null, null, null,
+        null, appsNum == Long.MAX_VALUE ? this.maxLoadedApplications : appsNum,
+        EnumSet.allOf(Field.class), UserGroupInformation.getLoginUser());
     Map<ApplicationId, ApplicationReport> apps =
     Map<ApplicationId, ApplicationReport> apps =
         new LinkedHashMap<ApplicationId, ApplicationReport>();
         new LinkedHashMap<ApplicationId, ApplicationReport>();
     if (entities != null && entities.getEntities() != null) {
     if (entities != null && entities.getEntities() != null) {

+ 23 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java

@@ -53,6 +53,7 @@ import org.junit.Test;
 public class TestApplicationHistoryClientService {
 public class TestApplicationHistoryClientService {
 
 
   private static ApplicationHistoryClientService clientService;
   private static ApplicationHistoryClientService clientService;
+  private static TimelineDataManager dataManager;
 
 
   @BeforeClass
   @BeforeClass
   public static void setup() throws Exception {
   public static void setup() throws Exception {
@@ -60,7 +61,7 @@ public class TestApplicationHistoryClientService {
     TimelineStore store =
     TimelineStore store =
         TestApplicationHistoryManagerOnTimelineStore.createStore(2);
         TestApplicationHistoryManagerOnTimelineStore.createStore(2);
     TimelineACLsManager aclsManager = new TimelineACLsManager(conf);
     TimelineACLsManager aclsManager = new TimelineACLsManager(conf);
-    TimelineDataManager dataManager =
+    dataManager =
         new TimelineDataManager(store, aclsManager);
         new TimelineDataManager(store, aclsManager);
     ApplicationACLsManager appAclsManager = new ApplicationACLsManager(conf);
     ApplicationACLsManager appAclsManager = new ApplicationACLsManager(conf);
     ApplicationHistoryManagerOnTimelineStore historyManager =
     ApplicationHistoryManagerOnTimelineStore historyManager =
@@ -101,8 +102,27 @@ public class TestApplicationHistoryClientService {
         clientService.getApplications(request);
         clientService.getApplications(request);
     List<ApplicationReport> appReport = response.getApplicationList();
     List<ApplicationReport> appReport = response.getApplicationList();
     Assert.assertNotNull(appReport);
     Assert.assertNotNull(appReport);
-    Assert.assertEquals(appId, appReport.get(0).getApplicationId());
-    Assert.assertEquals(appId1, appReport.get(1).getApplicationId());
+    Assert.assertEquals(appId, appReport.get(1).getApplicationId());
+    Assert.assertEquals(appId1, appReport.get(0).getApplicationId());
+
+    // Create a historyManager, and set the max_apps can be loaded
+    // as 1.
+    Configuration conf = new YarnConfiguration();
+    conf.setLong(YarnConfiguration.APPLICATION_HISTORY_PREFIX_MAX_APPS, 1);
+    ApplicationHistoryManagerOnTimelineStore historyManager2 =
+        new ApplicationHistoryManagerOnTimelineStore(dataManager,
+          new ApplicationACLsManager(conf));
+    historyManager2.init(conf);
+    historyManager2.start();
+    @SuppressWarnings("resource")
+    ApplicationHistoryClientService clientService2 =
+        new ApplicationHistoryClientService(historyManager2);
+    response = clientService2.getApplications(request);
+    appReport = response.getApplicationList();
+    Assert.assertNotNull(appReport);
+    Assert.assertTrue(appReport.size() == 1);
+    // Expected to get the appReport for application with appId1
+    Assert.assertEquals(appId1, appReport.get(0).getApplicationId());
   }
   }
 
 
   @Test
   @Test

+ 7 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java

@@ -180,8 +180,10 @@ public class TestApplicationHistoryManagerOnTimelineStore {
       Assert.assertEquals("test app type", app.getApplicationType());
       Assert.assertEquals("test app type", app.getApplicationType());
       Assert.assertEquals("user1", app.getUser());
       Assert.assertEquals("user1", app.getUser());
       Assert.assertEquals("test queue", app.getQueue());
       Assert.assertEquals("test queue", app.getQueue());
-      Assert.assertEquals(Integer.MAX_VALUE + 2L, app.getStartTime());
-      Assert.assertEquals(Integer.MAX_VALUE + 3L, app.getFinishTime());
+      Assert.assertEquals(Integer.MAX_VALUE + 2L
+          + app.getApplicationId().getId(), app.getStartTime());
+      Assert.assertEquals(Integer.MAX_VALUE + 3L
+          + +app.getApplicationId().getId(), app.getFinishTime());
       Assert.assertTrue(Math.abs(app.getProgress() - 1.0F) < 0.0001);
       Assert.assertTrue(Math.abs(app.getProgress() - 1.0F) < 0.0001);
       // App 2 doesn't have the ACLs, such that the default ACLs " " will be used.
       // App 2 doesn't have the ACLs, such that the default ACLs " " will be used.
       // Nobody except admin and owner has access to the details of the app.
       // Nobody except admin and owner has access to the details of the app.
@@ -335,7 +337,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
   @Test
   @Test
   public void testGetApplications() throws Exception {
   public void testGetApplications() throws Exception {
     Collection<ApplicationReport> apps =
     Collection<ApplicationReport> apps =
-        historyManager.getAllApplications().values();
+        historyManager.getApplications(Long.MAX_VALUE).values();
     Assert.assertNotNull(apps);
     Assert.assertNotNull(apps);
     Assert.assertEquals(SCALE + 1, apps.size());
     Assert.assertEquals(SCALE + 1, apps.size());
     ApplicationId ignoredAppId = ApplicationId.newInstance(0, SCALE + 2);
     ApplicationId ignoredAppId = ApplicationId.newInstance(0, SCALE + 2);
@@ -472,12 +474,12 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     entity.setOtherInfo(entityInfo);
     entity.setOtherInfo(entityInfo);
     TimelineEvent tEvent = new TimelineEvent();
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
     tEvent.setEventType(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    tEvent.setTimestamp(Integer.MAX_VALUE + 2L);
+    tEvent.setTimestamp(Integer.MAX_VALUE + 2L + appId.getId());
     entity.addEvent(tEvent);
     entity.addEvent(tEvent);
     tEvent = new TimelineEvent();
     tEvent = new TimelineEvent();
     tEvent.setEventType(
     tEvent.setEventType(
         ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
         ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    tEvent.setTimestamp(Integer.MAX_VALUE + 3L);
+    tEvent.setTimestamp(Integer.MAX_VALUE + 3L + appId.getId());
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     eventInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
     eventInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
         "test diagnostics info");
         "test diagnostics info");

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.webapp;
 
 
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_STATE;
 import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_STATE;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPS_NUM;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE;
 
 
@@ -70,10 +71,14 @@ public class AppsBlock extends HtmlBlock {
         reqAppStates.add(YarnApplicationState.valueOf(stateString.trim()));
         reqAppStates.add(YarnApplicationState.valueOf(stateString.trim()));
       }
       }
     }
     }
-
     callerUGI = getCallerUGI();
     callerUGI = getCallerUGI();
     final GetApplicationsRequest request =
     final GetApplicationsRequest request =
         GetApplicationsRequest.newInstance(reqAppStates);
         GetApplicationsRequest.newInstance(reqAppStates);
+    String appsNumStr = $(APPS_NUM);
+    if (appsNumStr != null && !appsNumStr.isEmpty()) {
+      long appsNum = Long.parseLong(appsNumStr);
+      request.setLimit(appsNum);
+    }
     if (callerUGI == null) {
     if (callerUGI == null) {
       appReports = appBaseProt.getApplications(request).getApplicationList();
       appReports = appBaseProt.getApplications(request).getApplicationList();
     } else {
     } else {

+ 5 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java

@@ -72,13 +72,11 @@ public class WebServices {
       String startedEnd, String finishBegin, String finishEnd,
       String startedEnd, String finishBegin, String finishEnd,
       Set<String> applicationTypes) {
       Set<String> applicationTypes) {
     UserGroupInformation callerUGI = getUser(req);
     UserGroupInformation callerUGI = getUser(req);
-    long num = 0;
-    boolean checkCount = false;
     boolean checkStart = false;
     boolean checkStart = false;
     boolean checkEnd = false;
     boolean checkEnd = false;
     boolean checkAppTypes = false;
     boolean checkAppTypes = false;
     boolean checkAppStates = false;
     boolean checkAppStates = false;
-    long countNum = 0;
+    long countNum = Long.MAX_VALUE;
 
 
     // set values suitable in case both of begin/end not specified
     // set values suitable in case both of begin/end not specified
     long sBegin = 0;
     long sBegin = 0;
@@ -87,7 +85,6 @@ public class WebServices {
     long fEnd = Long.MAX_VALUE;
     long fEnd = Long.MAX_VALUE;
 
 
     if (count != null && !count.isEmpty()) {
     if (count != null && !count.isEmpty()) {
-      checkCount = true;
       countNum = Long.parseLong(count);
       countNum = Long.parseLong(count);
       if (countNum <= 0) {
       if (countNum <= 0) {
         throw new BadRequestException("limit value must be greater then 0");
         throw new BadRequestException("limit value must be greater then 0");
@@ -148,19 +145,20 @@ public class WebServices {
 
 
     AppsInfo allApps = new AppsInfo();
     AppsInfo allApps = new AppsInfo();
     Collection<ApplicationReport> appReports = null;
     Collection<ApplicationReport> appReports = null;
+    final GetApplicationsRequest request =
+        GetApplicationsRequest.newInstance();
+    request.setLimit(countNum);
     try {
     try {
       if (callerUGI == null) {
       if (callerUGI == null) {
         // TODO: the request should take the params like what RMWebServices does
         // TODO: the request should take the params like what RMWebServices does
         // in YARN-1819.
         // in YARN-1819.
-        GetApplicationsRequest request = GetApplicationsRequest.newInstance();
         appReports = appBaseProt.getApplications(request).getApplicationList();
         appReports = appBaseProt.getApplications(request).getApplicationList();
       } else {
       } else {
         appReports = callerUGI.doAs(
         appReports = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ApplicationReport>> () {
             new PrivilegedExceptionAction<Collection<ApplicationReport>> () {
           @Override
           @Override
           public Collection<ApplicationReport> run() throws Exception {
           public Collection<ApplicationReport> run() throws Exception {
-            return appBaseProt.getApplications(
-                GetApplicationsRequest.newInstance()).getApplicationList();
+            return appBaseProt.getApplications(request).getApplicationList();
           }
           }
         });
         });
       }
       }
@@ -169,10 +167,6 @@ public class WebServices {
     }
     }
     for (ApplicationReport appReport : appReports) {
     for (ApplicationReport appReport : appReports) {
 
 
-      if (checkCount && num == countNum) {
-        break;
-      }
-
       if (checkAppStates &&
       if (checkAppStates &&
           !appStates.contains(StringUtils.toLowerCase(
           !appStates.contains(StringUtils.toLowerCase(
               appReport.getYarnApplicationState().toString()))) {
               appReport.getYarnApplicationState().toString()))) {
@@ -212,7 +206,6 @@ public class WebServices {
       AppInfo app = new AppInfo(appReport);
       AppInfo app = new AppInfo(appReport);
 
 
       allApps.add(app);
       allApps.add(app);
-      num++;
     }
     }
     return allApps;
     return allApps;
   }
   }

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md

@@ -183,6 +183,7 @@ selected if this policy is either of `HTTPS_ONLY` or `HTTP_AND_HTTPS`.
 | `yarn.timeline-service.handler-thread-count` | Handler thread count to serve the client RPC requests. Defaults to `10`. |
 | `yarn.timeline-service.handler-thread-count` | Handler thread count to serve the client RPC requests. Defaults to `10`. |
 | `yarn.timeline-service.client.max-retries` | The maximum number of retries for attempts to publish data to the timeline service.Defaults to `30`. |
 | `yarn.timeline-service.client.max-retries` | The maximum number of retries for attempts to publish data to the timeline service.Defaults to `30`. |
 | `yarn.timeline-service.client.retry-interval-ms` | The interval in milliseconds between retries for the timeline service client. Defaults to `1000`. |
 | `yarn.timeline-service.client.retry-interval-ms` | The interval in milliseconds between retries for the timeline service client. Defaults to `1000`. |
+| `yarn.timeline-service.generic-application-history.max-applications` | The max number of applications could be fetched by using REST API or application history protocol and shown in timeline server web ui. Defaults to `10000`. |