소스 검색

YARN-4356. Ensure the timeline service v.2 is disabled cleanly and has no
impact when it's turned off. Contributed by Sangjin Lee.

Li Lu 9 년 전
부모
커밋
89e5c44f9e
44개의 변경된 파일529개의 추가작업 그리고 374개의 파일을 삭제
  1. 36 27
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  2. 3 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  3. 0 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  4. 0 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  5. 2 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
  6. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
  7. 51 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  8. 66 87
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  9. 0 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  10. 2 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  11. 12 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
  12. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
  13. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  14. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
  15. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
  16. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
  17. 18 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  18. 29 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  19. 7 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java
  20. 42 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  21. 64 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
  22. 8 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
  23. 18 31
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
  24. 3 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
  25. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java
  26. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
  27. 8 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  28. 19 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  29. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  30. 18 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  31. 13 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  32. 8 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
  33. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
  34. 9 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  35. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.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/metrics/TestSystemMetricsPublisher.java
  37. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
  38. 22 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
  39. 10 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
  40. 11 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
  41. 7 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
  42. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
  43. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
  44. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java

+ 36 - 27
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java

@@ -19,9 +19,6 @@
 package org.apache.hadoop.mapreduce.jobhistory;
 package org.apache.hadoop.mapreduce.jobhistory;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -31,7 +28,11 @@ import java.util.Map;
 import java.util.Timer;
 import java.util.Timer;
 import java.util.TimerTask;
 import java.util.TimerTask;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -57,9 +58,9 @@ import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.MRAppMaster;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
-import org.apache.hadoop.mapreduce.v2.app.MRAppMaster;
 import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
@@ -81,11 +82,10 @@ import org.codehaus.jackson.node.ArrayNode;
 import org.codehaus.jackson.node.JsonNodeFactory;
 import org.codehaus.jackson.node.JsonNodeFactory;
 import org.codehaus.jackson.node.ObjectNode;
 import org.codehaus.jackson.node.ObjectNode;
 
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.sun.jersey.api.client.ClientHandlerException;
 import com.sun.jersey.api.client.ClientHandlerException;
 
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 /**
 /**
  * The job history events get routed to this class. This class writes the Job
  * The job history events get routed to this class. This class writes the Job
  * history events to the DFS directly into a staging dir and then moved to a
  * history events to the DFS directly into a staging dir and then moved to a
@@ -133,20 +133,17 @@ public class JobHistoryEventHandler extends AbstractService
 
 
   protected static final Map<JobId, MetaInfo> fileMap =
   protected static final Map<JobId, MetaInfo> fileMap =
     Collections.<JobId,MetaInfo>synchronizedMap(new HashMap<JobId,MetaInfo>());
     Collections.<JobId,MetaInfo>synchronizedMap(new HashMap<JobId,MetaInfo>());
-  
-  // For posting entities in new timeline service in a non-blocking way
-  // TODO YARN-3367 replace with event loop in TimelineClient.
-  private static ExecutorService threadPool =
-      Executors.newCachedThreadPool(
-          new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
-          .build());
 
 
   // should job completion be force when the AM shuts down?
   // should job completion be force when the AM shuts down?
   protected volatile boolean forceJobCompletion = false;
   protected volatile boolean forceJobCompletion = false;
 
 
   protected TimelineClient timelineClient;
   protected TimelineClient timelineClient;
   
   
-  private boolean newTimelineServiceEnabled = false;
+  private boolean timelineServiceV2Enabled = false;
+
+  // For posting entities in new timeline service in a non-blocking way
+  // TODO YARN-3367 replace with event loop in TimelineClient.
+  private ExecutorService threadPool;
 
 
   private static String MAPREDUCE_JOB_ENTITY_TYPE = "MAPREDUCE_JOB";
   private static String MAPREDUCE_JOB_ENTITY_TYPE = "MAPREDUCE_JOB";
   private static String MAPREDUCE_TASK_ENTITY_TYPE = "MAPREDUCE_TASK";
   private static String MAPREDUCE_TASK_ENTITY_TYPE = "MAPREDUCE_TASK";
@@ -276,22 +273,26 @@ public class JobHistoryEventHandler extends AbstractService
     // configuration status: off, on_with_v1 or on_with_v2.
     // configuration status: off, on_with_v1 or on_with_v2.
     if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA,
     if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA,
         MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)) {
         MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)) {
-      if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
-        
+      LOG.info("Emitting job history data to the timeline service is enabled");
+      if (YarnConfiguration.timelineServiceEnabled(conf)) {
+
         timelineClient = 
         timelineClient = 
             ((MRAppMaster.RunningAppContext)context).getTimelineClient();
             ((MRAppMaster.RunningAppContext)context).getTimelineClient();
         timelineClient.init(conf);
         timelineClient.init(conf);
-        newTimelineServiceEnabled = conf.getBoolean(
-            MRJobConfig.MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED,
-            MRJobConfig.DEFAULT_MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED);
-        LOG.info("Timeline service is enabled: " + (newTimelineServiceEnabled? "v2" : "v1"));
-        LOG.info("Emitting job history data to the timeline server is enabled");
+        timelineServiceV2Enabled =
+            YarnConfiguration.timelineServiceV2Enabled(conf);
+        LOG.info("Timeline service is enabled; version: " +
+            YarnConfiguration.getTimelineServiceVersion(conf));
+        if (timelineServiceV2Enabled) {
+          // initialize the thread pool for v.2 timeline service
+          threadPool = createThreadPool();
+        }
       } else {
       } else {
         LOG.info("Timeline service is not enabled");
         LOG.info("Timeline service is not enabled");
       }
       }
     } else {
     } else {
-      LOG.info("Emitting job history data to the timeline server is not enabled");
+      LOG.info("Emitting job history data to the timeline server is not " +
+          "enabled");
     }
     }
 
 
     // Flag for setting
     // Flag for setting
@@ -459,19 +460,27 @@ public class JobHistoryEventHandler extends AbstractService
     if (timelineClient != null) {
     if (timelineClient != null) {
       timelineClient.stop();
       timelineClient.stop();
     }
     }
-    shutdownAndAwaitTermination();
+    if (threadPool != null) {
+      shutdownAndAwaitTermination();
+    }
     LOG.info("Stopped JobHistoryEventHandler. super.stop()");
     LOG.info("Stopped JobHistoryEventHandler. super.stop()");
     super.serviceStop();
     super.serviceStop();
   }
   }
   
   
   // TODO remove threadPool after adding non-blocking call in TimelineClient
   // TODO remove threadPool after adding non-blocking call in TimelineClient
-  private static void shutdownAndAwaitTermination() {
+  private ExecutorService createThreadPool() {
+    return Executors.newCachedThreadPool(
+      new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
+      .build());
+  }
+
+  private void shutdownAndAwaitTermination() {
     threadPool.shutdown();
     threadPool.shutdown();
     try {
     try {
       if (!threadPool.awaitTermination(60, TimeUnit.SECONDS)) {
       if (!threadPool.awaitTermination(60, TimeUnit.SECONDS)) {
         threadPool.shutdownNow(); 
         threadPool.shutdownNow(); 
         if (!threadPool.awaitTermination(60, TimeUnit.SECONDS))
         if (!threadPool.awaitTermination(60, TimeUnit.SECONDS))
-            LOG.error("ThreadPool did not terminate");
+          LOG.error("ThreadPool did not terminate");
       }
       }
     } catch (InterruptedException ie) {
     } catch (InterruptedException ie) {
       threadPool.shutdownNow();
       threadPool.shutdownNow();
@@ -633,7 +642,7 @@ public class JobHistoryEventHandler extends AbstractService
         processEventForJobSummary(event.getHistoryEvent(), mi.getJobSummary(),
         processEventForJobSummary(event.getHistoryEvent(), mi.getJobSummary(),
             event.getJobID());
             event.getJobID());
         if (timelineClient != null) {
         if (timelineClient != null) {
-          if (newTimelineServiceEnabled) {
+          if (timelineServiceV2Enabled) {
             processEventForNewTimelineService(historyEvent, event.getJobID(),
             processEventForNewTimelineService(historyEvent, event.getJobID(),
                 event.getTimestamp());
                 event.getTimestamp());
           } else {
           } else {

+ 3 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -1078,14 +1078,9 @@ public class MRAppMaster extends CompositeService {
       this.taskAttemptFinishingMonitor = taskAttemptFinishingMonitor;
       this.taskAttemptFinishingMonitor = taskAttemptFinishingMonitor;
       if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA,
       if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA,
               MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)
               MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)
-            && conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-                YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
-
-        boolean newTimelineServiceEnabled = conf.getBoolean(
-            MRJobConfig.MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED,
-            MRJobConfig.DEFAULT_MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED);
-            
-        if (newTimelineServiceEnabled) {
+            && YarnConfiguration.timelineServiceEnabled(conf)) {
+
+        if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
           // create new version TimelineClient
           // create new version TimelineClient
           timelineClient = TimelineClient.createTimelineClient(
           timelineClient = TimelineClient.createTimelineClient(
               appAttemptID.getApplicationId());
               appAttemptID.getApplicationId());

+ 0 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java

@@ -474,11 +474,6 @@ public interface MRJobConfig {
     "mapreduce.job.emit-timeline-data";
     "mapreduce.job.emit-timeline-data";
   public static final boolean DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA =
   public static final boolean DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA =
       false;
       false;
-  
-  public static final String MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED =
-      "mapreduce.job.new-timeline-service.enabled";
-  public static final boolean DEFAULT_MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED =
-      false;
 
 
   public static final String MR_PREFIX = "yarn.app.mapreduce.";
   public static final String MR_PREFIX = "yarn.app.mapreduce.";
 
 

+ 0 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -637,13 +637,6 @@
     </description>
     </description>
 </property>
 </property>
 
 
- <property>
-    <name>mapreduce.job.new-timeline-service.enabled</name>
-    <value>false</value>
-    <description>Specifies if posting job and task events to new timeline service.
-    </description>
-</property>
-
 <property>
 <property>
   <name>mapreduce.input.fileinputformat.split.minsize</name>
   <name>mapreduce.input.fileinputformat.split.minsize</name>
   <value>0</value>
   <value>0</value>

+ 2 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java

@@ -166,11 +166,10 @@ public class TestMRTimelineEventHandling {
     LOG.info("testMRNewTimelineServiceEventHandling start.");
     LOG.info("testMRNewTimelineServiceEventHandling start.");
     Configuration conf = new YarnConfiguration();
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    // enable new timeline service
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
     conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
     conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
 
 
-    // enable new timeline serivce in MR side
-    conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED, true);
-
     // enable aux-service based timeline collectors
     // enable aux-service based timeline collectors
     conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
     conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
     conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME
     conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java

@@ -173,7 +173,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
     boolean enableTimelineAuxService = false;
     boolean enableTimelineAuxService = false;
     if (nmAuxServices != null) {
     if (nmAuxServices != null) {
       for (String nmAuxService: nmAuxServices) {
       for (String nmAuxService: nmAuxServices) {
-        if (nmAuxService == TIMELINE_AUX_SERVICE_NAME) {
+        if (nmAuxService.equals(TIMELINE_AUX_SERVICE_NAME)) {
           enableTimelineAuxService = true;
           enableTimelineAuxService = true;
           break;
           break;
         }
         }

+ 51 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -83,6 +83,10 @@ public class YarnConfiguration extends Configuration {
         new DeprecationDelta("yarn.client.max-nodemanagers-proxies",
         new DeprecationDelta("yarn.client.max-nodemanagers-proxies",
             NM_CLIENT_MAX_NM_PROXIES)
             NM_CLIENT_MAX_NM_PROXIES)
     });
     });
+    Configuration.addDeprecations(new DeprecationDelta[] {
+        new DeprecationDelta(RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
+            SYSTEM_METRICS_PUBLISHER_ENABLED)
+    });
   }
   }
 
 
   //Configurations
   //Configurations
@@ -486,7 +490,8 @@ public class YarnConfiguration extends Configuration {
 
 
   /**
   /**
    *  The setting that controls whether yarn system metrics is published on the
    *  The setting that controls whether yarn system metrics is published on the
-   *  timeline server or not by RM. This configuration setting is for ATS V1
+   *  timeline server or not by RM. This configuration setting is for ATS V1.
+   *  This is now deprecated in favor of SYSTEM_METRICS_PUBLISHER_ENABLED.
    */
    */
   public static final String RM_SYSTEM_METRICS_PUBLISHER_ENABLED = RM_PREFIX
   public static final String RM_SYSTEM_METRICS_PUBLISHER_ENABLED = RM_PREFIX
       + "system-metrics-publisher.enabled";
       + "system-metrics-publisher.enabled";
