Prechádzať zdrojové kódy

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 rokov pred
rodič
commit
54504133f4
13 zmenil súbory, kde vykonal 83 pridanie a 31 odobranie
  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. 
     (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
 
     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 =
       "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 */
   public static final String TIMELINE_SERVICE_STORE =
       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 APP_OWNER = "app.owner";
   String APP_STATE = "app.state";
+  String APPS_NUM = "apps.num";
   String QUEUE_NAME = "queue.name";
   String NODE_STATE = "node.state";
   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>
   </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>
     <description>Store class name for timeline store.</description>
     <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 {
     GetApplicationsResponse response =
         GetApplicationsResponse.newInstance(new ArrayList<ApplicationReport>(
-          history.getAllApplications().values()));
+          history.getApplications(request.getLimit()).values()));
     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;
 
   /**
-   * 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.
    * @throws YarnException
    * @throws IOException
    */
   @Public
   @Unstable
-  Map<ApplicationId, ApplicationReport> getAllApplications()
-      throws YarnException, IOException;
+  Map<ApplicationId, ApplicationReport>
+      getApplications(long appsNum) throws YarnException,
+          IOException;
 
   /**
    * 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
-  public Map<ApplicationId, ApplicationReport> getAllApplications()
+  public Map<ApplicationId, ApplicationReport> getApplications(long appsNum)
       throws IOException {
     Map<ApplicationId, ApplicationHistoryData> histData =
         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.TimelineEntity;
 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.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
@@ -78,6 +79,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
   private TimelineDataManager timelineDataManager;
   private ApplicationACLsManager aclsManager;
   private String serverHttpAddress;
+  private long maxLoadedApplications;
 
   public ApplicationHistoryManagerOnTimelineStore(
       TimelineDataManager timelineDataManager,
@@ -91,6 +93,9 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
   protected void serviceInit(Configuration conf) throws Exception {
     serverHttpAddress = WebAppUtils.getHttpSchemePrefix(conf) +
         WebAppUtils.getAHSWebAppURLWithoutScheme(conf);
+    maxLoadedApplications =
+        conf.getLong(YarnConfiguration.APPLICATION_HISTORY_PREFIX_MAX_APPS,
+          YarnConfiguration.DEFAULT_APPLICATION_HISTORY_PREFIX_MAX_APPS);
     super.serviceInit(conf);
   }
 
@@ -101,12 +106,12 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
   }
 
   @Override
-  public Map<ApplicationId, ApplicationReport> getAllApplications()
+  public Map<ApplicationId, ApplicationReport> getApplications(long appsNum)
       throws YarnException, IOException {
     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 =
         new LinkedHashMap<ApplicationId, ApplicationReport>();
     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 {
 
   private static ApplicationHistoryClientService clientService;
+  private static TimelineDataManager dataManager;
 
   @BeforeClass
   public static void setup() throws Exception {
@@ -60,7 +61,7 @@ public class TestApplicationHistoryClientService {
     TimelineStore store =
         TestApplicationHistoryManagerOnTimelineStore.createStore(2);
     TimelineACLsManager aclsManager = new TimelineACLsManager(conf);
-    TimelineDataManager dataManager =
+    dataManager =
         new TimelineDataManager(store, aclsManager);
     ApplicationACLsManager appAclsManager = new ApplicationACLsManager(conf);
     ApplicationHistoryManagerOnTimelineStore historyManager =
@@ -101,8 +102,27 @@ public class TestApplicationHistoryClientService {
         clientService.getApplications(request);
     List<ApplicationReport> appReport = response.getApplicationList();
     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

+ 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("user1", app.getUser());
       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);
       // 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.
@@ -335,7 +337,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
   @Test
   public void testGetApplications() throws Exception {
     Collection<ApplicationReport> apps =
-        historyManager.getAllApplications().values();
+        historyManager.getApplications(Long.MAX_VALUE).values();
     Assert.assertNotNull(apps);
     Assert.assertEquals(SCALE + 1, apps.size());
     ApplicationId ignoredAppId = ApplicationId.newInstance(0, SCALE + 2);
@@ -472,12 +474,12 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     entity.setOtherInfo(entityInfo);
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    tEvent.setTimestamp(Integer.MAX_VALUE + 2L);
+    tEvent.setTimestamp(Integer.MAX_VALUE + 2L + appId.getId());
     entity.addEvent(tEvent);
     tEvent = new TimelineEvent();
     tEvent.setEventType(
         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>();
     eventInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_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.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_VALUE;
 
@@ -70,10 +71,14 @@ public class AppsBlock extends HtmlBlock {
         reqAppStates.add(YarnApplicationState.valueOf(stateString.trim()));
       }
     }
-
     callerUGI = getCallerUGI();
     final GetApplicationsRequest request =
         GetApplicationsRequest.newInstance(reqAppStates);
+    String appsNumStr = $(APPS_NUM);
+    if (appsNumStr != null && !appsNumStr.isEmpty()) {
+      long appsNum = Long.parseLong(appsNumStr);
+      request.setLimit(appsNum);
+    }
     if (callerUGI == null) {
       appReports = appBaseProt.getApplications(request).getApplicationList();
     } 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,
       Set<String> applicationTypes) {
     UserGroupInformation callerUGI = getUser(req);
-    long num = 0;
-    boolean checkCount = false;
     boolean checkStart = false;
     boolean checkEnd = false;
     boolean checkAppTypes = false;
     boolean checkAppStates = false;
-    long countNum = 0;
+    long countNum = Long.MAX_VALUE;
 
     // set values suitable in case both of begin/end not specified
     long sBegin = 0;
@@ -87,7 +85,6 @@ public class WebServices {
     long fEnd = Long.MAX_VALUE;
 
     if (count != null && !count.isEmpty()) {
-      checkCount = true;
       countNum = Long.parseLong(count);
       if (countNum <= 0) {
         throw new BadRequestException("limit value must be greater then 0");
@@ -148,19 +145,20 @@ public class WebServices {
 
     AppsInfo allApps = new AppsInfo();
     Collection<ApplicationReport> appReports = null;
+    final GetApplicationsRequest request =
+        GetApplicationsRequest.newInstance();
+    request.setLimit(countNum);
     try {
       if (callerUGI == null) {
         // TODO: the request should take the params like what RMWebServices does
         // in YARN-1819.
-        GetApplicationsRequest request = GetApplicationsRequest.newInstance();
         appReports = appBaseProt.getApplications(request).getApplicationList();
       } else {
         appReports = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ApplicationReport>> () {
           @Override
           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) {
 
-      if (checkCount && num == countNum) {
-        break;
-      }
-
       if (checkAppStates &&
           !appStates.contains(StringUtils.toLowerCase(
               appReport.getYarnApplicationState().toString()))) {
@@ -212,7 +206,6 @@ public class WebServices {
       AppInfo app = new AppInfo(appReport);
 
       allApps.add(app);
-      num++;
     }
     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.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.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`. |