@@ -2781,13 +2786,52 @@ public class YarnConfiguration extends Configuration {
     }
     }
     return clusterId;
     return clusterId;
   }
   }
-  
-  public static boolean systemMetricsPublisherEnabled(Configuration conf) {
+
+  // helper methods for timeline service configuration
+  /**
+   * Returns whether the timeline service is enabled via configuration.
+   *
+   * @param conf the configuration
+   * @return whether the timeline service is enabled.
+   */
+  public static boolean timelineServiceEnabled(Configuration conf) {
     return conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
     return conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)
-        && conf.getBoolean(
-            YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED,
-            YarnConfiguration.DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED);  
+      YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
+  }
+
+  /**
+   * Returns the timeline service version. It does not check whether the
+   * timeline service itself is enabled.
+   *
+   * @param conf the configuration
+   * @return the timeline service version as a float.
+   */
+  public static float getTimelineServiceVersion(Configuration conf) {
+    return conf.getFloat(TIMELINE_SERVICE_VERSION,
+        DEFAULT_TIMELINE_SERVICE_VERSION);
+  }
+
+  /**
+   * Returns whether the timeline service v.2 is enabled via configuration.
+   *
+   * @param conf the configuration
+   * @return whether the timeline service v.2 is enabled. V.2 refers to a
+   * version greater than equal to 2 but smaller than 3.
+   */
+  public static boolean timelineServiceV2Enabled(Configuration conf) {
+    return timelineServiceEnabled(conf) &&
+        (int)getTimelineServiceVersion(conf) == 2;
+  }
+
+  /**
+   * Returns whether the system publisher is enabled.
+   *
+   * @param conf the configuration
+   * @return whether the system publisher is enabled.
+   */
+  public static boolean systemMetricsPublisherEnabled(Configuration conf) {
+    return conf.getBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED,
+        YarnConfiguration.DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED);
   }
   }
 
 
   /* For debugging. mp configurations to system output as XML format. */
   /* For debugging. mp configurations to system output as XML format. */

+ 66 - 87
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -223,14 +223,11 @@ public class ApplicationMaster {
   // Tracking url to which app master publishes info for clients to monitor
   // Tracking url to which app master publishes info for clients to monitor
   private String appMasterTrackingUrl = "";
   private String appMasterTrackingUrl = "";
 
 
-  private boolean newTimelineService = false;
+  private boolean timelineServiceV2 = false;
 
 
   // For posting entities in new timeline service in a non-blocking way
   // For posting entities in new timeline service in a non-blocking way
   // TODO replace with event loop in TimelineClient.
   // TODO replace with event loop in TimelineClient.
-  private static ExecutorService threadPool =
-      Executors.newCachedThreadPool(
-          new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
-          .build());
+  private ExecutorService threadPool;
 
 
   // App Master configuration
   // App Master configuration
   // No. of containers to run shell command on
   // No. of containers to run shell command on
@@ -331,8 +328,10 @@ public class ApplicationMaster {
       }
       }
       appMaster.run();
       appMaster.run();
       result = appMaster.finish();
       result = appMaster.finish();
-      
-      shutdownAndAwaitTermination();
+
+      if (appMaster.threadPool != null) {
+        appMaster.shutdownAndAwaitTermination();
+      }
     } catch (Throwable t) {
     } catch (Throwable t) {
       LOG.fatal("Error running ApplicationMaster", t);
       LOG.fatal("Error running ApplicationMaster", t);
       LogManager.shutdown();
       LogManager.shutdown();
@@ -346,16 +345,22 @@ public class ApplicationMaster {
       System.exit(2);
       System.exit(2);
     }
     }
   }
   }
-  
+
   //TODO remove threadPool after adding non-blocking call in TimelineClient
   //TODO remove threadPool after adding non-blocking call in TimelineClient
-  private static void shutdownAndAwaitTermination() {
+  private ExecutorService createThreadPool() {
+    return Executors.newCachedThreadPool(
+        new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
+        .build());
+  }
+
+  private void shutdownAndAwaitTermination() {
     threadPool.shutdown();
     threadPool.shutdown();
     try {
     try {
       // Wait a while for existing tasks to terminate
       // Wait a while for existing tasks to terminate
       if (!threadPool.awaitTermination(60, TimeUnit.SECONDS)) {
       if (!threadPool.awaitTermination(60, TimeUnit.SECONDS)) {
         threadPool.shutdownNow();
         threadPool.shutdownNow();
         if (!threadPool.awaitTermination(60, TimeUnit.SECONDS))
         if (!threadPool.awaitTermination(60, TimeUnit.SECONDS))
-            LOG.error("ThreadPool did not terminate");
+          LOG.error("ThreadPool did not terminate");
       }
       }
     } catch (InterruptedException ie) {
     } catch (InterruptedException ie) {
       threadPool.shutdownNow();
       threadPool.shutdownNow();
@@ -433,8 +438,7 @@ public class ApplicationMaster {
     opts.addOption("container_retry_interval", true,
     opts.addOption("container_retry_interval", true,
         "Interval between each retry, unit is milliseconds");
         "Interval between each retry, unit is milliseconds");
     opts.addOption("debug", false, "Dump out debug information");
     opts.addOption("debug", false, "Dump out debug information");
-    opts.addOption("timeline_service_version", true,
-        "Version for timeline service");
+
     opts.addOption("help", false, "Print usage");
     opts.addOption("help", false, "Print usage");
     CommandLine cliParser = new GnuParser().parse(opts, args);
     CommandLine cliParser = new GnuParser().parse(opts, args);
 
 
@@ -586,27 +590,15 @@ public class ApplicationMaster {
     containrRetryInterval = Integer.parseInt(cliParser.getOptionValue(
     containrRetryInterval = Integer.parseInt(cliParser.getOptionValue(
         "container_retry_interval", "0"));
         "container_retry_interval", "0"));
 
 
-    if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-      YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
-      if (cliParser.hasOption("timeline_service_version")) {
-        String timelineServiceVersion =
-            cliParser.getOptionValue("timeline_service_version", "v1");
-        if (timelineServiceVersion.trim().equalsIgnoreCase("v1")) {
-          newTimelineService = false;
-        } else if (timelineServiceVersion.trim().equalsIgnoreCase("v2")) {
-          newTimelineService = true;
-        } else {
-          throw new IllegalArgumentException(
-              "timeline_service_version is not set properly, should be 'v1' or 'v2'");
-        }
+    if (YarnConfiguration.timelineServiceEnabled(conf)) {
+      timelineServiceV2 =
+          YarnConfiguration.timelineServiceV2Enabled(conf);
+      if (timelineServiceV2) {
+        threadPool = createThreadPool();
       }
       }
     } else {
     } else {
       timelineClient = null;
       timelineClient = null;
       LOG.warn("Timeline service is not enabled");
       LOG.warn("Timeline service is not enabled");
-      if (cliParser.hasOption("timeline_service_version")) {
-        throw new IllegalArgumentException(
-            "Timeline service is not enabled");
-      }
     }
     }
 
 
     return true;
     return true;
@@ -668,16 +660,17 @@ public class ApplicationMaster {
     nmClientAsync.start();
     nmClientAsync.start();
 
 
     startTimelineClient(conf);
     startTimelineClient(conf);
-    // need to bind timelineClient
-    amRMClient.registerTimelineClient(timelineClient);
+    if (timelineServiceV2) {
+      // need to bind timelineClient
+      amRMClient.registerTimelineClient(timelineClient);
+    }
     if(timelineClient != null) {
     if(timelineClient != null) {
-      if (newTimelineService) {
-        publishApplicationAttemptEventOnNewTimelineService(timelineClient, 
-            appAttemptID.toString(), DSEvent.DS_APP_ATTEMPT_START, domainId, 
-            appSubmitterUgi);
+      if (timelineServiceV2) {
+        publishApplicationAttemptEventOnTimelineServiceV2(
+            DSEvent.DS_APP_ATTEMPT_START);
       } else {
       } else {
         publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
         publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-            DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi);
+          DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
       }
       }
     }
     }
 
 
@@ -748,10 +741,9 @@ public class ApplicationMaster {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Void>() {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Void>() {
         @Override
         @Override
         public Void run() throws Exception {
         public Void run() throws Exception {
-          if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
+          if (YarnConfiguration.timelineServiceEnabled(conf)) {
             // Creating the Timeline Client
             // Creating the Timeline Client
-            if (newTimelineService) {
+            if (timelineServiceV2) {
               timelineClient = TimelineClient.createTimelineClient(
               timelineClient = TimelineClient.createTimelineClient(
                   appAttemptID.getApplicationId());
                   appAttemptID.getApplicationId());
             } else {
             } else {
@@ -787,10 +779,9 @@ public class ApplicationMaster {
     }
     }
 
 
     if (timelineClient != null) {
     if (timelineClient != null) {
-      if (newTimelineService) {
-        publishApplicationAttemptEventOnNewTimelineService(timelineClient,
-          appAttemptID.toString(), DSEvent.DS_APP_ATTEMPT_END, domainId,
-          appSubmitterUgi);
+      if (timelineServiceV2) {
+        publishApplicationAttemptEventOnTimelineServiceV2(
+            DSEvent.DS_APP_ATTEMPT_END);
       } else {
       } else {
         publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
         publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
           DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
           DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
@@ -900,12 +891,11 @@ public class ApplicationMaster {
               + containerStatus.getContainerId());
               + containerStatus.getContainerId());
         }
         }
         if(timelineClient != null) {
         if(timelineClient != null) {
-          if (newTimelineService) {
-            publishContainerEndEventOnNewTimelineService(
-                timelineClient, containerStatus, domainId, appSubmitterUgi);
+          if (timelineServiceV2) {
+            publishContainerEndEventOnTimelineServiceV2(containerStatus);
           } else {
           } else {
             publishContainerEndEvent(
             publishContainerEndEvent(
-                timelineClient, containerStatus, domainId, appSubmitterUgi);
+              timelineClient, containerStatus, domainId, appSubmitterUgi);
           }
           }
         }
         }
       }
       }
@@ -1033,14 +1023,13 @@ public class ApplicationMaster {
             applicationMaster.timelineClient, container,
             applicationMaster.timelineClient, container,
             applicationMaster.domainId, applicationMaster.appSubmitterUgi);
             applicationMaster.domainId, applicationMaster.appSubmitterUgi);
 
 
-        if (applicationMaster.newTimelineService) {
-            ApplicationMaster.publishContainerStartEventOnNewTimelineService(
-                applicationMaster.timelineClient, container,
-                applicationMaster.domainId, applicationMaster.appSubmitterUgi);
+        if (applicationMaster.timelineServiceV2) {
+            applicationMaster.publishContainerStartEventOnTimelineServiceV2(
+                container);
         } else {
         } else {
           applicationMaster.publishContainerStartEvent(
           applicationMaster.publishContainerStartEvent(
-              applicationMaster.timelineClient, container,
-              applicationMaster.domainId, applicationMaster.appSubmitterUgi);
+            applicationMaster.timelineClient, container,
+            applicationMaster.domainId, applicationMaster.appSubmitterUgi);
         }
         }
       }
       }
     }
     }
@@ -1349,7 +1338,7 @@ public class ApplicationMaster {
       LOG.error("App Attempt "
       LOG.error("App Attempt "
           + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end")
           + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end")
           + " event could not be published for "
           + " event could not be published for "
-          + appAttemptId.toString(), e);
+          + appAttemptID, e);
     }
     }
   }
   }
 
 
@@ -1397,27 +1386,24 @@ public class ApplicationMaster {
     return new Thread(runnableLaunchContainer);
     return new Thread(runnableLaunchContainer);
   }
   }
   
   
-  private static void publishContainerStartEventOnNewTimelineService(
-      final TimelineClient timelineClient, final Container container,
-      final String domainId, final UserGroupInformation ugi) {
+  private void publishContainerStartEventOnTimelineServiceV2(
+      final Container container) {
     Runnable publishWrapper = new Runnable() {
     Runnable publishWrapper = new Runnable() {
       public void run() {
       public void run() {
-        publishContainerStartEventOnNewTimelineServiceBase(timelineClient,
-            container, domainId, ugi);
+        publishContainerStartEventOnTimelineServiceV2Base(container);
       }
       }
     };
     };
     threadPool.execute(publishWrapper);
     threadPool.execute(publishWrapper);
   }
   }
 
 
-  private static void publishContainerStartEventOnNewTimelineServiceBase(
-      final TimelineClient timelineClient, Container container, String domainId,
-      UserGroupInformation ugi) {
+  private void publishContainerStartEventOnTimelineServiceV2Base(
+      Container container) {
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
     entity.setId(container.getId().toString());
     entity.setId(container.getId().toString());
     entity.setType(DSEntity.DS_CONTAINER.toString());
     entity.setType(DSEntity.DS_CONTAINER.toString());
     //entity.setDomainId(domainId);
     //entity.setDomainId(domainId);
-    entity.addInfo("user", ugi.getShortUserName());
+    entity.addInfo("user", appSubmitterUgi.getShortUserName());
 
 
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
@@ -1428,7 +1414,7 @@ public class ApplicationMaster {
     entity.addEvent(event);
     entity.addEvent(event);
 
 
     try {
     try {
-      ugi.doAs(new PrivilegedExceptionAction<Object>() {
+      appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         @Override
         public TimelinePutResponse run() throws Exception {
         public TimelinePutResponse run() throws Exception {
           timelineClient.putEntities(entity);
           timelineClient.putEntities(entity);
@@ -1442,27 +1428,24 @@ public class ApplicationMaster {
     }
     }
   }
   }
 
 
-  private static void publishContainerEndEventOnNewTimelineService(
-      final TimelineClient timelineClient, final ContainerStatus container,
-      final String domainId, final UserGroupInformation ugi) {
+  private void publishContainerEndEventOnTimelineServiceV2(
+      final ContainerStatus container) {
     Runnable publishWrapper = new Runnable() {
     Runnable publishWrapper = new Runnable() {
       public void run() {
       public void run() {
-          publishContainerEndEventOnNewTimelineServiceBase(timelineClient,
-              container, domainId, ugi);
+          publishContainerEndEventOnTimelineServiceV2Base(container);
       }
       }
     };
     };
     threadPool.execute(publishWrapper);
     threadPool.execute(publishWrapper);
   }
   }
 
 
-  private static void publishContainerEndEventOnNewTimelineServiceBase(
-      final TimelineClient timelineClient, final ContainerStatus container,
-      final String domainId, final UserGroupInformation ugi) {
+  private void publishContainerEndEventOnTimelineServiceV2Base(
+      final ContainerStatus container) {
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
     entity.setId(container.getContainerId().toString());
     entity.setId(container.getContainerId().toString());
     entity.setType(DSEntity.DS_CONTAINER.toString());
     entity.setType(DSEntity.DS_CONTAINER.toString());
     //entity.setDomainId(domainId);
     //entity.setDomainId(domainId);
-    entity.addInfo("user", ugi.getShortUserName());
+    entity.addInfo("user", appSubmitterUgi.getShortUserName());
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
         new  org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
         new  org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
     event.setTimestamp(System.currentTimeMillis());
     event.setTimestamp(System.currentTimeMillis());
@@ -1472,7 +1455,7 @@ public class ApplicationMaster {
     entity.addEvent(event);
     entity.addEvent(event);
 
 
     try {
     try {
-      ugi.doAs(new PrivilegedExceptionAction<Object>() {
+      appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         @Override
         public TimelinePutResponse run() throws Exception {
         public TimelinePutResponse run() throws Exception {
           timelineClient.putEntities(entity);
           timelineClient.putEntities(entity);
@@ -1486,29 +1469,25 @@ public class ApplicationMaster {
     }
     }
   }
   }
 
 
-  private static void publishApplicationAttemptEventOnNewTimelineService(
-      final TimelineClient timelineClient, final String appAttemptId,
-      final DSEvent appEvent, final String domainId,
-      final UserGroupInformation ugi) {
+  private void publishApplicationAttemptEventOnTimelineServiceV2(
+      final DSEvent appEvent) {
 
 
     Runnable publishWrapper = new Runnable() {
     Runnable publishWrapper = new Runnable() {
       public void run() {
       public void run() {
-        publishApplicationAttemptEventOnNewTimelineServiceBase(timelineClient,
-            appAttemptId, appEvent, domainId, ugi);
+        publishApplicationAttemptEventOnTimelineServiceV2Base(appEvent);
       }
       }
     };
     };
     threadPool.execute(publishWrapper);
     threadPool.execute(publishWrapper);
   }
   }
 
 
-  private static void publishApplicationAttemptEventOnNewTimelineServiceBase(
-      final TimelineClient timelineClient, String appAttemptId,
-      DSEvent appEvent, String domainId, UserGroupInformation ugi) {
+  private void publishApplicationAttemptEventOnTimelineServiceV2Base(
+      DSEvent appEvent) {
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
-    entity.setId(appAttemptId);
+    entity.setId(appAttemptID.toString());
     entity.setType(DSEntity.DS_APP_ATTEMPT.toString());
     entity.setType(DSEntity.DS_APP_ATTEMPT.toString());
     //entity.setDomainId(domainId);
     //entity.setDomainId(domainId);
-    entity.addInfo("user", ugi.getShortUserName());
+    entity.addInfo("user", appSubmitterUgi.getShortUserName());
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
     event.setId(appEvent.toString());
     event.setId(appEvent.toString());
@@ -1516,7 +1495,7 @@ public class ApplicationMaster {
     entity.addEvent(event);
     entity.addEvent(event);
 
 
     try {
     try {
-      ugi.doAs(new PrivilegedExceptionAction<Object>() {
+      appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         @Override
         public TimelinePutResponse run() throws Exception {
         public TimelinePutResponse run() throws Exception {
           timelineClient.putEntities(entity);
           timelineClient.putEntities(entity);
@@ -1527,7 +1506,7 @@ public class ApplicationMaster {
       LOG.error("App Attempt "
       LOG.error("App Attempt "
           + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end")
           + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end")
           + " event could not be published for "
           + " event could not be published for "
-          + appAttemptId.toString(),
+          + appAttemptID,
           e instanceof UndeclaredThrowableException ? e.getCause() : e);
           e instanceof UndeclaredThrowableException ? e.getCause() : e);
     }
     }
   }
   }

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

@@ -196,8 +196,6 @@ public class Client {
   // Command line options
   // Command line options
   private Options opts;
   private Options opts;
 
 
-  private String timelineServiceVersion;
-
   private static final String shellCommandPath = "shellCommands";
   private static final String shellCommandPath = "shellCommands";
   private static final String shellArgsPath = "shellArgs";
   private static final String shellArgsPath = "shellArgs";
   private static final String appMasterJarPath = "AppMaster.jar";
   private static final String appMasterJarPath = "AppMaster.jar";
@@ -273,7 +271,6 @@ public class Client {
     opts.addOption("container_vcores", true, "Amount of virtual cores to be requested to run the shell command");
     opts.addOption("container_vcores", true, "Amount of virtual cores to be requested to run the shell command");
     opts.addOption("num_containers", true, "No. of containers on which the shell command needs to be executed");
     opts.addOption("num_containers", true, "No. of containers on which the shell command needs to be executed");
     opts.addOption("log_properties", true, "log4j.properties file");
     opts.addOption("log_properties", true, "log4j.properties file");
-    opts.addOption("timeline_service_version", true, "Version for timeline service");
     opts.addOption("keep_containers_across_application_attempts", false,
     opts.addOption("keep_containers_across_application_attempts", false,
       "Flag to indicate whether to keep containers across application attempts." +
       "Flag to indicate whether to keep containers across application attempts." +
       " If the flag is true, running containers will not be killed when" +
       " If the flag is true, running containers will not be killed when" +
@@ -387,16 +384,6 @@ public class Client {
           + " Specified virtual cores=" + amVCores);
           + " Specified virtual cores=" + amVCores);
     }
     }
 
 
-    if (cliParser.hasOption("timeline_service_version")) {
-      timelineServiceVersion =
-        cliParser.getOptionValue("timeline_service_version", "v1");
-      if (! (timelineServiceVersion.trim().equalsIgnoreCase("v1") ||
-          timelineServiceVersion.trim().equalsIgnoreCase("v2"))) {
-        throw new IllegalArgumentException(
-              "timeline_service_version is not set properly, should be 'v1' or 'v2'");
-      }
-    }
-
     if (!cliParser.hasOption("jar")) {
     if (!cliParser.hasOption("jar")) {
       throw new IllegalArgumentException("No jar file specified for application master");
       throw new IllegalArgumentException("No jar file specified for application master");
     }		
     }		
@@ -726,9 +713,6 @@ public class Client {
 
 
     vargs.addAll(containerRetryOptions);
     vargs.addAll(containerRetryOptions);
 
 
-    if (timelineServiceVersion != null) {
-      vargs.add("--timeline_service_version " + timelineServiceVersion);
-    }
     vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stdout");
     vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stdout");
     vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stderr");
     vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stderr");
 
 

+ 2 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -138,9 +138,10 @@ public class TestDistributedShell {
     conf.set("yarn.log.dir", "target");
     conf.set("yarn.log.dir", "target");
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     // mark if we need to launch the v1 timeline server
     // mark if we need to launch the v1 timeline server
-    boolean enableATSServer = true;
     // disable aux-service based timeline aggregators
     // disable aux-service based timeline aggregators
     conf.set(YarnConfiguration.NM_AUX_SERVICES, "");
     conf.set(YarnConfiguration.NM_AUX_SERVICES, "");
+    conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
+
     conf.set(YarnConfiguration.NM_VMEM_PMEM_RATIO, "8");
     conf.set(YarnConfiguration.NM_VMEM_PMEM_RATIO, "8");
     conf.set(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class.getName());
     conf.set(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class.getName());
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
@@ -157,7 +158,6 @@ public class TestDistributedShell {
         true);
         true);
     conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
     conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
           true);
           true);
-    conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, false);
 
 
     // ATS version specific settings
     // ATS version specific settings
     if (timelineVersion == 1.0f) {
     if (timelineVersion == 1.0f) {
@@ -177,7 +177,6 @@ public class TestDistributedShell {
           DistributedShellTimelinePlugin.class.getName());
           DistributedShellTimelinePlugin.class.getName());
     } else if (timelineVersion == 2.0f) {
     } else if (timelineVersion == 2.0f) {
       // disable v1 timeline server since we no longer have a server here
       // disable v1 timeline server since we no longer have a server here
-      enableATSServer = false;
       // enable aux-service based timeline aggregators
       // enable aux-service based timeline aggregators
       conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
       conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
       conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME
       conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME
@@ -331,12 +330,7 @@ public class TestDistributedShell {
     }
     }
     boolean isTestingTimelineV2 = false;
     boolean isTestingTimelineV2 = false;
     if (timelineVersionWatcher.getTimelineVersion() == 2.0f) {
     if (timelineVersionWatcher.getTimelineVersion() == 2.0f) {
-      String[] timelineArgs = {
-          "--timeline_service_version",
-          "v2"
-      };
       isTestingTimelineV2 = true;
       isTestingTimelineV2 = true;
-      args = mergeArgs(args, timelineArgs);
       if (!defaultFlow) {
       if (!defaultFlow) {
         String[] flowArgs = {
         String[] flowArgs = {
             "--flow_name",
             "--flow_name",

+ 12 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java

@@ -54,17 +54,21 @@ public abstract class TimelineClient extends AbstractService implements
    * current user may use {@link UserGroupInformation#doAs} another user to
    * current user may use {@link UserGroupInformation#doAs} another user to
    * construct and initialize a timeline client if the following operations are
    * construct and initialize a timeline client if the following operations are
    * supposed to be conducted by that user.
    * supposed to be conducted by that user.
-   *
-   * @return a timeline client
    */
    */
   protected ApplicationId contextAppId;
   protected ApplicationId contextAppId;
 
 
+  /**
+   * Creates an instance of the timeline v.1.x client.
+   */
   @Public
   @Public
   public static TimelineClient createTimelineClient() {
   public static TimelineClient createTimelineClient() {
     TimelineClient client = new TimelineClientImpl();
     TimelineClient client = new TimelineClientImpl();
     return client;
     return client;
   }
   }
 
 
+  /**
+   * Creates an instance of the timeline v.2 client.
+   */
   @Public
   @Public
   public static TimelineClient createTimelineClient(ApplicationId appId) {
   public static TimelineClient createTimelineClient(ApplicationId appId) {
     TimelineClient client = new TimelineClientImpl(appId);
     TimelineClient client = new TimelineClientImpl(appId);
@@ -203,8 +207,9 @@ public abstract class TimelineClient extends AbstractService implements
   /**
   /**
    * <p>
    * <p>
    * Send the information of a number of conceptual entities to the timeline
    * Send the information of a number of conceptual entities to the timeline
-   * aggregator. It is a blocking API. The method will not return until all the
-   * put entities have been persisted.
+   * service v.2 collector. It is a blocking API. The method will not return
+   * until all the put entities have been persisted. If this method is invoked
+   * for a non-v.2 timeline client instance, a YarnException is thrown.
    * </p>
    * </p>
    *
    *
    * @param entities
    * @param entities
@@ -220,8 +225,9 @@ public abstract class TimelineClient extends AbstractService implements
   /**
   /**
    * <p>
    * <p>
    * Send the information of a number of conceptual entities to the timeline
    * Send the information of a number of conceptual entities to the timeline
-   * aggregator. It is an asynchronous API. The method will return once all the
-   * entities are received.
+   * service v.2 collector. It is an asynchronous API. The method will return
+   * once all the entities are received. If this method is invoked for a
+   * non-v.2 timeline client instance, a YarnException is thrown.
    * </p>
    * </p>
    *
    *
    * @param entities
    * @param entities

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java

@@ -388,6 +388,9 @@ public class TimelineClientImpl extends TimelineClient {
   private void putEntities(boolean async,
   private void putEntities(boolean async,
       org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity... entities)
       org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity... entities)
       throws IOException, YarnException {
       throws IOException, YarnException {
+    if (!timelineServiceV2) {
+      throw new YarnException("v.2 method is invoked on a v.1.x client");
+    }
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
         entitiesContainer =
         entitiesContainer =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities();
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities();

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -785,14 +785,15 @@
   <property>
   <property>
     <description>The setting that controls whether yarn system metrics is
     <description>The setting that controls whether yarn system metrics is
     published to the Timeline server (version one) or not, by RM. 
     published to the Timeline server (version one) or not, by RM. 
-    This configuration is deprecated.</description>
+    This configuration is now deprecated in favor of
+    yarn.system-metrics-publisher.enabled.</description>
     <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
     <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
     <value>false</value>
     <value>false</value>
   </property>
   </property>
 
 
   <property>
   <property>
     <description>The setting that controls whether yarn system metrics is
     <description>The setting that controls whether yarn system metrics is
-    published on the Timeline server (version two) or not by RM And NM.</description>
+    published on the Timeline service or not by RM And NM.</description>
     <name>yarn.system-metrics-publisher.enabled</name>
     <name>yarn.system-metrics-publisher.enabled</name>
     <value>false</value>
     <value>false</value>
   </property>
   </property>

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java

@@ -261,10 +261,12 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private void initRegisteredCollectors() {
   private void initRegisteredCollectors() {
     NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
     NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
     List<AppCollectorsMapProto> list = p.getRegisteredCollectorsList();
     List<AppCollectorsMapProto> list = p.getRegisteredCollectorsList();
-    this.registeredCollectors = new HashMap<ApplicationId, String> ();
-    for (AppCollectorsMapProto c : list) {
-      ApplicationId appId = convertFromProtoFormat(c.getAppId());
-      this.registeredCollectors.put(appId, c.getAppCollectorAddr());
+    if (!list.isEmpty()) {
+      this.registeredCollectors = new HashMap<>();
+      for (AppCollectorsMapProto c : list) {
+        ApplicationId appId = convertFromProtoFormat(c.getAppId());
+        this.registeredCollectors.put(appId, c.getAppCollectorAddr());
+      }
     }
     }
   }
   }
 
 

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java

@@ -589,10 +589,12 @@ public class NodeHeartbeatResponsePBImpl extends
   private void initAppCollectorsMap() {
   private void initAppCollectorsMap() {
     NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
     NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
     List<AppCollectorsMapProto> list = p.getAppCollectorsMapList();
     List<AppCollectorsMapProto> list = p.getAppCollectorsMapList();
-    this.appCollectorsMap = new HashMap<ApplicationId, String> ();
-    for (AppCollectorsMapProto c : list) {
-      ApplicationId appId = convertFromProtoFormat(c.getAppId());
-      this.appCollectorsMap.put(appId, c.getAppCollectorAddr());
+    if (!list.isEmpty()) {
+      this.appCollectorsMap = new HashMap<>();
+      for (AppCollectorsMapProto c : list) {
+        ApplicationId appId = convertFromProtoFormat(c.getAppId());
+        this.appCollectorsMap.put(appId, c.getAppCollectorAddr());
+      }
     }
     }
   }
   }
 
 

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java

@@ -75,7 +75,8 @@ public interface Context {
 
 
   /**
   /**
    * Get the registered collectors that located on this NM.
    * Get the registered collectors that located on this NM.
-   * @return registered
+   * @return registered collectors, or null if the timeline service v.2 is not
+   * enabled
    */
    */
   Map<ApplicationId, String> getRegisteredCollectors();
   Map<ApplicationId, String> getRegisteredCollectors();
 
 

+ 18 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java

@@ -100,6 +100,7 @@ public class NodeManager extends CompositeService
   private Context context;
   private Context context;
   private AsyncDispatcher dispatcher;
   private AsyncDispatcher dispatcher;
   private ContainerManagerImpl containerManager;
   private ContainerManagerImpl containerManager;
+  // the NM collector service is set only if the timeline service v.2 is enabled
   private NMCollectorService nmCollectorService;
   private NMCollectorService nmCollectorService;
   private NodeStatusUpdater nodeStatusUpdater;
   private NodeStatusUpdater nodeStatusUpdater;
   private NodeResourceMonitor nodeResourceMonitor;
   private NodeResourceMonitor nodeResourceMonitor;
@@ -382,8 +383,10 @@ public class NodeManager extends CompositeService
 
 
     DefaultMetricsSystem.initialize("NodeManager");
     DefaultMetricsSystem.initialize("NodeManager");
 
 
-    this.nmCollectorService = createNMCollectorService(context);
-    addService(nmCollectorService);
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      this.nmCollectorService = createNMCollectorService(context);
+      addService(nmCollectorService);
+    }
 
 
     // StatusUpdater should be added last so that it get started last 
     // StatusUpdater should be added last so that it get started last 
     // so that we make sure everything is up before registering with RM. 
     // so that we make sure everything is up before registering with RM. 
@@ -480,8 +483,7 @@ public class NodeManager extends CompositeService
     protected final ConcurrentMap<ContainerId, Container> containers =
     protected final ConcurrentMap<ContainerId, Container> containers =
         new ConcurrentSkipListMap<ContainerId, Container>();
         new ConcurrentSkipListMap<ContainerId, Container>();
 
 
-    protected Map<ApplicationId, String> registeredCollectors =
-        new ConcurrentHashMap<ApplicationId, String>();
+    protected Map<ApplicationId, String> registeredCollectors;
 
 
     protected final ConcurrentMap<ContainerId,
     protected final ConcurrentMap<ContainerId,
         org.apache.hadoop.yarn.api.records.Container> increasedContainers =
         org.apache.hadoop.yarn.api.records.Container> increasedContainers =
@@ -514,6 +516,9 @@ public class NodeManager extends CompositeService
         LocalDirsHandlerService dirsHandler, ApplicationACLsManager aclsManager,
         LocalDirsHandlerService dirsHandler, ApplicationACLsManager aclsManager,
         NMStateStoreService stateStore, boolean isDistSchedulingEnabled,
         NMStateStoreService stateStore, boolean isDistSchedulingEnabled,
         Configuration conf) {
         Configuration conf) {
+      if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+        this.registeredCollectors = new ConcurrentHashMap<>();
+      }
       this.containerTokenSecretManager = containerTokenSecretManager;
       this.containerTokenSecretManager = containerTokenSecretManager;
       this.nmTokenSecretManager = nmTokenSecretManager;
       this.nmTokenSecretManager = nmTokenSecretManager;
       this.dirsHandler = dirsHandler;
       this.dirsHandler = dirsHandler;
@@ -790,7 +795,14 @@ public class NodeManager extends CompositeService
     return this.context;
     return this.context;
   }
   }
 
 
-  // For testing
+  /**
+   * Returns the NM collector service. It should be used only for testing
+   * purposes.
+   *
+   * @return the NM collector service, or null if the timeline service v.2 is
+   * not enabled
+   */
+  @VisibleForTesting
   NMCollectorService getNMCollectorService() {
   NMCollectorService getNMCollectorService() {
     return this.nmCollectorService;
     return this.nmCollectorService;
   }
   }
@@ -798,6 +810,7 @@ public class NodeManager extends CompositeService
   public static void main(String[] args) throws IOException {
   public static void main(String[] args) throws IOException {
     Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
     Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
     StringUtils.startupShutdownMessage(NodeManager.class, args, LOG);
     StringUtils.startupShutdownMessage(NodeManager.class, args, LOG);
+    @SuppressWarnings("resource")
     NodeManager nodeManager = new NodeManager();
     NodeManager nodeManager = new NodeManager();
     Configuration conf = new YarnConfiguration();
     Configuration conf = new YarnConfiguration();
     new GenericOptionsParser(conf, args);
     new GenericOptionsParser(conf, args);

+ 29 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -909,7 +909,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
                     newResource.toString());
                     newResource.toString());
               }
               }
             }
             }
-            if (YarnConfiguration.systemMetricsPublisherEnabled(context.getConf())) {
+            if (YarnConfiguration.timelineServiceV2Enabled(context.getConf())) {
               updateTimelineClientsAddress(response);
               updateTimelineClientsAddress(response);
             }
             }
 
 
@@ -943,7 +943,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       /**
       /**
        * Caller should take care of sending non null nodelabels for both
        * Caller should take care of sending non null nodelabels for both
        * arguments
        * arguments
-       * 
+       *
        * @param nodeLabelsNew
        * @param nodeLabelsNew
        * @param nodeLabelsOld
        * @param nodeLabelsOld
        * @return if the New node labels are diff from the older one.
        * @return if the New node labels are diff from the older one.
@@ -959,27 +959,37 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
 
       private void updateTimelineClientsAddress(
       private void updateTimelineClientsAddress(
           NodeHeartbeatResponse response) {
           NodeHeartbeatResponse response) {
-        Set<Map.Entry<ApplicationId, String>> rmKnownCollectors = 
-            response.getAppCollectorsMap().entrySet();
-        for (Map.Entry<ApplicationId, String> entry : rmKnownCollectors) {
-          ApplicationId appId = entry.getKey();
-          String collectorAddr = entry.getValue();
-
-          // Only handle applications running on local node.
-          // Not include apps with timeline collectors running in local
-          Application application = context.getApplications().get(appId);
-          if (application != null &&
-              !context.getRegisteredCollectors().containsKey(appId)) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Sync a new collector address: " + collectorAddr + 
-                  " for application: " + appId + " from RM.");
+        Map<ApplicationId, String> knownCollectorsMap =
+            response.getAppCollectorsMap();
+        if (knownCollectorsMap == null) {
+          LOG.warn("the collectors map is null");
+        } else {
+          Set<Map.Entry<ApplicationId, String>> rmKnownCollectors =
+              knownCollectorsMap.entrySet();
+          for (Map.Entry<ApplicationId, String> entry : rmKnownCollectors) {
+            ApplicationId appId = entry.getKey();
+            String collectorAddr = entry.getValue();
+
+            // Only handle applications running on local node.
+            // Not include apps with timeline collectors running in local
+            Application application = context.getApplications().get(appId);
+            // TODO this logic could be problematic if the collector address
+            // gets updated due to NM restart or collector service failure
+            if (application != null &&
+                !context.getRegisteredCollectors().containsKey(appId)) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Sync a new collector address: " + collectorAddr +
+                    " for application: " + appId + " from RM.");
+              }
+              TimelineClient client = application.getTimelineClient();
+              if (client != null) {
+                client.setTimelineServiceAddress(collectorAddr);
+              }
             }
             }
-            TimelineClient client = application.getTimelineClient();
-            client.setTimelineServiceAddress(collectorAddr);
           }
           }
         }
         }
       }
       }
-      
+
       private void updateMasterKeys(NodeHeartbeatResponse response) {
       private void updateMasterKeys(NodeHeartbeatResponse response) {
         // See if the master-key has rolled over
         // See if the master-key has rolled over
         MasterKey updatedMasterKey = response.getContainerTokenMasterKey();
         MasterKey updatedMasterKey = response.getContainerTokenMasterKey();

+ 7 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java

@@ -43,6 +43,10 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 
 
+/**
+ * Service that handles collector information. It is used only if the timeline
+ * service v.2 is enabled.
+ */
 public class NMCollectorService extends CompositeService implements
 public class NMCollectorService extends CompositeService implements
     CollectorNodemanagerProtocol {
     CollectorNodemanagerProtocol {
 
 
@@ -113,9 +117,9 @@ public class NMCollectorService extends CompositeService implements
         String collectorAddr = collector.getCollectorAddr();
         String collectorAddr = collector.getCollectorAddr();
         newCollectorsMap.put(appId, collectorAddr);
         newCollectorsMap.put(appId, collectorAddr);
         // set registered collector address to TimelineClient.
         // set registered collector address to TimelineClient.
-        if (YarnConfiguration.systemMetricsPublisherEnabled(context.getConf())) {
-          TimelineClient client = 
-              context.getApplications().get(appId).getTimelineClient();
+        TimelineClient client =
+            context.getApplications().get(appId).getTimelineClient();
+        if (client != null) {
           client.setTimelineServiceAddress(collectorAddr);
           client.setTimelineServiceAddress(collectorAddr);
         }
         }
       }
       }

+ 42 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -117,6 +117,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationFinishEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationFinishEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl.FlowContext;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationInitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationInitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
@@ -192,7 +193,8 @@ public class ContainerManagerImpl extends CompositeService implements
 
 
   private long waitForContainersOnShutdownMillis;
   private long waitForContainersOnShutdownMillis;
 
 
-  private final NMTimelinePublisher nmMetricsPublisher;
+  // NM metrics publisher is set only if the timeline service v.2 is enabled
+  private NMTimelinePublisher nmMetricsPublisher;
 
 
   public ContainerManagerImpl(Context context, ContainerExecutor exec,
   public ContainerManagerImpl(Context context, ContainerExecutor exec,
       DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
       DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
@@ -220,9 +222,17 @@ public class ContainerManagerImpl extends CompositeService implements
     auxiliaryServices.registerServiceListener(this);
     auxiliaryServices.registerServiceListener(this);
     addService(auxiliaryServices);
     addService(auxiliaryServices);
 
 
-    nmMetricsPublisher = createNMTimelinePublisher(context);
-    context.setNMTimelinePublisher(nmMetricsPublisher);
-    this.containersMonitor = createContainersMonitor(exec);
+    // initialize the metrics publisher if the timeline service v.2 is enabled
+    // and the system publisher is enabled
+    Configuration conf = context.getConf();
+    if (YarnConfiguration.timelineServiceV2Enabled(conf) &&
+        YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
+      LOG.info("YARN system metrics publishing service is enabled");
+      nmMetricsPublisher = createNMTimelinePublisher(context);
+      context.setNMTimelinePublisher(nmMetricsPublisher);
+    }
+    this.containersMonitor =
+        new ContainersMonitorImpl(exec, dispatcher, this.context);
     addService(this.containersMonitor);
     addService(this.containersMonitor);
 
 
     dispatcher.register(ContainerEventType.class,
     dispatcher.register(ContainerEventType.class,
@@ -238,7 +248,6 @@ public class ContainerManagerImpl extends CompositeService implements
     
     
     addService(dispatcher);
     addService(dispatcher);
 
 
-
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     this.readLock = lock.readLock();
     this.readLock = lock.readLock();
     this.writeLock = lock.writeLock();
     this.writeLock = lock.writeLock();
@@ -343,7 +352,7 @@ public class ContainerManagerImpl extends CompositeService implements
     LOG.info("Recovering application " + appId);
     LOG.info("Recovering application " + appId);
     //TODO: Recover flow and flow run ID
     //TODO: Recover flow and flow run ID
     ApplicationImpl app = new ApplicationImpl(
     ApplicationImpl app = new ApplicationImpl(
-        dispatcher, p.getUser(), null, null, 0L, appId, creds, context, 
+        dispatcher, p.getUser(), null, appId, creds, context, 
         p.getAppLogAggregationInitedTime());
         p.getAppLogAggregationInitedTime());
     context.getApplications().put(appId, app);
     context.getApplications().put(appId, app);
     app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
     app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
@@ -970,20 +979,27 @@ public class ContainerManagerImpl extends CompositeService implements
     try {
     try {
       if (!isServiceStopped()) {
       if (!isServiceStopped()) {
         // Create the application
         // Create the application
-        String flowName = launchContext.getEnvironment().get(
-            TimelineUtils.FLOW_NAME_TAG_PREFIX);
-        String flowVersion = launchContext.getEnvironment().get(
-            TimelineUtils.FLOW_VERSION_TAG_PREFIX);
-        String flowRunIdStr = launchContext.getEnvironment().get(
-            TimelineUtils.FLOW_RUN_ID_TAG_PREFIX);
-        long flowRunId = 0L;
-        if (flowRunIdStr != null && !flowRunIdStr.isEmpty()) {
-          flowRunId = Long.parseLong(flowRunIdStr);
+        // populate the flow context from the launch context if the timeline
+        // service v.2 is enabled
+        FlowContext flowContext = null;
+        if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+          String flowName = launchContext.getEnvironment().get(
+              TimelineUtils.FLOW_NAME_TAG_PREFIX);
+          String flowVersion = launchContext.getEnvironment().get(
+              TimelineUtils.FLOW_VERSION_TAG_PREFIX);
+          String flowRunIdStr = launchContext.getEnvironment().get(
+              TimelineUtils.FLOW_RUN_ID_TAG_PREFIX);
+          long flowRunId = 0L;
+          if (flowRunIdStr != null && !flowRunIdStr.isEmpty()) {
+            flowRunId = Long.parseLong(flowRunIdStr);
+          }
+          flowContext =
+              new FlowContext(flowName, flowVersion, flowRunId);
         }
         }
         if (!context.getApplications().containsKey(applicationID)) {
         if (!context.getApplications().containsKey(applicationID)) {
           Application application =
           Application application =
-              new ApplicationImpl(dispatcher, user, flowName, flowVersion,
-                  flowRunId, applicationID, credentials, context);
+              new ApplicationImpl(dispatcher, user, flowContext,
+                  applicationID, credentials, context);
           if (context.getApplications().putIfAbsent(applicationID,
           if (context.getApplications().putIfAbsent(applicationID,
               application) == null) {
               application) == null) {
             LOG.info("Creating a new application reference for app "
             LOG.info("Creating a new application reference for app "
@@ -1335,7 +1351,9 @@ public class ContainerManagerImpl extends CompositeService implements
       Container c = containers.get(event.getContainerID());
       Container c = containers.get(event.getContainerID());
       if (c != null) {
       if (c != null) {
         c.handle(event);
         c.handle(event);
-        nmMetricsPublisher.publishContainerEvent(event);
+        if (nmMetricsPublisher != null) {
+          nmMetricsPublisher.publishContainerEvent(event);
+        }
       } else {
       } else {
         LOG.warn("Event " + event + " sent to absent container " +
         LOG.warn("Event " + event + " sent to absent container " +
             event.getContainerID());
             event.getContainerID());
@@ -1351,7 +1369,9 @@ public class ContainerManagerImpl extends CompositeService implements
               event.getApplicationID());
               event.getApplicationID());
       if (app != null) {
       if (app != null) {
         app.handle(event);
         app.handle(event);
-        nmMetricsPublisher.publishApplicationEvent(event);
+        if (nmMetricsPublisher != null) {
+          nmMetricsPublisher.publishApplicationEvent(event);
+        }
       } else {
       } else {
         LOG.warn("Event " + event + " sent to absent application "
         LOG.warn("Event " + event + " sent to absent application "
             + event.getApplicationID());
             + event.getApplicationID());
@@ -1374,7 +1394,9 @@ public class ContainerManagerImpl extends CompositeService implements
     @Override
     @Override
     public void handle(LocalizationEvent event) {
     public void handle(LocalizationEvent event) {
       origLocalizationEventHandler.handle(event);
       origLocalizationEventHandler.handle(event);
-      timelinePublisher.publishLocalizationEvent(event);
+      if (timelinePublisher != null) {
+        timelinePublisher.publishLocalizationEvent(event);
+      }
     }
     }
   }
   }
 
 

+ 64 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java

@@ -74,9 +74,8 @@ public class ApplicationImpl implements Application {
 
 
   final Dispatcher dispatcher;
   final Dispatcher dispatcher;
   final String user;
   final String user;
-  final String flowName;
-  final String flowVersion;
-  final long flowRunId;
+  // flow context is set only if the timeline service v.2 is enabled
+  private FlowContext flowContext;
   final ApplicationId appId;
   final ApplicationId appId;
   final Credentials credentials;
   final Credentials credentials;
   Map<ApplicationAccessType, String> applicationACLs;
   Map<ApplicationAccessType, String> applicationACLs;
@@ -102,15 +101,24 @@ public class ApplicationImpl implements Application {
   private long applicationLogInitedTimestamp = -1;
   private long applicationLogInitedTimestamp = -1;
   private final NMStateStoreService appStateStore;
   private final NMStateStoreService appStateStore;
 
 
-  public ApplicationImpl(Dispatcher dispatcher, String user, String flowName,
-      String flowVersion, long flowRunId, ApplicationId appId,
-      Credentials credentials, Context context,
+  public ApplicationImpl(Dispatcher dispatcher, String user,
+      ApplicationId appId, Credentials credentials, Context context) {
+    this(dispatcher, user, null, appId, credentials, context, -1L);
+  }
+
+  public ApplicationImpl(Dispatcher dispatcher, String user,
+      ApplicationId appId, Credentials credentials, Context context,
       long recoveredLogInitedTime) {
       long recoveredLogInitedTime) {
+    this(dispatcher, user, null, appId, credentials, context,
+      recoveredLogInitedTime);
+  }
+
+  public ApplicationImpl(Dispatcher dispatcher, String user,
+      FlowContext flowContext, ApplicationId appId, Credentials credentials,
+      Context context, long recoveredLogInitedTime) {
     this.dispatcher = dispatcher;
     this.dispatcher = dispatcher;
     this.user = user;
     this.user = user;
-    this.flowName = flowName;
-    this.flowVersion = flowVersion;
-    this.flowRunId = flowRunId;
+    this.flowContext = flowContext;
     this.appId = appId;
     this.appId = appId;
     this.credentials = credentials;
     this.credentials = credentials;
     this.aclsManager = context.getApplicationACLsManager();
     this.aclsManager = context.getApplicationACLsManager();
@@ -123,17 +131,50 @@ public class ApplicationImpl implements Application {
     setAppLogInitedTimestamp(recoveredLogInitedTime);
     setAppLogInitedTimestamp(recoveredLogInitedTime);
   }
   }
 
 
-  public ApplicationImpl(Dispatcher dispatcher, String user, String flowId,
-      String flowVersion, long flowRunId, ApplicationId appId,
+  public ApplicationImpl(Dispatcher dispatcher, String user,
+      FlowContext flowContext, ApplicationId appId,
       Credentials credentials, Context context) {
       Credentials credentials, Context context) {
-    this(dispatcher, user, flowId, flowVersion, flowRunId, appId, credentials,
+    this(dispatcher, user, flowContext, appId, credentials,
       context, -1);
       context, -1);
     Configuration conf = context.getConf();
     Configuration conf = context.getConf();
-    if (YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
-      createAndStartTimelineClient(conf);
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      if (flowContext == null) {
+        throw new IllegalArgumentException("flow context cannot be null");
+      }
+      this.flowContext = flowContext;
+      if (YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
+        createAndStartTimelineClient(conf);
+      }
     }
     }
   }
   }
-  
+
+  /**
+   * Data object that encapsulates the flow context for the application purpose.
+   */
+  public static class FlowContext {
+    private final String flowName;
+    private final String flowVersion;
+    private final long flowRunId;
+
+    public FlowContext(String flowName, String flowVersion, long flowRunId) {
+      this.flowName = flowName;
+      this.flowVersion = flowVersion;
+      this.flowRunId = flowRunId;
+    }
+
+    public String getFlowName() {
+      return flowName;
+    }
+
+    public String getFlowVersion() {
+      return flowVersion;
+    }
+
+    public long getFlowRunId() {
+      return flowRunId;
+    }
+  }
+
   private void createAndStartTimelineClient(Configuration conf) {
   private void createAndStartTimelineClient(Configuration conf) {
     // create and start timeline client
     // create and start timeline client
     this.timelineClient = TimelineClient.createTimelineClient(appId);
     this.timelineClient = TimelineClient.createTimelineClient(appId);
@@ -528,7 +569,11 @@ public class ApplicationImpl implements Application {
       // Remove collectors info for finished apps.
       // Remove collectors info for finished apps.
       // TODO check we remove related collectors info in failure cases
       // TODO check we remove related collectors info in failure cases
       // (YARN-3038)
       // (YARN-3038)
-      app.context.getRegisteredCollectors().remove(app.getAppId());
+      Map<ApplicationId, String> registeredCollectors =
+          app.context.getRegisteredCollectors();
+      if (registeredCollectors != null) {
+        registeredCollectors.remove(app.getAppId());
+      }
       // stop timelineClient when application get finished.
       // stop timelineClient when application get finished.
       TimelineClient timelineClient = app.getTimelineClient();
       TimelineClient timelineClient = app.getTimelineClient();
       if (timelineClient != null) {
       if (timelineClient != null) {
@@ -595,16 +640,16 @@ public class ApplicationImpl implements Application {
 
 
   @Override
   @Override
   public String getFlowName() {
   public String getFlowName() {
-    return flowName;
+    return flowContext == null ? null : flowContext.getFlowName();
   }
   }
 
 
   @Override
   @Override
   public String getFlowVersion() {
   public String getFlowVersion() {
-    return flowVersion;
+    return flowContext == null ? null : flowContext.getFlowVersion();
   }
   }
 
 
   @Override
   @Override
   public long getFlowRunId() {
   public long getFlowRunId() {
-    return flowRunId;
+    return flowContext == null ? 0L : flowContext.getFlowRunId();
   }
   }
 }
 }

+ 8 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
+import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
@@ -573,9 +574,13 @@ public class ContainersMonitorImpl extends AbstractService implements
 
 
             ContainerImpl container =
             ContainerImpl container =
                 (ContainerImpl) context.getContainers().get(containerId);
                 (ContainerImpl) context.getContainers().get(containerId);
-            container.getNMTimelinePublisher().reportContainerResourceUsage(
-                container, currentTime, pId, currentPmemUsage,
-                cpuUsageTotalCoresPercentage);
+            NMTimelinePublisher nmMetricsPublisher =
+                container.getNMTimelinePublisher();
+            if (nmMetricsPublisher != null) {
+              nmMetricsPublisher.reportContainerResourceUsage(
+                  container, currentTime, pId, currentPmemUsage,
+                  cpuUsageTotalCoresPercentage);
+            }
           } catch (Exception e) {
           } catch (Exception e) {
             // Log the exception and proceed to the next container.
             // Log the exception and proceed to the next container.
             LOG.warn("Uncaught exception in ContainersMonitorImpl "
             LOG.warn("Uncaught exception in ContainersMonitorImpl "

+ 18 - 31
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java

@@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -56,12 +55,16 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
 
+/**
+ * Metrics publisher service that publishes data to the timeline service v.2. It
+ * is used only if the timeline service v.2 is enabled and the system publishing
+ * of events and metrics is enabled.
+ */
 public class NMTimelinePublisher extends CompositeService {
 public class NMTimelinePublisher extends CompositeService {
 
 
   private static final Log LOG = LogFactory.getLog(NMTimelinePublisher.class);
   private static final Log LOG = LogFactory.getLog(NMTimelinePublisher.class);
 
 
   private Dispatcher dispatcher;
   private Dispatcher dispatcher;
-  private boolean publishSystemMetrics;
 
 
   private Context context;
   private Context context;
 
 
@@ -76,24 +79,16 @@ public class NMTimelinePublisher extends CompositeService {
 
 
   @Override
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
   protected void serviceInit(Configuration conf) throws Exception {
-    publishSystemMetrics =
-        YarnConfiguration.systemMetricsPublisherEnabled(conf);
-
-    if (publishSystemMetrics) {
-      dispatcher = new AsyncDispatcher();
-      dispatcher.register(NMTimelineEventType.class,
-          new ForwardingEventHandler());
-      dispatcher
-          .register(ContainerEventType.class, new ContainerEventHandler());
-      dispatcher.register(ApplicationEventType.class,
-          new ApplicationEventHandler());
-      dispatcher.register(LocalizationEventType.class,
-          new LocalizationEventDispatcher());
-      addIfService(dispatcher);
-      LOG.info("YARN system metrics publishing service is enabled");
-    } else {
-      LOG.info("YARN system metrics publishing service is not enabled");
-    }
+    dispatcher = new AsyncDispatcher();
+    dispatcher.register(NMTimelineEventType.class,
+        new ForwardingEventHandler());
+    dispatcher
+        .register(ContainerEventType.class, new ContainerEventHandler());
+    dispatcher.register(ApplicationEventType.class,
+        new ApplicationEventHandler());
+    dispatcher.register(LocalizationEventType.class,
+        new LocalizationEventDispatcher());
+    addIfService(dispatcher);
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }
 
 
@@ -121,8 +116,9 @@ public class NMTimelinePublisher extends CompositeService {
   public void reportContainerResourceUsage(Container container,
   public void reportContainerResourceUsage(Container container,
       long createdTime, String pId, Long pmemUsage,
       long createdTime, String pId, Long pmemUsage,
       Float cpuUsageTotalCoresPercentage) {
       Float cpuUsageTotalCoresPercentage) {
-    if (publishSystemMetrics
-        && (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE || cpuUsageTotalCoresPercentage != ResourceCalculatorProcessTree.UNAVAILABLE)) {
+    if (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE ||
+        cpuUsageTotalCoresPercentage !=
+            ResourceCalculatorProcessTree.UNAVAILABLE) {
       ContainerEntity entity =
       ContainerEntity entity =
           createContainerEntity(container.getContainerId());
           createContainerEntity(container.getContainerId());
       long currentTimeMillis = System.currentTimeMillis();
       long currentTimeMillis = System.currentTimeMillis();
@@ -219,9 +215,6 @@ public class NMTimelinePublisher extends CompositeService {
   }
   }
 
 
   public void publishApplicationEvent(ApplicationEvent event) {
   public void publishApplicationEvent(ApplicationEvent event) {
-    if (!publishSystemMetrics) {
-      return;
-    }
     // publish only when the desired event is received
     // publish only when the desired event is received
     switch (event.getType()) {
     switch (event.getType()) {
     case INIT_APPLICATION:
     case INIT_APPLICATION:
@@ -242,9 +235,6 @@ public class NMTimelinePublisher extends CompositeService {
   }
   }
 
 
   public void publishContainerEvent(ContainerEvent event) {
   public void publishContainerEvent(ContainerEvent event) {
-    if (!publishSystemMetrics) {
-      return;
-    }
     // publish only when the desired event is received
     // publish only when the desired event is received
     switch (event.getType()) {
     switch (event.getType()) {
     case INIT_CONTAINER:
     case INIT_CONTAINER:
@@ -262,9 +252,6 @@ public class NMTimelinePublisher extends CompositeService {
   }
   }
 
 
   public void publishLocalizationEvent(LocalizationEvent event) {
   public void publishLocalizationEvent(LocalizationEvent event) {
-    if (!publishSystemMetrics) {
-      return;
-    }
     // publish only when the desired event is received
     // publish only when the desired event is received
     switch (event.getType()) {
     switch (event.getType()) {
     case CONTAINER_RESOURCES_LOCALIZED:
     case CONTAINER_RESOURCES_LOCALIZED:

+ 3 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java

@@ -50,7 +50,6 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -95,7 +94,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
-import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMMemoryStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMMemoryStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
@@ -106,7 +104,6 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
-import org.mockito.Mockito;
 
 
 public class TestContainerManagerRecovery extends BaseContainerManagerTest {
 public class TestContainerManagerRecovery extends BaseContainerManagerTest {
 
 
@@ -726,9 +723,9 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
           }
           }
 
 
           @Override
           @Override
-          public NMTimelinePublisher createNMTimelinePublisher(Context context) {
-            NMTimelinePublisher timelinePublisher = mock(NMTimelinePublisher.class);
-            return timelinePublisher;
+          public NMTimelinePublisher
+              createNMTimelinePublisher(Context context) {
+            return null;
           }
           }
     };
     };
   }
   }

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -555,7 +554,7 @@ public class TestApplication {
       this.appId = BuilderUtils.newApplicationId(timestamp, id);
       this.appId = BuilderUtils.newApplicationId(timestamp, id);
 
 
       app = new ApplicationImpl(
       app = new ApplicationImpl(
-          dispatcher, this.user, null, null, 0, appId, null, context);
+          dispatcher, this.user, appId, null, context);
       containers = new ArrayList<Container>();
       containers = new ArrayList<Container>();
       for (int i = 0; i < numContainers; i++) {
       for (int i = 0; i < numContainers; i++) {
         Container container = createMockedContainer(this.appId, i);
         Container container = createMockedContainer(this.appId, i);

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java

@@ -32,15 +32,14 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 
 import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
-import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
@@ -63,6 +62,7 @@ import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Document;
@@ -339,7 +339,7 @@ public class TestNMWebServices extends JerseyTestBase {
     final String filename = "logfile1";
     final String filename = "logfile1";
     final String logMessage = "log message\n";
     final String logMessage = "log message\n";
     nmContext.getApplications().put(appId, new ApplicationImpl(null, "user",
     nmContext.getApplications().put(appId, new ApplicationImpl(null, "user",
-        null, null, 0, appId, null, nmContext));
+        appId, null, nmContext));
     
     
     MockContainer container = new MockContainer(appAttemptId,
     MockContainer container = new MockContainer(appAttemptId,
         new AsyncDispatcher(), new Configuration(), "user", appId, 1);
         new AsyncDispatcher(), new Configuration(), "user", appId, 1);

+ 8 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -311,8 +311,11 @@ public class ApplicationMasterService extends AbstractService implements
 
 
     RMApp rmApp =
     RMApp rmApp =
         rmContext.getRMApps().get(applicationAttemptId.getApplicationId());
         rmContext.getRMApps().get(applicationAttemptId.getApplicationId());
+
     // Remove collector address when app get finished.
     // Remove collector address when app get finished.
-    rmApp.removeCollectorAddr();
+    if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+      rmApp.removeCollectorAddr();
+    }
     // checking whether the app exits in RMStateStore at first not to throw
     // checking whether the app exits in RMStateStore at first not to throw
     // ApplicationDoesNotExistInCacheException before and after
     // ApplicationDoesNotExistInCacheException before and after
     // RM work-preserving restart.
     // RM work-preserving restart.
@@ -575,8 +578,10 @@ public class ApplicationMasterService extends AbstractService implements
       allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
       allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
 
 
       // add collector address for this application
       // add collector address for this application
-      allocateResponse.setCollectorAddr(
-          this.rmContext.getRMApps().get(applicationId).getCollectorAddr());
+      if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+        allocateResponse.setCollectorAddr(
+            this.rmContext.getRMApps().get(applicationId).getCollectorAddr());
+      }
 
 
       // add preemption to the allocateResponse message (if any)
       // add preemption to the allocateResponse message (if any)
       allocateResponse
       allocateResponse

+ 19 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -584,24 +584,27 @@ public class ClientRMService extends AbstractService implements
       throw RPCUtil.getRemoteException(ie);
       throw RPCUtil.getRemoteException(ie);
     }
     }
 
 
-    // Sanity check for flow run
-    String value = null;
-    try {
-      for (String tag : submissionContext.getApplicationTags()) {
-        if (tag.startsWith(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX + ":") ||
-            tag.startsWith(
-                TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.toLowerCase() + ":")) {
-          value = tag.substring(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.length() + 1);
-          Long.valueOf(value);
+    if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+      // Sanity check for flow run
+      String value = null;
+      try {
+        for (String tag : submissionContext.getApplicationTags()) {
+          if (tag.startsWith(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX + ":") ||
+              tag.startsWith(
+                  TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.toLowerCase() + ":")) {
+            value = tag.substring(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.length()
+                + 1);
+            Long.valueOf(value);
+          }
         }
         }
+      } catch (NumberFormatException e) {
+        LOG.warn("Invalid to flow run: " + value +
+            ". Flow run should be a long integer", e);
+        RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST,
+            e.getMessage(), "ClientRMService",
+            "Exception in submitting application", applicationId);
+        throw RPCUtil.getRemoteException(e);
       }
       }
-    } catch (NumberFormatException e) {
-      LOG.warn("Invalid to flow run: " + value +
-          ". Flow run should be a long integer", e);
-      RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST,
-          e.getMessage(), "ClientRMService",
-          "Exception in submitting application", applicationId);
-      throw RPCUtil.getRemoteException(e);
     }
     }
 
 
     // Check whether app has already been put into rmContext,
     // Check whether app has already been put into rmContext,

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -402,8 +402,11 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       LOG.warn(message);
       LOG.warn(message);
       throw new YarnException(message);
       throw new YarnException(message);
     }
     }
-    // Start timeline collector for the submitted app
-    application.startTimelineCollector();
+
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      // Start timeline collector for the submitted app
+      application.startTimelineCollector();
+    }
     // Inform the ACLs Manager
     // Inform the ACLs Manager
     this.applicationACLsManager.addApplication(applicationId,
     this.applicationACLsManager.addApplication(applicationId,
         submissionContext.getAMContainerSpec().getApplicationACLs());
         submissionContext.getAMContainerSpec().getApplicationACLs());

+ 18 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -469,18 +469,19 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
   }
 
 
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
-    boolean timelineServiceEnabled =
-        conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
-    SystemMetricsPublisher publisher = null;
-    if (timelineServiceEnabled) {
-      if (conf.getBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
-          YarnConfiguration.DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED)) {
-        LOG.info("TimelineService V1 is configured");
-        publisher = new TimelineServiceV1Publisher();
-      } else {
-        LOG.info("TimelineService V2 is configured");
+    SystemMetricsPublisher publisher;
+    if (YarnConfiguration.timelineServiceEnabled(conf) &&
+        YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
+      if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+        // we're dealing with the v.2.x publisher
+        LOG.info("system metrics publisher with the timeline service V2 is " +
+            "configured");
         publisher = new TimelineServiceV2Publisher(rmContext);
         publisher = new TimelineServiceV2Publisher(rmContext);
+      } else {
+        // we're dealing with the v.1.x publisher
+        LOG.info("system metrics publisher with the timeline service V1 is " +
+            "configured");
+        publisher = new TimelineServiceV1Publisher();
       }
       }
     } else {
     } else {
       LOG.info("TimelineServicePublisher is not configured");
       LOG.info("TimelineServicePublisher is not configured");
@@ -606,10 +607,12 @@ public class ResourceManager extends CompositeService implements Recoverable {
       addService(rmApplicationHistoryWriter);
       addService(rmApplicationHistoryWriter);
       rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
       rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
 
 
-      RMTimelineCollectorManager timelineCollectorManager =
-          createRMTimelineCollectorManager();
-      addService(timelineCollectorManager);
-      rmContext.setRMTimelineCollectorManager(timelineCollectorManager);
+      if (YarnConfiguration.timelineServiceV2Enabled(configuration)) {
+        RMTimelineCollectorManager timelineCollectorManager =
+            createRMTimelineCollectorManager();
+        addService(timelineCollectorManager);
+        rmContext.setRMTimelineCollectorManager(timelineCollectorManager);
+      }
 
 
       // Register event handler for NodesListManager
       // Register event handler for NodesListManager
       nodesListManager = new NodesListManager(rmContext);
       nodesListManager = new NodesListManager(rmContext);

+ 13 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -517,10 +517,15 @@ public class ResourceTrackerService extends AbstractService implements
           message);
           message);
     }
     }
 
 
-    // Check & update collectors info from request.
-    // TODO make sure it won't have race condition issue for AM failed over case
-    // that the older registration could possible override the newer one.
-    updateAppCollectorsMap(request);
+    boolean timelineV2Enabled =
+        YarnConfiguration.timelineServiceV2Enabled(getConfig());
+    if (timelineV2Enabled) {
+      // Check & update collectors info from request.
+      // TODO make sure it won't have race condition issue for AM failed over
+      // case that the older registration could possible override the newer
+      // one.
+      updateAppCollectorsMap(request);
+    }
 
 
     // Heartbeat response
     // Heartbeat response
     NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
     NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
@@ -539,12 +544,12 @@ public class ResourceTrackerService extends AbstractService implements
       nodeHeartBeatResponse.setSystemCredentialsForApps(systemCredentials);
       nodeHeartBeatResponse.setSystemCredentialsForApps(systemCredentials);
     }
     }
 
 
-    // Return collectors' map that NM needs to know
-    // TODO we should optimize this to only include collector info that NM
-    // doesn't know yet.
     List<ApplicationId> keepAliveApps =
     List<ApplicationId> keepAliveApps =
         remoteNodeStatus.getKeepAliveApplications();
         remoteNodeStatus.getKeepAliveApplications();
-    if (keepAliveApps != null) {
+    if (timelineV2Enabled && keepAliveApps != null) {
+      // Return collectors' map that NM needs to know
+      // TODO we should optimize this to only include collector info that NM
+      // doesn't know yet.
       setAppCollectorsMapToResponse(keepAliveApps, nodeHeartBeatResponse);
       setAppCollectorsMapToResponse(keepAliveApps, nodeHeartBeatResponse);
     }
     }
 
 

+ 8 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java

@@ -229,12 +229,14 @@ public class AMLauncher implements Runnable {
             .get(applicationId)
             .get(applicationId)
             .getSubmitTime()));
             .getSubmitTime()));
 
 
-    // Set flow context info
-    for (String tag :
-        rmContext.getRMApps().get(applicationId).getApplicationTags()) {
-      setFlowTags(environment, TimelineUtils.FLOW_NAME_TAG_PREFIX, tag);
-      setFlowTags(environment, TimelineUtils.FLOW_VERSION_TAG_PREFIX, tag);
-      setFlowTags(environment, TimelineUtils.FLOW_RUN_ID_TAG_PREFIX, tag);
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      // Set flow context info
+      for (String tag :
+          rmContext.getRMApps().get(applicationId).getApplicationTags()) {
+        setFlowTags(environment, TimelineUtils.FLOW_NAME_TAG_PREFIX, tag);
+        setFlowTags(environment, TimelineUtils.FLOW_VERSION_TAG_PREFIX, tag);
+        setFlowTags(environment, TimelineUtils.FLOW_RUN_ID_TAG_PREFIX, tag);
+      }
     }
     }
     Credentials credentials = new Credentials();
     Credentials credentials = new Credentials();
     DataInputByteBuffer dibb = new DataInputByteBuffer();
     DataInputByteBuffer dibb = new DataInputByteBuffer();

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

@@ -59,7 +59,7 @@ import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 
 
 /**
 /**
- * This class is responsible for posting application, appattempt & Container
+ * This class is responsible for posting application, appattempt &amp; Container
  * lifecycle related events to timeline service V2
  * lifecycle related events to timeline service V2
  */
  */
 @Private
 @Private

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

@@ -955,15 +955,17 @@ public class RMAppImpl implements RMApp, Recoverable {
       extends RMAppTransition {
       extends RMAppTransition {
 
 
     public void transition(RMAppImpl app, RMAppEvent event) {
     public void transition(RMAppImpl app, RMAppEvent event) {
-      LOG.info("Updating collector info for app: " + app.getApplicationId());
+      if (YarnConfiguration.timelineServiceV2Enabled(app.conf)) {
+        LOG.info("Updating collector info for app: " + app.getApplicationId());
 
 
-      RMAppCollectorUpdateEvent appCollectorUpdateEvent =
-          (RMAppCollectorUpdateEvent) event;
-      // Update collector address
-      app.setCollectorAddr(appCollectorUpdateEvent.getAppCollectorAddr());
+        RMAppCollectorUpdateEvent appCollectorUpdateEvent =
+            (RMAppCollectorUpdateEvent) event;
+        // Update collector address
+        app.setCollectorAddr(appCollectorUpdateEvent.getAppCollectorAddr());
 
 
-      // TODO persistent to RMStateStore for recover
-      // Save to RMStateStore
+        // TODO persistent to RMStateStore for recover
+        // Save to RMStateStore
+      }
     };
     };
   }
   }
 
 

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

@@ -656,6 +656,7 @@ public class TestClientRMService {
     ClientRMService rmService =
     ClientRMService rmService =
         new ClientRMService(rmContext, yarnScheduler, appManager,
         new ClientRMService(rmContext, yarnScheduler, appManager,
             mockAclsManager, mockQueueACLsManager, null);
             mockAclsManager, mockQueueACLsManager, null);
+    rmService.init(new Configuration());
 
 
     // without name and queue
     // without name and queue
 
 
@@ -749,6 +750,7 @@ public class TestClientRMService {
     ClientRMService rmService =
     ClientRMService rmService =
         new ClientRMService(rmContext, yarnScheduler, appManager,
         new ClientRMService(rmContext, yarnScheduler, appManager,
             mockAclsManager, mockQueueACLsManager, null);
             mockAclsManager, mockQueueACLsManager, null);
+    rmService.init(new Configuration());
 
 
     // Initialize appnames and queues
     // Initialize appnames and queues
     String[] queues = {QUEUE_1, QUEUE_2};
     String[] queues = {QUEUE_1, QUEUE_2};
@@ -912,6 +914,7 @@ public class TestClientRMService {
     final ClientRMService rmService =
     final ClientRMService rmService =
         new ClientRMService(rmContext, yarnScheduler, appManager, null, null,
         new ClientRMService(rmContext, yarnScheduler, appManager, null, null,
             null);
             null);
+    rmService.init(new Configuration());
 
 
     // submit an app and wait for it to block while in app submission
     // submit an app and wait for it to block while in app submission
     Thread t = new Thread() {
     Thread t = new Thread() {

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

@@ -75,7 +75,7 @@ public class TestSystemMetricsPublisher {
   public static void setup() throws Exception {
   public static void setup() throws Exception {
     YarnConfiguration conf = new YarnConfiguration();
     YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
     conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
     conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
         MemoryTimelineStore.class, TimelineStore.class);
         MemoryTimelineStore.class, TimelineStore.class);
     conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STATE_STORE_CLASS,
     conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STATE_STORE_CLASS,

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

@@ -131,6 +131,7 @@ public class TestSystemMetricsPublisherForV2 {
   private static Configuration getTimelineV2Conf() {
   private static Configuration getTimelineV2Conf() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
     conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
     conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
     conf.setInt(
     conf.setInt(
         YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE, 2);
         YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE, 2);

+ 22 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java

@@ -20,41 +20,55 @@ package org.apache.hadoop.yarn.server.timelineservice;
 
 
 
 
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
-import static org.mockito.Mockito.any;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 
 
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ExitUtil;
 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.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.timelineservice.*;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationAttemptEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.ClusterEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.QueueEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.api.records.timelineservice.UserEntity;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
 import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
-import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
 import org.apache.hadoop.yarn.server.timelineservice.collector.NodeTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.timelineservice.collector.NodeTimelineCollectorManager;
+import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 
 
-import java.io.IOException;
-
 public class TestTimelineServiceClientIntegration {
 public class TestTimelineServiceClientIntegration {
   private static NodeTimelineCollectorManager collectorManager;
   private static NodeTimelineCollectorManager collectorManager;
   private static PerNodeTimelineCollectorsAuxService auxService;
   private static PerNodeTimelineCollectorsAuxService auxService;
+  private static Configuration conf;
 
 
   @BeforeClass
   @BeforeClass
   public static void setupClass() throws Exception {
   public static void setupClass() throws Exception {
     try {
     try {
       collectorManager = new MockNodeTimelineCollectorManager();
       collectorManager = new MockNodeTimelineCollectorManager();
+      conf = new YarnConfiguration();
+      // enable timeline service v.2
+      conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+      conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
       auxService =
       auxService =
           PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
           PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
-              collectorManager);
+              collectorManager, conf);
       auxService.addApplication(ApplicationId.newInstance(0, 1));
       auxService.addApplication(ApplicationId.newInstance(0, 1));
     } catch (ExitUtil.ExitException e) {
     } catch (ExitUtil.ExitException e) {
       fail();
       fail();
@@ -76,7 +90,7 @@ public class TestTimelineServiceClientIntegration {
       // set the timeline service address manually
       // set the timeline service address manually
       client.setTimelineServiceAddress(
       client.setTimelineServiceAddress(
           collectorManager.getRestServerBindAddress());
           collectorManager.getRestServerBindAddress());
-      client.init(new YarnConfiguration());
+      client.init(conf);
       client.start();
       client.start();
       TimelineEntity entity = new TimelineEntity();
       TimelineEntity entity = new TimelineEntity();
       entity.setType("test entity type");
       entity.setType("test entity type");
@@ -103,7 +117,7 @@ public class TestTimelineServiceClientIntegration {
       // set the timeline service address manually
       // set the timeline service address manually
       client.setTimelineServiceAddress(
       client.setTimelineServiceAddress(
           collectorManager.getRestServerBindAddress());
           collectorManager.getRestServerBindAddress());
-      client.init(new YarnConfiguration());
+      client.init(conf);
       client.start();
       client.start();
       ClusterEntity cluster = new ClusterEntity();
       ClusterEntity cluster = new ClusterEntity();
       cluster.setId(YarnConfiguration.DEFAULT_RM_CLUSTER_ID);
       cluster.setId(YarnConfiguration.DEFAULT_RM_CLUSTER_ID);

+ 10 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java

@@ -30,12 +30,11 @@ import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
 import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
 import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
 import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
 import org.apache.hadoop.yarn.server.api.AuxiliaryService;
 import org.apache.hadoop.yarn.server.api.AuxiliaryService;
-import org.apache.hadoop.yarn.server.api.ContainerContext;
 import org.apache.hadoop.yarn.server.api.ContainerInitializationContext;
 import org.apache.hadoop.yarn.server.api.ContainerInitializationContext;
 import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
 import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.api.ContainerType;
@@ -68,6 +67,9 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
 
 
   @Override
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
   protected void serviceInit(Configuration conf) throws Exception {
+    if (!YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      throw new YarnException("Timeline service v2 is not enabled");
+    }
     collectorManager.init(conf);
     collectorManager.init(conf);
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }
@@ -175,7 +177,8 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
 
 
   @VisibleForTesting
   @VisibleForTesting
   public static PerNodeTimelineCollectorsAuxService
   public static PerNodeTimelineCollectorsAuxService
-      launchServer(String[] args, NodeTimelineCollectorManager collectorManager) {
+      launchServer(String[] args, NodeTimelineCollectorManager collectorManager,
+      Configuration conf) {
     Thread
     Thread
       .setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
       .setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
     StringUtils.startupShutdownMessage(
     StringUtils.startupShutdownMessage(
@@ -187,7 +190,6 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
           new PerNodeTimelineCollectorsAuxService(collectorManager);
           new PerNodeTimelineCollectorsAuxService(collectorManager);
       ShutdownHookManager.get().addShutdownHook(new ShutdownHook(auxService),
       ShutdownHookManager.get().addShutdownHook(new ShutdownHook(auxService),
           SHUTDOWN_HOOK_PRIORITY);
           SHUTDOWN_HOOK_PRIORITY);
-      YarnConfiguration conf = new YarnConfiguration();
       auxService.init(conf);
       auxService.init(conf);
       auxService.start();
       auxService.start();
     } catch (Throwable t) {
     } catch (Throwable t) {
@@ -210,6 +212,9 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
   }
   }
 
 
   public static void main(String[] args) {
   public static void main(String[] args) {
-    launchServer(args, null);
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+    launchServer(args, null, conf);
   }
   }
 }
 }

+ 11 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
@@ -66,6 +67,10 @@ public class TimelineReaderServer extends CompositeService {
 
 
   @Override
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
   protected void serviceInit(Configuration conf) throws Exception {
+    if (!YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      throw new YarnException("timeline service v.2 is not enabled");
+    }
+
     TimelineReader timelineReaderStore = createTimelineReaderStore(conf);
     TimelineReader timelineReaderStore = createTimelineReaderStore(conf);
     addService(timelineReaderStore);
     addService(timelineReaderStore);
     timelineReaderManager = createTimelineReaderManager(timelineReaderStore);
     timelineReaderManager = createTimelineReaderManager(timelineReaderStore);
@@ -143,7 +148,8 @@ public class TimelineReaderServer extends CompositeService {
     return readerWebServer.getConnectorAddress(0).getPort();
     return readerWebServer.getConnectorAddress(0).getPort();
   }
   }
 
 
-  static TimelineReaderServer startTimelineReaderServer(String[] args) {
+  static TimelineReaderServer startTimelineReaderServer(String[] args,
+      Configuration conf) {
     Thread.setDefaultUncaughtExceptionHandler(
     Thread.setDefaultUncaughtExceptionHandler(
         new YarnUncaughtExceptionHandler());
         new YarnUncaughtExceptionHandler());
     StringUtils.startupShutdownMessage(TimelineReaderServer.class,
     StringUtils.startupShutdownMessage(TimelineReaderServer.class,
@@ -154,7 +160,6 @@ public class TimelineReaderServer extends CompositeService {
       ShutdownHookManager.get().addShutdownHook(
       ShutdownHookManager.get().addShutdownHook(
           new CompositeServiceShutdownHook(timelineReaderServer),
           new CompositeServiceShutdownHook(timelineReaderServer),
           SHUTDOWN_HOOK_PRIORITY);
           SHUTDOWN_HOOK_PRIORITY);
-      YarnConfiguration conf = new YarnConfiguration();
       timelineReaderServer.init(conf);
       timelineReaderServer.init(conf);
       timelineReaderServer.start();
       timelineReaderServer.start();
     } catch (Throwable t) {
     } catch (Throwable t) {
@@ -165,6 +170,9 @@ public class TimelineReaderServer extends CompositeService {
   }
   }
 
 
   public static void main(String[] args) {
   public static void main(String[] args) {
-    startTimelineReaderServer(args);
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+    startTimelineReaderServer(args, conf);
   }
   }
 }
 }

+ 7 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java

@@ -50,11 +50,16 @@ import java.io.IOException;
 public class TestPerNodeTimelineCollectorsAuxService {
 public class TestPerNodeTimelineCollectorsAuxService {
   private ApplicationAttemptId appAttemptId;
   private ApplicationAttemptId appAttemptId;
   private PerNodeTimelineCollectorsAuxService auxService;
   private PerNodeTimelineCollectorsAuxService auxService;
+  private Configuration conf;
 
 
   public TestPerNodeTimelineCollectorsAuxService() {
   public TestPerNodeTimelineCollectorsAuxService() {
     ApplicationId appId =
     ApplicationId appId =
         ApplicationId.newInstance(System.currentTimeMillis(), 1);
         ApplicationId.newInstance(System.currentTimeMillis(), 1);
     appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
     appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+    conf = new YarnConfiguration();
+    // enable timeline service v.2
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
   }
   }
 
 
   @After
   @After
@@ -134,7 +139,7 @@ public class TestPerNodeTimelineCollectorsAuxService {
     try {
     try {
       auxService =
       auxService =
           PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
           PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
-              createCollectorManager());
+              createCollectorManager(), conf);
     } catch (ExitUtil.ExitException e) {
     } catch (ExitUtil.ExitException e) {
       assertEquals(0, e.status);
       assertEquals(0, e.status);
       ExitUtil.resetFirstExitException();
       ExitUtil.resetFirstExitException();
@@ -160,7 +165,7 @@ public class TestPerNodeTimelineCollectorsAuxService {
     NodeTimelineCollectorManager collectorManager = createCollectorManager();
     NodeTimelineCollectorManager collectorManager = createCollectorManager();
     PerNodeTimelineCollectorsAuxService auxService =
     PerNodeTimelineCollectorsAuxService auxService =
         spy(new PerNodeTimelineCollectorsAuxService(collectorManager));
         spy(new PerNodeTimelineCollectorsAuxService(collectorManager));
-    auxService.init(new YarnConfiguration());
+    auxService.init(conf);
     auxService.start();
     auxService.start();
     return auxService;
     return auxService;
   }
   }

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java

@@ -30,8 +30,11 @@ public class TestTimelineReaderServer {
 
 
   @Test(timeout = 60000)
   @Test(timeout = 60000)
   public void testStartStopServer() throws Exception {
   public void testStartStopServer() throws Exception {
+    @SuppressWarnings("resource")
     TimelineReaderServer server = new TimelineReaderServer();
     TimelineReaderServer server = new TimelineReaderServer();
     Configuration config = new YarnConfiguration();
     Configuration config = new YarnConfiguration();
+    config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
     config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
     config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
         "localhost:0");
         "localhost:0");
     try {
     try {

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java

@@ -72,6 +72,8 @@ public class TestTimelineReaderWebServices {
   public void init() throws Exception {
   public void init() throws Exception {
     try {
     try {
       Configuration config = new YarnConfiguration();
       Configuration config = new YarnConfiguration();
+      config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+      config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
       config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, 
       config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, 
           "localhost:0");
           "localhost:0");
       config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
       config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java

@@ -238,6 +238,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void init() throws Exception {
   public void init() throws Exception {
     try {
     try {
       Configuration config = util.getConfiguration();
       Configuration config = util.getConfiguration();
+      config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+      config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
       config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
       config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
           "localhost:0");
           "localhost:0");
       config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
       config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");