Forráskód Böngészése

YARN-5355. Backported YARN-2928 into our branch-2 feature branch.

Sangjin Lee 9 éve
szülő
commit
e212e562ec
100 módosított fájl, 7047 hozzáadás és 319 törlés
  1. 1 0
      hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
  2. 286 37
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  3. 19 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  4. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  5. 10 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  6. 16 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
  7. 27 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
  8. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
  9. 26 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
  10. 18 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
  11. 33 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
  12. 18 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
  13. 23 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
  14. 18 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
  15. 18 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
  16. 18 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
  17. 64 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
  18. 25 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
  19. 31 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
  20. 19 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java
  21. 32 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
  22. 26 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
  23. 27 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
  24. 33 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
  25. 27 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
  26. 26 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
  27. 20 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
  28. 18 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
  29. 83 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java
  30. 6 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
  31. 309 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
  32. 39 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
  33. 57 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/EntityWriterV2.java
  34. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java
  35. 1 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java
  36. 161 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java
  37. 43 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterConstants.java
  38. 6 26
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java
  39. 131 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java
  40. 6 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java
  41. 206 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java
  42. 83 47
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineServicePerformance.java
  43. 20 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
  44. 16 16
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java
  45. 127 0
      hadoop-project/pom.xml
  46. 2 0
      hadoop-project/src/site/site.xml
  47. 3 0
      hadoop-yarn-project/hadoop-yarn/bin/yarn
  48. 8 1
      hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
  49. 27 0
      hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
  50. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
  51. 14 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
  52. 6 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java
  53. 3 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java
  54. 41 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationAttemptEntity.java
  55. 52 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationEntity.java
  56. 40 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ClusterEntity.java
  57. 41 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ContainerEntity.java
  58. 191 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java
  59. 126 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java
  60. 133 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/HierarchicalTimelineEntity.java
  61. 40 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/QueueEntity.java
  62. 62 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntities.java
  63. 584 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java
  64. 101 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java
  65. 133 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java
  66. 289 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java
  67. 115 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricCalculator.java
  68. 167 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricOperation.java
  69. 167 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineWriteResponse.java
  70. 40 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/UserEntity.java
  71. 26 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/package-info.java
  72. 150 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  73. 49 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/TimelineServiceHelper.java
  74. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  75. 100 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java
  76. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
  77. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
  78. 160 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  79. 42 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  80. 281 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  81. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java
  82. 19 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
  83. 18 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
  84. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
  85. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
  86. 17 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
  87. 80 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
  88. 452 34
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
  89. 49 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
  90. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
  91. 0 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/timeline/.keep
  92. 85 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  93. 15 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
  94. 15 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
  95. 312 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java
  96. 378 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
  97. 83 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java
  98. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
  99. 73 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocol.java
  100. 34 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocolPB.java

+ 1 - 0
hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md

@@ -128,6 +128,7 @@ REST API compatibility corresponds to both the requests (URLs) and responses to
 * [MR Application Master](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredAppMasterRest.html)
 * [History Server](../../hadoop-mapreduce-client/hadoop-mapreduce-client-hs/HistoryServerRest.html)
 * [Timeline Server v1 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html)
+* [Timeline Service v2 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html)
 
 #### Policy
 

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

@@ -25,10 +25,12 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -44,16 +46,17 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TaskStatus;
 import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.CounterGroup;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
+import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 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.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
@@ -67,16 +70,15 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.node.ArrayNode;
 import org.codehaus.jackson.node.JsonNodeFactory;
-import org.codehaus.jackson.node.ObjectNode;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.sun.jersey.api.client.ClientHandlerException;
@@ -134,8 +136,12 @@ public class JobHistoryEventHandler extends AbstractService
 
   protected TimelineClient timelineClient;
 
+  private boolean timelineServiceV2Enabled = false;
+
   private static String MAPREDUCE_JOB_ENTITY_TYPE = "MAPREDUCE_JOB";
   private static String MAPREDUCE_TASK_ENTITY_TYPE = "MAPREDUCE_TASK";
+  private static final String MAPREDUCE_TASK_ATTEMPT_ENTITY_TYPE =
+      "MAPREDUCE_TASK_ATTEMPT";
 
   public JobHistoryEventHandler(AppContext context, int startCount) {
     super("JobHistoryEventHandler");
@@ -255,19 +261,28 @@ public class JobHistoryEventHandler extends AbstractService
             MRJobConfig.MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD,
             MRJobConfig.DEFAULT_MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD);
 
+    // TODO replace MR specific configurations on timeline service with getting
+    // configuration from RM through registerApplicationMaster() in
+    // ApplicationMasterProtocol with return value for timeline service
+    // configuration status: off, on_with_v1 or on_with_v2.
     if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA,
         MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)) {
-      if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
-        timelineClient = TimelineClient.createTimelineClient();
+      LOG.info("Emitting job history data to the timeline service is enabled");
+      if (YarnConfiguration.timelineServiceEnabled(conf)) {
+
+        timelineClient =
+            ((MRAppMaster.RunningAppContext)context).getTimelineClient();
         timelineClient.init(conf);
-        LOG.info("Timeline service is enabled");
-        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));
       } else {
         LOG.info("Timeline service is not enabled");
       }
     } 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
@@ -592,8 +607,13 @@ public class JobHistoryEventHandler extends AbstractService
         processEventForJobSummary(event.getHistoryEvent(), mi.getJobSummary(),
             event.getJobID());
         if (timelineClient != null) {
-          processEventForTimelineServer(historyEvent, event.getJobID(),
-              event.getTimestamp());
+          if (timelineServiceV2Enabled) {
+            processEventForNewTimelineService(historyEvent, event.getJobID(),
+                event.getTimestamp());
+          } else {
+            processEventForTimelineServer(historyEvent, event.getJobID(),
+                event.getTimestamp());
+          }
         }
         if (LOG.isDebugEnabled()) {
           LOG.debug("In HistoryEventHandler "
@@ -835,11 +855,11 @@ public class JobHistoryEventHandler extends AbstractService
         tEvent.addEventInfo("FINISHED_MAPS", jfe.getFinishedMaps());
         tEvent.addEventInfo("FINISHED_REDUCES", jfe.getFinishedReduces());
         tEvent.addEventInfo("MAP_COUNTERS_GROUPS",
-                countersToJSON(jfe.getMapCounters()));
+            JobHistoryEventUtils.countersToJSON(jfe.getMapCounters()));
         tEvent.addEventInfo("REDUCE_COUNTERS_GROUPS",
-                countersToJSON(jfe.getReduceCounters()));
+            JobHistoryEventUtils.countersToJSON(jfe.getReduceCounters()));
         tEvent.addEventInfo("TOTAL_COUNTERS_GROUPS",
-                countersToJSON(jfe.getTotalCounters()));
+            JobHistoryEventUtils.countersToJSON(jfe.getTotalCounters()));
         tEvent.addEventInfo("JOB_STATUS", JobState.SUCCEEDED.toString());
         tEntity.addEvent(tEvent);
         tEntity.setEntityId(jobId.toString());
@@ -865,7 +885,7 @@ public class JobHistoryEventHandler extends AbstractService
                 tfe.getFailedAttemptID() == null ?
                 "" : tfe.getFailedAttemptID().toString());
         tEvent.addEventInfo("COUNTERS_GROUPS",
-                countersToJSON(tfe.getCounters()));
+            JobHistoryEventUtils.countersToJSON(tfe.getCounters()));
         tEntity.addEvent(tEvent);
         tEntity.setEntityId(tfe.getTaskId().toString());
         tEntity.setEntityType(MAPREDUCE_TASK_ENTITY_TYPE);
@@ -883,7 +903,7 @@ public class JobHistoryEventHandler extends AbstractService
         TaskFinishedEvent tfe2 = (TaskFinishedEvent) event;
         tEvent.addEventInfo("TASK_TYPE", tfe2.getTaskType().toString());
         tEvent.addEventInfo("COUNTERS_GROUPS",
-                countersToJSON(tfe2.getCounters()));
+            JobHistoryEventUtils.countersToJSON(tfe2.getCounters()));
         tEvent.addEventInfo("FINISH_TIME", tfe2.getFinishTime());
         tEvent.addEventInfo("STATUS", TaskStatus.State.SUCCEEDED.toString());
         tEvent.addEventInfo("SUCCESSFUL_TASK_ATTEMPT_ID",
@@ -905,7 +925,6 @@ public class JobHistoryEventHandler extends AbstractService
         tEvent.addEventInfo("START_TIME", tase.getStartTime());
         tEvent.addEventInfo("HTTP_PORT", tase.getHttpPort());
         tEvent.addEventInfo("TRACKER_NAME", tase.getTrackerName());
-        tEvent.addEventInfo("TASK_TYPE", tase.getTaskType().toString());
         tEvent.addEventInfo("SHUFFLE_PORT", tase.getShufflePort());
         tEvent.addEventInfo("CONTAINER_ID", tase.getContainerId() == null ?
             "" : tase.getContainerId().toString());
@@ -938,7 +957,7 @@ public class JobHistoryEventHandler extends AbstractService
         tEvent.addEventInfo("SORT_FINISH_TIME", tauce.getFinishTime());
         tEvent.addEventInfo("MAP_FINISH_TIME", tauce.getFinishTime());
         tEvent.addEventInfo("COUNTERS_GROUPS",
-                countersToJSON(tauce.getCounters()));
+            JobHistoryEventUtils.countersToJSON(tauce.getCounters()));
         tEntity.addEvent(tEvent);
         tEntity.setEntityId(tauce.getTaskId().toString());
         tEntity.setEntityType(MAPREDUCE_TASK_ENTITY_TYPE);
@@ -952,7 +971,7 @@ public class JobHistoryEventHandler extends AbstractService
         tEvent.addEventInfo("STATE", mafe.getState());
         tEvent.addEventInfo("MAP_FINISH_TIME", mafe.getMapFinishTime());
         tEvent.addEventInfo("COUNTERS_GROUPS",
-                countersToJSON(mafe.getCounters()));
+            JobHistoryEventUtils.countersToJSON(mafe.getCounters()));
         tEvent.addEventInfo("HOSTNAME", mafe.getHostname());
         tEvent.addEventInfo("PORT", mafe.getPort());
         tEvent.addEventInfo("RACK_NAME", mafe.getRackName());
@@ -974,7 +993,7 @@ public class JobHistoryEventHandler extends AbstractService
         tEvent.addEventInfo("SHUFFLE_FINISH_TIME", rafe.getShuffleFinishTime());
         tEvent.addEventInfo("SORT_FINISH_TIME", rafe.getSortFinishTime());
         tEvent.addEventInfo("COUNTERS_GROUPS",
-                countersToJSON(rafe.getCounters()));
+            JobHistoryEventUtils.countersToJSON(rafe.getCounters()));
         tEvent.addEventInfo("HOSTNAME", rafe.getHostname());
         tEvent.addEventInfo("PORT", rafe.getPort());
         tEvent.addEventInfo("RACK_NAME", rafe.getRackName());
@@ -993,7 +1012,7 @@ public class JobHistoryEventHandler extends AbstractService
         tEvent.addEventInfo("STATUS", tafe.getTaskStatus());
         tEvent.addEventInfo("STATE", tafe.getState());
         tEvent.addEventInfo("COUNTERS_GROUPS",
-                countersToJSON(tafe.getCounters()));
+            JobHistoryEventUtils.countersToJSON(tafe.getCounters()));
         tEvent.addEventInfo("HOSTNAME", tafe.getHostname());
         tEntity.addEvent(tEvent);
         tEntity.setEntityId(tafe.getTaskId().toString());
@@ -1043,24 +1062,254 @@ public class JobHistoryEventHandler extends AbstractService
     }
   }
 
-  @Private
-  public JsonNode countersToJSON(Counters counters) {
-    ArrayNode nodes = FACTORY.arrayNode();
-    if (counters != null) {
-      for (CounterGroup counterGroup : counters) {
-        ObjectNode groupNode = nodes.addObject();
-        groupNode.put("NAME", counterGroup.getName());
-        groupNode.put("DISPLAY_NAME", counterGroup.getDisplayName());
-        ArrayNode countersNode = groupNode.putArray("COUNTERS");
-        for (Counter counter : counterGroup) {
-          ObjectNode counterNode = countersNode.addObject();
-          counterNode.put("NAME", counter.getName());
-          counterNode.put("DISPLAY_NAME", counter.getDisplayName());
-          counterNode.put("VALUE", counter.getValue());
+  // create JobEntity from HistoryEvent with adding other info, like:
+  // jobId, timestamp and entityType.
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createJobEntity(HistoryEvent event, long timestamp, JobId jobId,
+      String entityType, boolean setCreatedTime) {
+
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
+        createBaseEntity(event, timestamp, entityType, setCreatedTime);
+    entity.setId(jobId.toString());
+    return entity;
+  }
+
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createJobEntity(JobId jobId) {
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
+        new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
+    entity.setId(jobId.toString());
+    entity.setType(MAPREDUCE_JOB_ENTITY_TYPE);
+    return entity;
+  }
+
+  // create ApplicationEntity with job finished Metrics from HistoryEvent
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createAppEntityWithJobMetrics(HistoryEvent event, JobId jobId) {
+    ApplicationEntity entity = new ApplicationEntity();
+    entity.setId(jobId.getAppId().toString());
+    entity.setMetrics(event.getTimelineMetrics());
+    return entity;
+  }
+
+  // create BaseEntity from HistoryEvent with adding other info, like:
+  // timestamp and entityType.
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createBaseEntity(HistoryEvent event, long timestamp, String entityType,
+      boolean setCreatedTime) {
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent tEvent =
+        event.toTimelineEvent();
+    tEvent.setTimestamp(timestamp);
+
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
+        new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
+    entity.addEvent(tEvent);
+    entity.setType(entityType);
+    if (setCreatedTime) {
+      entity.setCreatedTime(timestamp);
+    }
+    Set<TimelineMetric> timelineMetrics = event.getTimelineMetrics();
+    if (timelineMetrics != null) {
+      entity.setMetrics(timelineMetrics);
+    }
+    return entity;
+  }
+
+  // create TaskEntity from HistoryEvent with adding other info, like:
+  // taskId, jobId, timestamp, entityType and relatedJobEntity.
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createTaskEntity(HistoryEvent event, long timestamp, String taskId,
+      String entityType, String relatedJobEntity, JobId jobId,
+      boolean setCreatedTime) {
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
+        createBaseEntity(event, timestamp, entityType, setCreatedTime);
+    entity.setId(taskId);
+    if (event.getEventType() == EventType.TASK_STARTED) {
+      entity.addInfo("TASK_TYPE",
+          ((TaskStartedEvent)event).getTaskType().toString());
+    }
+    entity.addIsRelatedToEntity(relatedJobEntity, jobId.toString());
+    return entity;
+  }
+
+  // create TaskAttemptEntity from HistoryEvent with adding other info, like:
+  // timestamp, taskAttemptId, entityType, relatedTaskEntity and taskId.
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createTaskAttemptEntity(HistoryEvent event, long timestamp,
+      String taskAttemptId, String entityType, String relatedTaskEntity,
+      String taskId, boolean setCreatedTime) {
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
+        createBaseEntity(event, timestamp, entityType, setCreatedTime);
+    entity.setId(taskAttemptId);
+    entity.addIsRelatedToEntity(relatedTaskEntity, taskId);
+    return entity;
+  }
+
+  private void publishConfigsOnJobSubmittedEvent(JobSubmittedEvent event,
+      JobId jobId) {
+    if (event.getJobConf() == null) {
+      return;
+    }
+    // Publish job configurations both as job and app entity.
+    // Configs are split into multiple entities if they exceed 100kb in size.
+    org.apache.hadoop.yarn.api.records.timelineservice.
+        TimelineEntity jobEntityForConfigs = createJobEntity(jobId);
+    ApplicationEntity appEntityForConfigs = new ApplicationEntity();
+    String appId = jobId.getAppId().toString();
+    appEntityForConfigs.setId(appId);
+    try {
+      int configSize = 0;
+      for (Map.Entry<String, String> entry : event.getJobConf()) {
+        int size = entry.getKey().length() + entry.getValue().length();
+        configSize += size;
+        if (configSize > JobHistoryEventUtils.ATS_CONFIG_PUBLISH_SIZE_BYTES) {
+          if (jobEntityForConfigs.getConfigs().size() > 0) {
+            timelineClient.putEntities(jobEntityForConfigs);
+            timelineClient.putEntities(appEntityForConfigs);
+            jobEntityForConfigs = createJobEntity(jobId);
+            appEntityForConfigs = new ApplicationEntity();
+            appEntityForConfigs.setId(appId);
+          }
+          configSize = size;
         }
+        jobEntityForConfigs.addConfig(entry.getKey(), entry.getValue());
+        appEntityForConfigs.addConfig(entry.getKey(), entry.getValue());
+      }
+      if (configSize > 0) {
+        timelineClient.putEntities(jobEntityForConfigs);
+        timelineClient.putEntities(appEntityForConfigs);
       }
+    } catch (IOException | YarnException e) {
+      LOG.error("Exception while publishing configs on JOB_SUBMITTED Event " +
+          " for the job : " + jobId, e);
+    }
+  }
+
+  private void processEventForNewTimelineService(HistoryEvent event,
+      JobId jobId, long timestamp) {
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity tEntity =
+        null;
+    String taskId = null;
+    String taskAttemptId = null;
+    boolean setCreatedTime = false;
+
+    switch (event.getEventType()) {
+    // Handle job events
+    case JOB_SUBMITTED:
+      setCreatedTime = true;
+      break;
+    case JOB_STATUS_CHANGED:
+    case JOB_INFO_CHANGED:
+    case JOB_INITED:
+    case JOB_PRIORITY_CHANGED:
+    case JOB_QUEUE_CHANGED:
+    case JOB_FAILED:
+    case JOB_KILLED:
+    case JOB_ERROR:
+    case JOB_FINISHED:
+    case AM_STARTED:
+    case NORMALIZED_RESOURCE:
+      break;
+    // Handle task events
+    case TASK_STARTED:
+      setCreatedTime = true;
+      taskId = ((TaskStartedEvent)event).getTaskId().toString();
+      break;
+    case TASK_FAILED:
+      taskId = ((TaskFailedEvent)event).getTaskId().toString();
+      break;
+    case TASK_UPDATED:
+      taskId = ((TaskUpdatedEvent)event).getTaskId().toString();
+      break;
+    case TASK_FINISHED:
+      taskId = ((TaskFinishedEvent)event).getTaskId().toString();
+      break;
+    case MAP_ATTEMPT_STARTED:
+    case REDUCE_ATTEMPT_STARTED:
+      setCreatedTime = true;
+      taskId = ((TaskAttemptStartedEvent)event).getTaskId().toString();
+      taskAttemptId = ((TaskAttemptStartedEvent)event).
+          getTaskAttemptId().toString();
+      break;
+    case CLEANUP_ATTEMPT_STARTED:
+    case SETUP_ATTEMPT_STARTED:
+      taskId = ((TaskAttemptStartedEvent)event).getTaskId().toString();
+      taskAttemptId = ((TaskAttemptStartedEvent)event).
+          getTaskAttemptId().toString();
+      break;
+    case MAP_ATTEMPT_FAILED:
+    case CLEANUP_ATTEMPT_FAILED:
+    case REDUCE_ATTEMPT_FAILED:
+    case SETUP_ATTEMPT_FAILED:
+    case MAP_ATTEMPT_KILLED:
+    case CLEANUP_ATTEMPT_KILLED:
+    case REDUCE_ATTEMPT_KILLED:
+    case SETUP_ATTEMPT_KILLED:
+      taskId = ((TaskAttemptUnsuccessfulCompletionEvent)event).
+          getTaskId().toString();
+      taskAttemptId = ((TaskAttemptUnsuccessfulCompletionEvent)event).
+          getTaskAttemptId().toString();
+      break;
+    case MAP_ATTEMPT_FINISHED:
+      taskId = ((MapAttemptFinishedEvent)event).getTaskId().toString();
+      taskAttemptId = ((MapAttemptFinishedEvent)event).
+          getAttemptId().toString();
+      break;
+    case REDUCE_ATTEMPT_FINISHED:
+      taskId = ((ReduceAttemptFinishedEvent)event).getTaskId().toString();
+      taskAttemptId = ((ReduceAttemptFinishedEvent)event).
+          getAttemptId().toString();
+      break;
+    case SETUP_ATTEMPT_FINISHED:
+    case CLEANUP_ATTEMPT_FINISHED:
+      taskId = ((TaskAttemptFinishedEvent)event).getTaskId().toString();
+      taskAttemptId = ((TaskAttemptFinishedEvent)event).
+          getAttemptId().toString();
+      break;
+    default:
+      LOG.warn("EventType: " + event.getEventType() + " cannot be recognized" +
+          " and handled by timeline service.");
+      return;
+    }
+
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+        appEntityWithJobMetrics = null;
+    if (taskId == null) {
+      // JobEntity
+      tEntity = createJobEntity(event, timestamp, jobId,
+          MAPREDUCE_JOB_ENTITY_TYPE, setCreatedTime);
+      if (event.getEventType() == EventType.JOB_FINISHED
+          && event.getTimelineMetrics() != null) {
+        appEntityWithJobMetrics = createAppEntityWithJobMetrics(event, jobId);
+      }
+    } else {
+      if (taskAttemptId == null) {
+        // TaskEntity
+        tEntity = createTaskEntity(event, timestamp, taskId,
+            MAPREDUCE_TASK_ENTITY_TYPE, MAPREDUCE_JOB_ENTITY_TYPE,
+            jobId, setCreatedTime);
+      } else {
+        // TaskAttemptEntity
+        tEntity = createTaskAttemptEntity(event, timestamp, taskAttemptId,
+            MAPREDUCE_TASK_ATTEMPT_ENTITY_TYPE, MAPREDUCE_TASK_ENTITY_TYPE,
+            taskId, setCreatedTime);
+      }
+    }
+    try {
+      if (appEntityWithJobMetrics == null) {
+        timelineClient.putEntitiesAsync(tEntity);
+      } else {
+        timelineClient.putEntities(tEntity, appEntityWithJobMetrics);
+      }
+    } catch (IOException | YarnException e) {
+      LOG.error("Failed to process Event " + event.getEventType()
+          + " for the job : " + jobId, e);
+      return;
+    }
+    if (event.getEventType() == EventType.JOB_SUBMITTED) {
+      // Publish configs after main job submitted event has been posted.
+      publishConfigsOnJobSubmittedEvent((JobSubmittedEvent)event, jobId);
     }
-    return nodes;
   }
 
   private void setSummarySlotSeconds(JobSummary summary, Counters allCounters) {

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

@@ -137,6 +137,7 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+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.Dispatcher;
@@ -1057,6 +1058,7 @@ public class MRAppMaster extends CompositeService {
     private final ClusterInfo clusterInfo = new ClusterInfo();
     private final ClientToAMTokenSecretManager clientToAMTokenSecretManager;
     private String historyUrl = null;
+    private TimelineClient timelineClient = null;
 
     private final TaskAttemptFinishingMonitor taskAttemptFinishingMonitor;
 
@@ -1066,6 +1068,18 @@ public class MRAppMaster extends CompositeService {
       this.clientToAMTokenSecretManager =
           new ClientToAMTokenSecretManager(appAttemptID, null);
       this.taskAttemptFinishingMonitor = taskAttemptFinishingMonitor;
+      if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA,
+              MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)
+            && YarnConfiguration.timelineServiceEnabled(conf)) {
+
+        if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+          // create new version TimelineClient
+          timelineClient = TimelineClient.createTimelineClient(
+              appAttemptID.getApplicationId());
+        } else {
+          timelineClient = TimelineClient.createTimelineClient();
+        }
+      }
     }
 
     @Override
@@ -1165,6 +1179,11 @@ public class MRAppMaster extends CompositeService {
     public void setHistoryUrl(String historyUrl) {
       this.historyUrl = historyUrl;
     }
+
+    // Get Timeline Collector's address (get sync from RM)
+    public TimelineClient getTimelineClient() {
+      return timelineClient;
+    }
   }
 
   @SuppressWarnings("unchecked")

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -1465,7 +1465,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
             job.conf.get(MRJobConfig.WORKFLOW_NAME, ""),
             job.conf.get(MRJobConfig.WORKFLOW_NODE_NAME, ""),
             getWorkflowAdjacencies(job.conf),
-            job.conf.get(MRJobConfig.WORKFLOW_TAGS, ""));
+            job.conf.get(MRJobConfig.WORKFLOW_TAGS, ""), job.conf);
         job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse));
         //TODO JH Verify jobACLs, UserName via UGI?
 

+ 10 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.MRAppMaster;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent;
@@ -858,6 +859,15 @@ public class RMContainerAllocator extends RMContainerRequestor
 
     handleUpdatedNodes(response);
     handleJobPriorityChange(response);
+    // handle receiving the timeline collector address for this app
+    String collectorAddr = response.getCollectorAddr();
+    MRAppMaster.RunningAppContext appContext =
+        (MRAppMaster.RunningAppContext)this.getContext();
+    if (collectorAddr != null && !collectorAddr.isEmpty()
+        && appContext.getTimelineClient() != null) {
+      appContext.getTimelineClient().setTimelineServiceAddress(
+          response.getCollectorAddr());
+    }
 
     for (ContainerStatus cont : finishedContainers) {
       processFinishedContainer(cont);

+ 16 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java

@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
-
-import static org.junit.Assert.*;
+import java.util.Set;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -35,6 +37,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.junit.Test;
 
 public class TestEvents {
@@ -405,6 +409,16 @@ public class TestEvents {
       this.datum = datum;
     }
 
+    @Override
+    public TimelineEvent toTimelineEvent() {
+      return null;
+    }
+
+    @Override
+    public Set<TimelineMetric> getTimelineMetrics() {
+      return null;
+    }
+
   }
 
 }

+ 27 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java

@@ -56,8 +56,10 @@ import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
+import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.MRAppMaster.RunningAppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
@@ -70,6 +72,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
@@ -560,7 +563,7 @@ public class TestJobHistoryEventHandler {
   // stored to the Timeline store
   @Test (timeout=50000)
   public void testTimelineEventHandling() throws Exception {
-    TestParams t = new TestParams(false);
+    TestParams t = new TestParams(RunningAppContext.class, false);
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     long currentTime = System.currentTimeMillis();
@@ -738,7 +741,7 @@ public class TestJobHistoryEventHandler {
     group2.addCounter("MARTHA_JONES", "Martha Jones", 3);
     group2.addCounter("DONNA_NOBLE", "Donna Noble", 2);
     group2.addCounter("ROSE_TYLER", "Rose Tyler", 1);
-    JsonNode jsonNode = jheh.countersToJSON(counters);
+    JsonNode jsonNode = JobHistoryEventUtils.countersToJSON(counters);
     String jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
     String expected = "[{\"NAME\":\"COMPANIONS\",\"DISPLAY_NAME\":\"Companions "
         + "of the Doctor\",\"COUNTERS\":[{\"NAME\":\"AMY_POND\",\"DISPLAY_NAME\""
@@ -761,19 +764,19 @@ public class TestJobHistoryEventHandler {
   public void testCountersToJSONEmpty() throws Exception {
     JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0);
     Counters counters = null;
-    JsonNode jsonNode = jheh.countersToJSON(counters);
+    JsonNode jsonNode = JobHistoryEventUtils.countersToJSON(counters);
     String jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
     String expected = "[]";
     Assert.assertEquals(expected, jsonStr);
 
     counters = new Counters();
-    jsonNode = jheh.countersToJSON(counters);
+    jsonNode = JobHistoryEventUtils.countersToJSON(counters);
     jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
     expected = "[]";
     Assert.assertEquals(expected, jsonStr);
 
     counters.addGroup("DOCTORS", "Incarnations of the Doctor");
-    jsonNode = jheh.countersToJSON(counters);
+    jsonNode = JobHistoryEventUtils.countersToJSON(counters);
     jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
     expected = "[{\"NAME\":\"DOCTORS\",\"DISPLAY_NAME\":\"Incarnations of the "
         + "Doctor\",\"COUNTERS\":[]}]";
@@ -809,21 +812,30 @@ public class TestJobHistoryEventHandler {
     }
   }
 
-  private AppContext mockAppContext(ApplicationId appId, boolean isLastAMRetry) {
-    JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId));
-    AppContext mockContext = mock(AppContext.class);
+  private Job mockJob() {
     Job mockJob = mock(Job.class);
     when(mockJob.getAllCounters()).thenReturn(new Counters());
     when(mockJob.getTotalMaps()).thenReturn(10);
     when(mockJob.getTotalReduces()).thenReturn(10);
     when(mockJob.getName()).thenReturn("mockjob");
+    return mockJob;
+  }
+
+  private AppContext mockAppContext(Class<? extends AppContext> contextClass,
+      ApplicationId appId, boolean isLastAMRetry) {
+    JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId));
+    AppContext mockContext = mock(contextClass);
+    Job mockJob = mockJob();
     when(mockContext.getJob(jobId)).thenReturn(mockJob);
     when(mockContext.getApplicationID()).thenReturn(appId);
     when(mockContext.isLastAMRetry()).thenReturn(isLastAMRetry);
+    if (mockContext instanceof RunningAppContext) {
+      when(((RunningAppContext)mockContext).getTimelineClient()).
+          thenReturn(TimelineClient.createTimelineClient());
+    }
     return mockContext;
   }
 
-
   private class TestParams {
     boolean isLastAMRetry;
     String workDir = setupTestWorkDir();
@@ -838,11 +850,15 @@ public class TestJobHistoryEventHandler {
     AppContext mockAppContext;
 
     public TestParams() {
-      this(false);
+      this(AppContext.class, false);
     }
     public TestParams(boolean isLastAMRetry) {
+      this(AppContext.class, isLastAMRetry);
+    }
+    public TestParams(Class<? extends AppContext> contextClass,
+        boolean isLastAMRetry) {
       this.isLastAMRetry = isLastAMRetry;
-      mockAppContext = mockAppContext(appId, this.isLastAMRetry);
+      mockAppContext = mockAppContext(contextClass, appId, this.isLastAMRetry);
     }
   }
 

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java

@@ -61,6 +61,7 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.ClusterInfo;
 import org.apache.hadoop.mapreduce.v2.app.MRApp;
 import org.apache.hadoop.mapreduce.v2.app.MRAppMaster;
+import org.apache.hadoop.mapreduce.v2.app.MRAppMaster.RunningAppContext;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
@@ -1869,7 +1870,7 @@ public class TestRMContainerAllocator {
     private AllocateResponse allocateResponse;
     private static AppContext createAppContext(
         ApplicationAttemptId appAttemptId, Job job) {
-      AppContext context = mock(AppContext.class);
+      AppContext context = mock(RunningAppContext.class);
       ApplicationId appId = appAttemptId.getApplicationId();
       when(context.getApplicationID()).thenReturn(appId);
       when(context.getApplicationAttemptId()).thenReturn(appAttemptId);

+ 26 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java

@@ -18,14 +18,18 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
-import org.apache.avro.util.Utf8;
-
 /**
  * Event to record start of a task attempt
  * 
@@ -166,4 +170,24 @@ public class AMStartedEvent implements HistoryEvent {
   public EventType getEventType() {
     return EventType.AM_STARTED;
   }
+
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("APPLICATION_ATTEMPT_ID",
+        getAppAttemptId() == null ? "" : getAppAttemptId().toString());
+    tEvent.addInfo("CONTAINER_ID", getContainerId() == null ?
+        "" : getContainerId().toString());
+    tEvent.addInfo("NODE_MANAGER_HOST", getNodeManagerHost());
+    tEvent.addInfo("NODE_MANAGER_PORT", getNodeManagerPort());
+    tEvent.addInfo("NODE_MANAGER_HTTP_PORT", getNodeManagerHttpPort());
+    tEvent.addInfo("START_TIME", getStartTime());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

+ 18 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java

@@ -18,8 +18,12 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Interface for event wrapper classes.  Implementations each wrap an
@@ -37,4 +41,18 @@ public interface HistoryEvent {
 
   /** Set the Avro datum wrapped by this. */
   void setDatum(Object datum);
+
+  /**
+   * Map HistoryEvent to TimelineEvent.
+   *
+   * @return the timeline event
+   */
+  TimelineEvent toTimelineEvent();
+
+  /**
+   * Counters or Metrics if any else return null.
+   *
+   * @return the set of timeline metrics
+   */
+  Set<TimelineMetric> getTimelineMetrics();
 }

+ 33 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java

@@ -18,11 +18,17 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record successful completion of job
@@ -133,4 +139,31 @@ public class JobFinishedEvent  implements HistoryEvent {
   public Counters getReduceCounters() {
     return reduceCounters;
   }
+
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("FINISH_TIME", getFinishTime());
+    tEvent.addInfo("NUM_MAPS", getFinishedMaps());
+    tEvent.addInfo("NUM_REDUCES", getFinishedReduces());
+    tEvent.addInfo("FAILED_MAPS", getFailedMaps());
+    tEvent.addInfo("FAILED_REDUCES", getFailedReduces());
+    tEvent.addInfo("FINISHED_MAPS", getFinishedMaps());
+    tEvent.addInfo("FINISHED_REDUCES", getFinishedReduces());
+    // TODO replace SUCCEEDED with JobState.SUCCEEDED.toString()
+    tEvent.addInfo("JOB_STATUS", "SUCCEEDED");
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> jobMetrics = JobHistoryEventUtils.
+        countersToTimelineMetric(getTotalCounters(), finishTime);
+    jobMetrics.addAll(JobHistoryEventUtils.
+        countersToTimelineMetric(getMapCounters(), finishTime, "MAP:"));
+    jobMetrics.addAll(JobHistoryEventUtils.
+        countersToTimelineMetric(getReduceCounters(), finishTime, "REDUCE:"));
+    return jobMetrics;
+  }
 }

+ 18 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java

@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobID;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record changes in the submit and launch time of
@@ -65,4 +67,17 @@ public class JobInfoChangeEvent implements HistoryEvent {
     return EventType.JOB_INFO_CHANGED;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("SUBMIT_TIME", getSubmitTime());
+    tEvent.addInfo("LAUNCH_TIME", getLaunchTime());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

+ 23 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java

@@ -18,11 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobID;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the initialization of a job
@@ -73,4 +77,21 @@ public class JobInitedEvent implements HistoryEvent {
   }
   /** Get whether the job's map and reduce stages were combined */
   public boolean getUberized() { return datum.getUberized(); }
+
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("START_TIME", getLaunchTime());
+    tEvent.addInfo("STATUS", getStatus());
+    tEvent.addInfo("TOTAL_MAPS", getTotalMaps());
+    tEvent.addInfo("TOTAL_REDUCES", getTotalReduces());
+    tEvent.addInfo("UBERIZED", getUberized());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

+ 18 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java

@@ -18,14 +18,16 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.JobPriority;
 import org.apache.hadoop.mapreduce.JobID;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the change of priority of a job
@@ -65,4 +67,17 @@ public class JobPriorityChangeEvent implements HistoryEvent {
     return EventType.JOB_PRIORITY_CHANGED;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("PRIORITY", getPriority().toString());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

+ 18 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java

@@ -18,8 +18,13 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 @SuppressWarnings("deprecation")
 public class JobQueueChangeEvent implements HistoryEvent {
@@ -60,4 +65,17 @@ public class JobQueueChangeEvent implements HistoryEvent {
     return null;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("QUEUE_NAMES", getJobQueueName());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

+ 18 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java

@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobID;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the change of status for a job
@@ -61,4 +63,17 @@ public class JobStatusChangedEvent implements HistoryEvent {
     return EventType.JOB_STATUS_CHANGED;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("STATUS", getStatus());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

+ 64 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java

@@ -21,14 +21,18 @@ package org.apache.hadoop.mapreduce.jobhistory;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.security.authorize.AccessControlList;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the submission of a job
@@ -38,6 +42,7 @@ import org.apache.avro.util.Utf8;
 @InterfaceStability.Unstable
 public class JobSubmittedEvent implements HistoryEvent {
   private JobSubmitted datum = new JobSubmitted();
+  private JobConf jobConf = null;
 
   /**
    * Create an event to record job submission
@@ -80,6 +85,31 @@ public class JobSubmittedEvent implements HistoryEvent {
         workflowAdjacencies, "");
   }
 
+  /**
+   * Create an event to record job submission.
+   * @param id The job Id of the job
+   * @param jobName Name of the job
+   * @param userName Name of the user who submitted the job
+   * @param submitTime Time of submission
+   * @param jobConfPath Path of the Job Configuration file
+   * @param jobACLs The configured acls for the job.
+   * @param jobQueueName The job-queue to which this job was submitted to
+   * @param workflowId The Id of the workflow
+   * @param workflowName The name of the workflow
+   * @param workflowNodeName The node name of the workflow
+   * @param workflowAdjacencies The adjacencies of the workflow
+   * @param workflowTags Comma-separated tags for the workflow
+   */
+  public JobSubmittedEvent(JobID id, String jobName, String userName,
+      long submitTime, String jobConfPath,
+      Map<JobACL, AccessControlList> jobACLs, String jobQueueName,
+      String workflowId, String workflowName, String workflowNodeName,
+      String workflowAdjacencies, String workflowTags) {
+    this(id, jobName, userName, submitTime, jobConfPath, jobACLs,
+        jobQueueName, workflowId, workflowName, workflowNodeName,
+        workflowAdjacencies, workflowTags, null);
+  }
+
     /**
      * Create an event to record job submission
      * @param id The job Id of the job
@@ -94,12 +124,13 @@ public class JobSubmittedEvent implements HistoryEvent {
      * @param workflowNodeName The node name of the workflow
      * @param workflowAdjacencies The adjacencies of the workflow
      * @param workflowTags Comma-separated tags for the workflow
+     * @param conf Job configuration
      */
     public JobSubmittedEvent(JobID id, String jobName, String userName,
         long submitTime, String jobConfPath,
         Map<JobACL, AccessControlList> jobACLs, String jobQueueName,
         String workflowId, String workflowName, String workflowNodeName,
-        String workflowAdjacencies, String workflowTags) {
+        String workflowAdjacencies, String workflowTags, JobConf conf) {
     datum.setJobid(new Utf8(id.toString()));
     datum.setJobName(new Utf8(jobName));
     datum.setUserName(new Utf8(userName));
@@ -129,6 +160,7 @@ public class JobSubmittedEvent implements HistoryEvent {
     if (workflowTags != null) {
       datum.setWorkflowTags(new Utf8(workflowTags));
     }
+    jobConf = conf;
   }
 
   JobSubmittedEvent() {}
@@ -206,4 +238,33 @@ public class JobSubmittedEvent implements HistoryEvent {
   /** Get the event type */
   public EventType getEventType() { return EventType.JOB_SUBMITTED; }
 
+  public JobConf getJobConf() {
+    return jobConf;
+  }
+
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("SUBMIT_TIME", getSubmitTime());
+    tEvent.addInfo("QUEUE_NAME", getJobQueueName());
+    tEvent.addInfo("JOB_NAME", getJobName());
+    tEvent.addInfo("USER_NAME", getUserName());
+    tEvent.addInfo("JOB_CONF_PATH", getJobConfPath());
+    tEvent.addInfo("ACLS", getJobAcls());
+    tEvent.addInfo("JOB_QUEUE_NAME", getJobQueueName());
+    tEvent.addInfo("WORKLFOW_ID", getWorkflowId());
+    tEvent.addInfo("WORKFLOW_NAME", getWorkflowName());
+    tEvent.addInfo("WORKFLOW_NODE_NAME", getWorkflowNodeName());
+    tEvent.addInfo("WORKFLOW_ADJACENCIES",
+        getWorkflowAdjacencies());
+    tEvent.addInfo("WORKFLOW_TAGS", getWorkflowTags());
+
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

+ 25 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java

@@ -18,14 +18,18 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import com.google.common.base.Joiner;
+import java.util.Collections;
+import java.util.Set;
 
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
-import java.util.Collections;
+import com.google.common.base.Joiner;
 
 /**
  * Event to record Failed and Killed completion of jobs
@@ -119,4 +123,23 @@ public class JobUnsuccessfulCompletionEvent implements HistoryEvent {
     final CharSequence diagnostics = datum.getDiagnostics();
     return diagnostics == null ? NODIAGS : diagnostics.toString();
   }
+
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("FINISH_TIME", getFinishTime());
+    tEvent.addInfo("NUM_MAPS", getFinishedMaps());
+    tEvent.addInfo("NUM_REDUCES", getFinishedReduces());
+    tEvent.addInfo("JOB_STATUS", getStatus());
+    tEvent.addInfo("DIAGNOSTICS", getDiagnostics());
+    tEvent.addInfo("FINISHED_MAPS", getFinishedMaps());
+    tEvent.addInfo("FINISHED_REDUCES", getFinishedReduces());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

+ 31 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -26,6 +28,10 @@ import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record successful completion of a map attempt
@@ -33,7 +39,7 @@ import org.apache.hadoop.mapreduce.TaskType;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class MapAttemptFinishedEvent  implements HistoryEvent {
+public class MapAttemptFinishedEvent implements HistoryEvent {
 
   private MapAttemptFinished datum = null;
 
@@ -218,4 +224,28 @@ public class MapAttemptFinishedEvent  implements HistoryEvent {
     return physMemKbytes;
   }
   
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("TASK_TYPE", getTaskType().toString());
+    tEvent.addInfo("FINISH_TIME", getFinishTime());
+    tEvent.addInfo("STATUS", getTaskStatus());
+    tEvent.addInfo("STATE", getState());
+    tEvent.addInfo("MAP_FINISH_TIME", getMapFinishTime());
+    tEvent.addInfo("HOSTNAME", getHostname());
+    tEvent.addInfo("PORT", getPort());
+    tEvent.addInfo("RACK_NAME", getRackName());
+    tEvent.addInfo("ATTEMPT_ID", getAttemptId() == null ?
+        "" : getAttemptId().toString());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
+
 }

+ 19 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java

@@ -17,9 +17,14 @@
  */
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the normalized map/reduce requirements.
@@ -71,4 +76,18 @@ public class NormalizedResourceEvent implements HistoryEvent {
   public void setDatum(Object datum) {
     throw new UnsupportedOperationException("Not a seriable object");
   }
+
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("MEMORY", "" + getMemory());
+    tEvent.addInfo("TASK_TYPE", getTaskType());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

+ 32 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -26,6 +28,10 @@ import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record successful completion of a reduce attempt
@@ -33,7 +39,7 @@ import org.apache.hadoop.mapreduce.TaskType;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class ReduceAttemptFinishedEvent  implements HistoryEvent {
+public class ReduceAttemptFinishedEvent implements HistoryEvent {
 
   private ReduceAttemptFinished datum = null;
 
@@ -223,4 +229,29 @@ public class ReduceAttemptFinishedEvent  implements HistoryEvent {
     return physMemKbytes;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("TASK_TYPE", getTaskType().toString());
+    tEvent.addInfo("ATTEMPT_ID", getAttemptId() == null ?
+        "" : getAttemptId().toString());
+    tEvent.addInfo("FINISH_TIME", getFinishTime());
+    tEvent.addInfo("STATUS", getTaskStatus());
+    tEvent.addInfo("STATE", getState());
+    tEvent.addInfo("SHUFFLE_FINISH_TIME", getShuffleFinishTime());
+    tEvent.addInfo("SORT_FINISH_TIME", getSortFinishTime());
+    tEvent.addInfo("HOSTNAME", getHostname());
+    tEvent.addInfo("PORT", getPort());
+    tEvent.addInfo("RACK_NAME", getRackName());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
+
 }

+ 26 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -25,6 +27,10 @@ import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record successful task completion
@@ -136,4 +142,24 @@ public class TaskAttemptFinishedEvent  implements HistoryEvent {
            : EventType.REDUCE_ATTEMPT_FINISHED;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("TASK_TYPE", getTaskType().toString());
+    tEvent.addInfo("ATTEMPT_ID", getAttemptId() == null ?
+        "" : getAttemptId().toString());
+    tEvent.addInfo("FINISH_TIME", getFinishTime());
+    tEvent.addInfo("STATUS", getTaskStatus());
+    tEvent.addInfo("STATE", getState());
+    tEvent.addInfo("HOSTNAME", getHostname());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
 }

+ 27 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java

@@ -18,16 +18,20 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
-import org.apache.avro.util.Utf8;
-
 /**
  * Event to record start of a task attempt
  *
@@ -133,4 +137,25 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
     return null;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("TASK_TYPE", getTaskType().toString());
+    tEvent.addInfo("TASK_ATTEMPT_ID",
+        getTaskAttemptId().toString());
+    tEvent.addInfo("START_TIME", getStartTime());
+    tEvent.addInfo("HTTP_PORT", getHttpPort());
+    tEvent.addInfo("TRACKER_NAME", getTrackerName());
+    tEvent.addInfo("SHUFFLE_PORT", getShufflePort());
+    tEvent.addInfo("CONTAINER_ID", getContainerId() == null ?
+        "" : getContainerId().toString());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

+ 33 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java

@@ -18,17 +18,21 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapred.ProgressSplitsBlock;
 import org.apache.hadoop.mapred.TaskStatus;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
-
-import org.apache.hadoop.mapred.ProgressSplitsBlock;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record unsuccessful (Killed/Failed) completion of task attempts
@@ -248,4 +252,29 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
     return physMemKbytes;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("TASK_TYPE", getTaskType().toString());
+    tEvent.addInfo("TASK_ATTEMPT_ID", getTaskAttemptId() == null ?
+        "" : getTaskAttemptId().toString());
+    tEvent.addInfo("FINISH_TIME", getFinishTime());
+    tEvent.addInfo("ERROR", getError());
+    tEvent.addInfo("STATUS", getTaskStatus());
+    tEvent.addInfo("HOSTNAME", getHostname());
+    tEvent.addInfo("PORT", getPort());
+    tEvent.addInfo("RACK_NAME", getRackName());
+    tEvent.addInfo("SHUFFLE_FINISH_TIME", getFinishTime());
+    tEvent.addInfo("SORT_FINISH_TIME", getFinishTime());
+    tEvent.addInfo("MAP_FINISH_TIME", getFinishTime());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
 }

+ 27 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java

@@ -18,14 +18,20 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapred.TaskStatus;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the failure of a task
@@ -137,4 +143,23 @@ public class TaskFailedEvent implements HistoryEvent {
     return EventType.TASK_FAILED;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("TASK_TYPE", getTaskType().toString());
+    tEvent.addInfo("STATUS", TaskStatus.State.FAILED.toString());
+    tEvent.addInfo("FINISH_TIME", getFinishTime());
+    tEvent.addInfo("ERROR", getError());
+    tEvent.addInfo("FAILED_ATTEMPT_ID",
+        getFailedAttemptID() == null ? "" : getFailedAttemptID().toString());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
 }

+ 26 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java

@@ -18,13 +18,20 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapred.TaskStatus;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the successful completion of a task
@@ -115,5 +122,23 @@ public class TaskFinishedEvent implements HistoryEvent {
     return EventType.TASK_FINISHED;
   }
 
-  
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("TASK_TYPE", getTaskType().toString());
+    tEvent.addInfo("FINISH_TIME", getFinishTime());
+    tEvent.addInfo("STATUS", TaskStatus.State.SUCCEEDED.toString());
+    tEvent.addInfo("SUCCESSFUL_TASK_ATTEMPT_ID",
+        getSuccessfulTaskAttemptId() == null ? "" :
+            getSuccessfulTaskAttemptId().toString());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> jobMetrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return jobMetrics;
+  }
 }

+ 20 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java

@@ -18,11 +18,16 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the start of a task
@@ -72,4 +77,19 @@ public class TaskStartedEvent implements HistoryEvent {
     return EventType.TASK_STARTED;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("TASK_TYPE", getTaskType().toString());
+    tEvent.addInfo("START_TIME", getStartTime());
+    tEvent.addInfo("SPLIT_LOCATIONS", getSplitLocations());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

+ 18 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java

@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.TaskID;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record updates to a task
@@ -61,4 +63,17 @@ public class TaskUpdatedEvent implements HistoryEvent {
     return EventType.TASK_UPDATED;
   }
 
+  @Override
+  public TimelineEvent toTimelineEvent() {
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
+    tEvent.addInfo("FINISH_TIME", getFinishTime());
+    return tEvent;
+  }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

+ 83 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java

@@ -0,0 +1,83 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.hadoop.mapreduce.util;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.CounterGroup;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.node.ArrayNode;
+import org.codehaus.jackson.node.ObjectNode;
+
+/**
+ * Class containing utility methods to be used by JobHistoryEventHandler.
+ */
+public final class JobHistoryEventUtils {
+  private JobHistoryEventUtils() {
+  }
+
+  // Number of bytes of config which can be published in one shot to ATSv2.
+  public static final int ATS_CONFIG_PUBLISH_SIZE_BYTES = 10 * 1024;
+
+  public static JsonNode countersToJSON(Counters counters) {
+    ObjectMapper mapper = new ObjectMapper();
+    ArrayNode nodes = mapper.createArrayNode();
+    if (counters != null) {
+      for (CounterGroup counterGroup : counters) {
+        ObjectNode groupNode = nodes.addObject();
+        groupNode.put("NAME", counterGroup.getName());
+        groupNode.put("DISPLAY_NAME", counterGroup.getDisplayName());
+        ArrayNode countersNode = groupNode.putArray("COUNTERS");
+        for (Counter counter : counterGroup) {
+          ObjectNode counterNode = countersNode.addObject();
+          counterNode.put("NAME", counter.getName());
+          counterNode.put("DISPLAY_NAME", counter.getDisplayName());
+          counterNode.put("VALUE", counter.getValue());
+        }
+      }
+    }
+    return nodes;
+  }
+
+  public static Set<TimelineMetric> countersToTimelineMetric(Counters counters,
+      long timestamp) {
+    return countersToTimelineMetric(counters, timestamp, "");
+  }
+
+  public static Set<TimelineMetric> countersToTimelineMetric(Counters counters,
+      long timestamp, String groupNamePrefix) {
+    Set<TimelineMetric> entityMetrics = new HashSet<TimelineMetric>();
+    for (CounterGroup g : counters) {
+      String groupName = g.getName();
+      for (Counter c : g) {
+        String name = groupNamePrefix + groupName + ":" + c.getName();
+        TimelineMetric metric = new TimelineMetric();
+        metric.setId(name);
+        metric.addValue(timestamp, c.getValue());
+        entityMetrics.add(metric);
+      }
+    }
+    return entityMetrics;
+  }
+
+}

+ 6 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml

@@ -99,6 +99,12 @@
       <scope>test</scope>
       <type>test-jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
     <dependency>
       <groupId>org.hsqldb</groupId>
       <artifactId>hsqldb</artifactId>

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

@@ -18,6 +18,21 @@
 
 package org.apache.hadoop.mapred;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -26,17 +41,33 @@ import org.apache.hadoop.mapreduce.jobhistory.EventType;
 import org.apache.hadoop.mapreduce.jobhistory.TestJobHistoryEventHandler;
 import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.timeline.TimelineStore;
-
+import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
+import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.Sets;
+
 public class TestMRTimelineEventHandling {
 
+  private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector";
+  private static final Log LOG =
+      LogFactory.getLog(TestMRTimelineEventHandling.class);
+
   @Test
   public void testTimelineServiceStartInMiniCluster() throws Exception {
     Configuration conf = new YarnConfiguration();
@@ -50,7 +81,7 @@ public class TestMRTimelineEventHandling {
     MiniMRYarnCluster cluster = null;
     try {
       cluster = new MiniMRYarnCluster(
-          TestJobHistoryEventHandler.class.getSimpleName(), 1);
+        TestMRTimelineEventHandling.class.getSimpleName(), 1);
       cluster.init(conf);
       cluster.start();
 
@@ -91,7 +122,7 @@ public class TestMRTimelineEventHandling {
     MiniMRYarnCluster cluster = null;
     try {
       cluster = new MiniMRYarnCluster(
-          TestJobHistoryEventHandler.class.getSimpleName(), 1);
+        TestMRTimelineEventHandling.class.getSimpleName(), 1);
       cluster.init(conf);
       cluster.start();
       conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
@@ -140,6 +171,280 @@ public class TestMRTimelineEventHandling {
     }
   }
 
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testMRNewTimelineServiceEventHandling() throws Exception {
+    LOG.info("testMRNewTimelineServiceEventHandling start.");
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    // enable new timeline service
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+    conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
+        FileSystemTimelineWriterImpl.class, TimelineWriter.class);
+    conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
+
+    // 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
+        + ".class", PerNodeTimelineCollectorsAuxService.class.getName());
+
+    conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
+
+    MiniMRYarnCluster cluster = null;
+    try {
+      cluster = new MiniMRYarnCluster(
+          TestMRTimelineEventHandling.class.getSimpleName(), 1, true);
+      cluster.init(conf);
+      cluster.start();
+      LOG.info("A MiniMRYarnCluster get start.");
+
+      Path inDir = new Path("input");
+      Path outDir = new Path("output");
+      LOG.info("Run 1st job which should be successful.");
+      JobConf successConf = new JobConf(conf);
+      successConf.set("dummy_conf1",
+          UtilsForTests.createConfigValue(51 * 1024));
+      successConf.set("dummy_conf2",
+          UtilsForTests.createConfigValue(51 * 1024));
+      successConf.set("huge_dummy_conf1",
+          UtilsForTests.createConfigValue(101 * 1024));
+      successConf.set("huge_dummy_conf2",
+          UtilsForTests.createConfigValue(101 * 1024));
+      RunningJob job =
+          UtilsForTests.runJobSucceed(successConf, inDir, outDir);
+      Assert.assertEquals(JobStatus.SUCCEEDED,
+          job.getJobStatus().getState().getValue());
+
+      YarnClient yarnClient = YarnClient.createYarnClient();
+      yarnClient.init(new Configuration(cluster.getConfig()));
+      yarnClient.start();
+      EnumSet<YarnApplicationState> appStates =
+          EnumSet.allOf(YarnApplicationState.class);
+
+      ApplicationId firstAppId = null;
+      List<ApplicationReport> apps = yarnClient.getApplications(appStates);
+      Assert.assertEquals(apps.size(), 1);
+      ApplicationReport appReport = apps.get(0);
+      firstAppId = appReport.getApplicationId();
+      UtilsForTests.waitForAppFinished(job, cluster);
+      checkNewTimelineEvent(firstAppId, appReport);
+
+      LOG.info("Run 2nd job which should be failed.");
+      job = UtilsForTests.runJobFail(new JobConf(conf), inDir, outDir);
+      Assert.assertEquals(JobStatus.FAILED,
+          job.getJobStatus().getState().getValue());
+
+      apps = yarnClient.getApplications(appStates);
+      Assert.assertEquals(apps.size(), 2);
+
+      appReport = apps.get(0).getApplicationId().equals(firstAppId) ?
+          apps.get(0) : apps.get(1);
+
+      checkNewTimelineEvent(firstAppId, appReport);
+
+    } finally {
+      if (cluster != null) {
+        cluster.stop();
+      }
+      // Cleanup test file
+      String testRoot =
+          FileSystemTimelineWriterImpl.
+              DEFAULT_TIMELINE_SERVICE_STORAGE_DIR_ROOT;
+      File testRootFolder = new File(testRoot);
+      if(testRootFolder.isDirectory()) {
+        FileUtils.deleteDirectory(testRootFolder);
+      }
+
+    }
+  }
+
+  private void checkNewTimelineEvent(ApplicationId appId,
+      ApplicationReport appReport) throws IOException {
+    String tmpRoot =
+        FileSystemTimelineWriterImpl.DEFAULT_TIMELINE_SERVICE_STORAGE_DIR_ROOT
+            + "/entities/";
+
+    File tmpRootFolder = new File(tmpRoot);
+
+    Assert.assertTrue(tmpRootFolder.isDirectory());
+    String basePath = tmpRoot + YarnConfiguration.DEFAULT_RM_CLUSTER_ID +
+        "/" + UserGroupInformation.getCurrentUser().getShortUserName() +
+        "/" + appReport.getName() +
+        "/" + TimelineUtils.DEFAULT_FLOW_VERSION +
+        "/" + appReport.getStartTime() +
+        "/" + appId.toString();
+    // for this test, we expect MAPREDUCE_JOB and MAPREDUCE_TASK dirs
+    String outputDirJob = basePath + "/MAPREDUCE_JOB/";
+
+    File entityFolder = new File(outputDirJob);
+    Assert.assertTrue("Job output directory: " + outputDirJob +
+        " does not exist.",
+        entityFolder.isDirectory());
+
+    // check for job event file
+    String jobEventFileName = appId.toString().replaceAll("application", "job")
+        + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+
+    String jobEventFilePath = outputDirJob + jobEventFileName;
+    File jobEventFile = new File(jobEventFilePath);
+    Assert.assertTrue("jobEventFilePath: " + jobEventFilePath +
+        " does not exist.",
+        jobEventFile.exists());
+    verifyEntity(jobEventFile, EventType.JOB_FINISHED.name(),
+        true, false, null);
+    Set<String> cfgsToCheck = Sets.newHashSet("dummy_conf1", "dummy_conf2",
+        "huge_dummy_conf1", "huge_dummy_conf2");
+    verifyEntity(jobEventFile, null, false, true, cfgsToCheck);
+
+    // for this test, we expect MR job metrics are published in YARN_APPLICATION
+    String outputAppDir = basePath + "/YARN_APPLICATION/";
+    entityFolder = new File(outputAppDir);
+    Assert.assertTrue(
+        "Job output directory: " + outputAppDir +
+        " does not exist.",
+        entityFolder.isDirectory());
+
+    // check for job event file
+    String appEventFileName = appId.toString()
+        + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+
+    String appEventFilePath = outputAppDir + appEventFileName;
+    File appEventFile = new File(appEventFilePath);
+    Assert.assertTrue(
+        "appEventFilePath: " + appEventFilePath +
+        " does not exist.",
+        appEventFile.exists());
+    verifyEntity(appEventFile, null, true, false, null);
+    verifyEntity(appEventFile, null, false, true, cfgsToCheck);
+
+    // check for task event file
+    String outputDirTask = basePath + "/MAPREDUCE_TASK/";
+    File taskFolder = new File(outputDirTask);
+    Assert.assertTrue("Task output directory: " + outputDirTask +
+        " does not exist.",
+        taskFolder.isDirectory());
+
+    String taskEventFileName =
+        appId.toString().replaceAll("application", "task") +
+        "_m_000000" +
+        FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+
+    String taskEventFilePath = outputDirTask + taskEventFileName;
+    File taskEventFile = new File(taskEventFilePath);
+    Assert.assertTrue("taskEventFileName: " + taskEventFilePath +
+        " does not exist.",
+        taskEventFile.exists());
+    verifyEntity(taskEventFile, EventType.TASK_FINISHED.name(),
+        true, false, null);
+
+    // check for task attempt event file
+    String outputDirTaskAttempt = basePath + "/MAPREDUCE_TASK_ATTEMPT/";
+    File taskAttemptFolder = new File(outputDirTaskAttempt);
+    Assert.assertTrue("TaskAttempt output directory: " + outputDirTaskAttempt +
+        " does not exist.", taskAttemptFolder.isDirectory());
+
+    String taskAttemptEventFileName = appId.toString().replaceAll(
+        "application", "attempt") + "_m_000000_0" +
+        FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+
+    String taskAttemptEventFilePath = outputDirTaskAttempt +
+        taskAttemptEventFileName;
+    File taskAttemptEventFile = new File(taskAttemptEventFilePath);
+    Assert.assertTrue("taskAttemptEventFileName: " + taskAttemptEventFilePath +
+        " does not exist.", taskAttemptEventFile.exists());
+    verifyEntity(taskAttemptEventFile, EventType.MAP_ATTEMPT_FINISHED.name(),
+        true, false, null);
+  }
+
+  /**
+   * Verifies entity by reading the entity file written via FS impl.
+   * @param entityFile File to be read.
+   * @param eventId Event to be checked.
+   * @param chkMetrics If event is not null, this flag determines if metrics
+   *     exist when the event is encountered. If event is null, we merely check
+   *     if metrics exist in the entity file.
+   * @param chkCfg If event is not null, this flag determines if configs
+   *     exist when the event is encountered. If event is null, we merely check
+   *     if configs exist in the entity file.
+   * @param cfgsToVerify a set of configs which should exist in the entity file.
+   * @throws IOException
+   */
+  private void verifyEntity(File entityFile, String eventId,
+      boolean chkMetrics, boolean chkCfg, Set<String> cfgsToVerify)
+      throws IOException {
+    BufferedReader reader = null;
+    String strLine;
+    try {
+      reader = new BufferedReader(new FileReader(entityFile));
+      while ((strLine = reader.readLine()) != null) {
+        if (strLine.trim().length() > 0) {
+          org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+              entity =
+                  FileSystemTimelineReaderImpl.getTimelineRecordFromJSON(
+                      strLine.trim(),
+                      org.apache.hadoop.yarn.api.records.timelineservice.
+                          TimelineEntity.class);
+          if (eventId == null) {
+            // Job metrics are published without any events for
+            // ApplicationEntity. There is also possibility that some other
+            // ApplicationEntity is published without events, hence loop till
+            // its found. Same applies to configs.
+            if (chkMetrics && entity.getMetrics().size() > 0) {
+              return;
+            }
+            if (chkCfg && entity.getConfigs().size() > 0) {
+              if (cfgsToVerify == null) {
+                return;
+              } else {
+                // Have configs to verify. Keep on removing configs from the set
+                // of configs to verify as they are found. When the all the
+                // entities have been looped through, we will check if the set
+                // is empty or not(indicating if all configs have been found or
+                // not).
+                for (Iterator<String> itr =
+                    cfgsToVerify.iterator(); itr.hasNext();) {
+                  String config = itr.next();
+                  if (entity.getConfigs().containsKey(config)) {
+                    itr.remove();
+                  }
+                }
+                // All the required configs have been verified, so return.
+                if (cfgsToVerify.isEmpty()) {
+                  return;
+                }
+              }
+            }
+          } else {
+            for (TimelineEvent event : entity.getEvents()) {
+              if (event.getId().equals(eventId)) {
+                if (chkMetrics) {
+                  assertTrue(entity.getMetrics().size() > 0);
+                }
+                if (chkCfg) {
+                  assertTrue(entity.getConfigs().size() > 0);
+                  if (cfgsToVerify != null) {
+                    for (String cfg : cfgsToVerify) {
+                      assertTrue(entity.getConfigs().containsKey(cfg));
+                    }
+                  }
+                }
+                return;
+              }
+            }
+          }
+        }
+      }
+      if (cfgsToVerify != null) {
+        assertTrue(cfgsToVerify.isEmpty());
+        return;
+      }
+      fail("Expected event : " + eventId + " not found in the file "
+          + entityFile);
+    } finally {
+      reader.close();
+    }
+  }
+
   @Test
   public void testMapreduceJobTimelineServiceEnabled()
       throws Exception {
@@ -153,7 +458,7 @@ public class TestMRTimelineEventHandling {
     try {
       fs = FileSystem.get(conf);
       cluster = new MiniMRYarnCluster(
-          TestJobHistoryEventHandler.class.getSimpleName(), 1);
+        TestMRTimelineEventHandling.class.getSimpleName(), 1);
       cluster.init(conf);
       cluster.start();
       conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,

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

@@ -31,8 +31,10 @@ import java.util.Enumeration;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,10 +54,13 @@ import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSeq
 import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
-
-import org.apache.commons.logging.Log;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
+import com.google.common.base.Supplier;
 
 /** 
  * Utilities used in unit test.
@@ -150,6 +155,14 @@ public class UtilsForTests {
     return buf.toString();
   }
 
+  public static String createConfigValue(int msgSize) {
+    StringBuilder sb = new StringBuilder(msgSize);
+    for (int i = 0; i < msgSize; i++) {
+      sb.append('a');
+    }
+    return sb.toString();
+  }
+
   public static String safeGetCanonicalPath(File f) {
     try {
       String s = f.getCanonicalPath();
@@ -607,6 +620,29 @@ public class UtilsForTests {
     return job;
   }
 
+  public static void waitForAppFinished(RunningJob job,
+      MiniMRYarnCluster cluster) throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(
+        Long.parseLong(job.getID().getJtIdentifier()), job.getID().getId());
+    ConcurrentMap<ApplicationId, RMApp> rmApps = cluster.getResourceManager()
+        .getRMContext().getRMApps();
+    if (!rmApps.containsKey(appId)) {
+      throw new IOException("Job not found");
+    }
+    final RMApp rmApp = rmApps.get(appId);
+    try {
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          return RMAppImpl.isAppInFinalState(rmApp);
+        }
+      }, 1000, 1000 * 180);
+    } catch (TimeoutException | InterruptedException e1) {
+      throw new IOException("Yarn application with " + appId + " didn't finish "
+          + "did not reach finale State", e1);
+    }
+  }
+
   // Run a job that will be succeeded and wait until it completes
   public static RunningJob runJobSucceed(JobConf conf, Path inDir, Path outDir)
          throws IOException {

+ 57 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/EntityWriterV2.java

@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager;
+
+/**
+ * Base mapper for writing entities to the timeline service. Subclasses
+ * override {@link #writeEntities(Configuration, TimelineCollectorManager,
+ * org.apache.hadoop.mapreduce.Mapper.Context)} to create and write entities
+ * to the timeline service.
+ */
+abstract class EntityWriterV2
+    extends org.apache.hadoop.mapreduce.Mapper
+        <IntWritable, IntWritable, Writable, Writable> {
+  @Override
+  public void map(IntWritable key, IntWritable val, Context context)
+      throws IOException {
+
+    // create the timeline collector manager wired with the writer
+    Configuration tlConf = new YarnConfiguration();
+    TimelineCollectorManager manager = new TimelineCollectorManager("test");
+    manager.init(tlConf);
+    manager.start();
+    try {
+      // invoke the method to have the subclass write entities
+      writeEntities(tlConf, manager, context);
+    } finally {
+      manager.close();
+    }
+  }
+
+  protected abstract void writeEntities(Configuration tlConf,
+      TimelineCollectorManager manager, Context context) throws IOException;
+}

+ 3 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java

@@ -28,6 +28,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
 
+/**
+ * Used to parse job history and configuration files.
+ */
 class JobHistoryFileParser {
   private static final Log LOG = LogFactory.getLog(JobHistoryFileParser.class);
 

+ 1 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java

@@ -20,33 +20,21 @@ package org.apache.hadoop.mapreduce;
 
 import java.io.IOException;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Mapper.Context;
-import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters;
-import org.apache.hadoop.mapreduce.JobHistoryFileReplayHelper;
 import org.apache.hadoop.mapreduce.JobHistoryFileReplayHelper.JobFiles;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.TypeConverter;
+import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;

+ 161 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java

@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.hadoop.mapreduce.JobHistoryFileReplayHelper.JobFiles;
+import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager;
+
+/**
+ * Mapper for TimelineServicePerformance that replays job history files to the
+ * timeline service v.2.
+ *
+ */
+class JobHistoryFileReplayMapperV2 extends EntityWriterV2 {
+  private static final Log LOG =
+      LogFactory.getLog(JobHistoryFileReplayMapperV2.class);
+
+  @Override
+  protected void writeEntities(Configuration tlConf,
+      TimelineCollectorManager manager, Context context) throws IOException {
+    JobHistoryFileReplayHelper helper = new JobHistoryFileReplayHelper(context);
+    int replayMode = helper.getReplayMode();
+    JobHistoryFileParser parser = helper.getParser();
+    TimelineEntityConverterV2 converter = new TimelineEntityConverterV2();
+
+    // collect the apps it needs to process
+    Collection<JobFiles> jobs = helper.getJobFiles();
+    if (jobs.isEmpty()) {
+      LOG.info(context.getTaskAttemptID().getTaskID() +
+          " will process no jobs");
+    } else {
+      LOG.info(context.getTaskAttemptID().getTaskID() + " will process " +
+          jobs.size() + " jobs");
+    }
+    for (JobFiles job: jobs) {
+      // process each job
+      String jobIdStr = job.getJobId();
+      // skip if either of the file is missing
+      if (job.getJobConfFilePath() == null ||
+          job.getJobHistoryFilePath() == null) {
+        LOG.info(jobIdStr + " missing either the job history file or the " +
+            "configuration file. Skipping.");
+        continue;
+      }
+      LOG.info("processing " + jobIdStr + "...");
+      JobId jobId = TypeConverter.toYarn(JobID.forName(jobIdStr));
+      ApplicationId appId = jobId.getAppId();
+
+      // create the app level timeline collector and start it
+      AppLevelTimelineCollector collector =
+          new AppLevelTimelineCollector(appId);
+      manager.putIfAbsent(appId, collector);
+      try {
+        // parse the job info and configuration
+        JobInfo jobInfo =
+            parser.parseHistoryFile(job.getJobHistoryFilePath());
+        Configuration jobConf =
+            parser.parseConfiguration(job.getJobConfFilePath());
+        LOG.info("parsed the job history file and the configuration file " +
+            "for job " + jobIdStr);
+
+        // set the context
+        // flow id: job name, flow run id: timestamp, user id
+        TimelineCollectorContext tlContext =
+            collector.getTimelineEntityContext();
+        tlContext.setFlowName(jobInfo.getJobname());
+        tlContext.setFlowRunId(jobInfo.getSubmitTime());
+        tlContext.setUserId(jobInfo.getUsername());
+
+        // create entities from job history and write them
+        long totalTime = 0;
+        List<TimelineEntity> entitySet =
+            converter.createTimelineEntities(jobInfo, jobConf);
+        LOG.info("converted them into timeline entities for job " + jobIdStr);
+        // use the current user for this purpose
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        long startWrite = System.nanoTime();
+        try {
+          switch (replayMode) {
+          case JobHistoryFileReplayHelper.WRITE_ALL_AT_ONCE:
+            writeAllEntities(collector, entitySet, ugi);
+            break;
+          case JobHistoryFileReplayHelper.WRITE_PER_ENTITY:
+            writePerEntity(collector, entitySet, ugi);
+            break;
+          default:
+            break;
+          }
+        } catch (Exception e) {
+          context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_FAILURES).
+              increment(1);
+          LOG.error("writing to the timeline service failed", e);
+        }
+        long endWrite = System.nanoTime();
+        totalTime += TimeUnit.NANOSECONDS.toMillis(endWrite-startWrite);
+        int numEntities = entitySet.size();
+        LOG.info("wrote " + numEntities + " entities in " + totalTime + " ms");
+
+        context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_TIME).
+            increment(totalTime);
+        context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER).
+            increment(numEntities);
+      } finally {
+        manager.remove(appId);
+        context.progress(); // move it along
+      }
+    }
+  }
+
+  private void writeAllEntities(AppLevelTimelineCollector collector,
+      List<TimelineEntity> entitySet, UserGroupInformation ugi)
+      throws IOException {
+    TimelineEntities entities = new TimelineEntities();
+    entities.setEntities(entitySet);
+    collector.putEntities(entities, ugi);
+  }
+
+  private void writePerEntity(AppLevelTimelineCollector collector,
+      List<TimelineEntity> entitySet, UserGroupInformation ugi)
+      throws IOException {
+    for (TimelineEntity entity : entitySet) {
+      TimelineEntities entities = new TimelineEntities();
+      entities.addEntity(entity);
+      collector.putEntities(entities, ugi);
+      LOG.info("wrote entity " + entity.getId());
+    }
+  }
+}

+ 43 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterConstants.java

@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce;
+
+/**
+ * Constants for simple entity writers.
+ */
+interface SimpleEntityWriterConstants {
+  // constants for mtype = 1
+  String KBS_SENT = "kbs sent";
+  int KBS_SENT_DEFAULT = 1;
+  String TEST_TIMES = "testtimes";
+  int TEST_TIMES_DEFAULT = 100;
+  String TIMELINE_SERVICE_PERFORMANCE_RUN_ID =
+      "timeline.server.performance.run.id";
+
+  /**
+   *  To ensure that the compression really gets exercised, generate a
+   *  random alphanumeric fixed length payload.
+   */
+  char[] ALPHA_NUMS = new char[] {'a', 'b', 'c', 'd', 'e', 'f',
+      'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r',
+      's', 't', 'u', 'v', 'w', 'x', 'y', 'z', 'A', 'B', 'C', 'D',
+      'E', 'F', 'G', 'H', 'I', 'J', 'K', 'L', 'M', 'N', 'O', 'P',
+      'Q', 'R', 'S', 'T', 'U', 'V', 'W', 'X', 'Y', 'Z', '1', '2',
+      '3', '4', '5', '6', '7', '8', '9', '0', ' '};
+}

+ 6 - 26
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java

@@ -27,45 +27,25 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Mapper.Context;
 import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 /**
    * Adds simple entities with random string payload, events, metrics, and
    * configuration.
    */
-class SimpleEntityWriterV1 extends
-    org.apache.hadoop.mapreduce.Mapper<IntWritable,IntWritable,Writable,Writable> {
+class SimpleEntityWriterV1
+    extends org.apache.hadoop.mapreduce.Mapper
+        <IntWritable, IntWritable, Writable, Writable>
+    implements SimpleEntityWriterConstants {
   private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV1.class);
 
-  // constants for mtype = 1
-  static final String KBS_SENT = "kbs sent";
-  static final int KBS_SENT_DEFAULT = 1;
-  static final String TEST_TIMES = "testtimes";
-  static final int TEST_TIMES_DEFAULT = 100;
-  static final String TIMELINE_SERVICE_PERFORMANCE_RUN_ID =
-      "timeline.server.performance.run.id";
-  /**
-   *  To ensure that the compression really gets exercised, generate a
-   *  random alphanumeric fixed length payload
-   */
-  private static char[] ALPHA_NUMS = new char[] { 'a', 'b', 'c', 'd', 'e', 'f',
-    'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r',
-    's', 't', 'u', 'v', 'w', 'x', 'y', 'z', 'A', 'B', 'C', 'D',
-    'E', 'F', 'G', 'H', 'I', 'J', 'K', 'L', 'M', 'N', 'O', 'P',
-    'Q', 'R', 'S', 'T', 'U', 'V', 'W', 'X', 'Y', 'Z', '1', '2',
-    '3', '4', '5', '6', '7', '8', '9', '0', ' ' };
-
-  public void map(IntWritable key, IntWritable val, Context context) throws IOException {
+  public void map(IntWritable key, IntWritable val, Context context)
+      throws IOException {
     TimelineClient tlc = new TimelineClientImpl();
     Configuration conf = context.getConfiguration();
 

+ 131 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java

@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager;
+
+/**
+ * Adds simple entities with random string payload, events, metrics, and
+ * configuration.
+ */
+class SimpleEntityWriterV2 extends EntityWriterV2
+    implements SimpleEntityWriterConstants {
+  private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV2.class);
+
+  protected void writeEntities(Configuration tlConf,
+      TimelineCollectorManager manager, Context context) throws IOException {
+    Configuration conf = context.getConfiguration();
+    // simulate the app id with the task id
+    int taskId = context.getTaskAttemptID().getTaskID().getId();
+    long timestamp = conf.getLong(TIMELINE_SERVICE_PERFORMANCE_RUN_ID, 0);
+    ApplicationId appId = ApplicationId.newInstance(timestamp, taskId);
+
+    // create the app level timeline collector
+    AppLevelTimelineCollector collector =
+        new AppLevelTimelineCollector(appId);
+    manager.putIfAbsent(appId, collector);
+
+    try {
+      // set the context
+      // flow id: job name, flow run id: timestamp, user id
+      TimelineCollectorContext tlContext =
+          collector.getTimelineEntityContext();
+      tlContext.setFlowName(context.getJobName());
+      tlContext.setFlowRunId(timestamp);
+      tlContext.setUserId(context.getUser());
+
+      final int kbs = conf.getInt(KBS_SENT, KBS_SENT_DEFAULT);
+
+      long totalTime = 0;
+      final int testtimes = conf.getInt(TEST_TIMES, TEST_TIMES_DEFAULT);
+      final Random rand = new Random();
+      final TaskAttemptID taskAttemptId = context.getTaskAttemptID();
+      final char[] payLoad = new char[kbs * 1024];
+
+      for (int i = 0; i < testtimes; i++) {
+        // Generate a fixed length random payload
+        for (int xx = 0; xx < kbs * 1024; xx++) {
+          int alphaNumIdx =
+              rand.nextInt(ALPHA_NUMS.length);
+          payLoad[xx] = ALPHA_NUMS[alphaNumIdx];
+        }
+        String entId = taskAttemptId + "_" + Integer.toString(i);
+        final TimelineEntity entity = new TimelineEntity();
+        entity.setId(entId);
+        entity.setType("FOO_ATTEMPT");
+        entity.addInfo("PERF_TEST", payLoad);
+        // add an event
+        TimelineEvent event = new TimelineEvent();
+        event.setId("foo_event_id");
+        event.setTimestamp(System.currentTimeMillis());
+        event.addInfo("foo_event", "test");
+        entity.addEvent(event);
+        // add a metric
+        TimelineMetric metric = new TimelineMetric();
+        metric.setId("foo_metric");
+        metric.addValue(System.currentTimeMillis(), 123456789L);
+        entity.addMetric(metric);
+        // add a config
+        entity.addConfig("foo", "bar");
+
+        TimelineEntities entities = new TimelineEntities();
+        entities.addEntity(entity);
+        // use the current user for this purpose
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        long startWrite = System.nanoTime();
+        try {
+          collector.putEntities(entities, ugi);
+        } catch (Exception e) {
+          context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_FAILURES).
+              increment(1);
+          LOG.error("writing to the timeline service failed", e);
+        }
+        long endWrite = System.nanoTime();
+        totalTime += TimeUnit.NANOSECONDS.toMillis(endWrite-startWrite);
+      }
+      LOG.info("wrote " + testtimes + " entities (" + kbs*testtimes +
+          " kB) in " + totalTime + " ms");
+      context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_TIME).
+          increment(totalTime);
+      context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER).
+          increment(testtimes);
+      context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_KBS).
+          increment(kbs*testtimes);
+    } finally {
+      // clean up
+      manager.remove(appId);
+    }
+  }
+}

+ 6 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java

@@ -25,11 +25,6 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.CounterGroup;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
@@ -95,9 +90,10 @@ class TimelineEntityConverterV1 {
     return job;
   }
 
-  private Set<TimelineEntity> createTaskAndTaskAttemptEntities(JobInfo jobInfo) {
+  private Set<TimelineEntity>
+      createTaskAndTaskAttemptEntities(JobInfo jobInfo) {
     Set<TimelineEntity> entities = new HashSet<>();
-    Map<TaskID,TaskInfo> taskInfoMap = jobInfo.getAllTasks();
+    Map<TaskID, TaskInfo> taskInfoMap = jobInfo.getAllTasks();
     LOG.info("job " + jobInfo.getJobId()+ " has " + taskInfoMap.size() +
         " tasks");
     for (TaskInfo taskInfo: taskInfoMap.values()) {
@@ -129,7 +125,7 @@ class TimelineEntityConverterV1 {
 
   private Set<TimelineEntity> createTaskAttemptEntities(TaskInfo taskInfo) {
     Set<TimelineEntity> taskAttempts = new HashSet<TimelineEntity>();
-    Map<TaskAttemptID,TaskAttemptInfo> taskAttemptInfoMap =
+    Map<TaskAttemptID, TaskAttemptInfo> taskAttemptInfoMap =
         taskInfo.getAllTaskAttempts();
     LOG.info("task " + taskInfo.getTaskId() + " has " +
         taskAttemptInfoMap.size() + " task attempts");
@@ -140,7 +136,8 @@ class TimelineEntityConverterV1 {
     return taskAttempts;
   }
 
-  private TimelineEntity createTaskAttemptEntity(TaskAttemptInfo taskAttemptInfo) {
+  private TimelineEntity
+      createTaskAttemptEntity(TaskAttemptInfo taskAttemptInfo) {
     TimelineEntity taskAttempt = new TimelineEntity();
     taskAttempt.setEntityType(TASK_ATTEMPT);
     taskAttempt.setEntityId(taskAttemptInfo.getAttemptId().toString());

+ 206 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java

@@ -0,0 +1,206 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+
+class TimelineEntityConverterV2 {
+  private static final Log LOG =
+      LogFactory.getLog(TimelineEntityConverterV2.class);
+
+  static final String JOB = "MAPREDUCE_JOB";
+  static final String TASK = "MAPREDUCE_TASK";
+  static final String TASK_ATTEMPT = "MAPREDUCE_TASK_ATTEMPT";
+
+  /**
+   * Creates job, task, and task attempt entities based on the job history info
+   * and configuration.
+   *
+   * Note: currently these are plan timeline entities created for mapreduce
+   * types. These are not meant to be the complete and accurate entity set-up
+   * for mapreduce jobs. We do not leverage hierarchical timeline entities. If
+   * we create canonical mapreduce hierarchical timeline entities with proper
+   * parent-child relationship, we could modify this to use that instead.
+   *
+   * Note that we also do not add info to the YARN application entity, which
+   * would be needed for aggregation.
+   */
+  public List<TimelineEntity> createTimelineEntities(JobInfo jobInfo,
+      Configuration conf) {
+    List<TimelineEntity> entities = new ArrayList<>();
+
+    // create the job entity
+    TimelineEntity job = createJobEntity(jobInfo, conf);
+    entities.add(job);
+
+    // create the task and task attempt entities
+    List<TimelineEntity> tasksAndAttempts =
+        createTaskAndTaskAttemptEntities(jobInfo);
+    entities.addAll(tasksAndAttempts);
+
+    return entities;
+  }
+
+  private TimelineEntity createJobEntity(JobInfo jobInfo, Configuration conf) {
+    TimelineEntity job = new TimelineEntity();
+    job.setType(JOB);
+    job.setId(jobInfo.getJobId().toString());
+    job.setCreatedTime(jobInfo.getSubmitTime());
+
+    job.addInfo("JOBNAME", jobInfo.getJobname());
+    job.addInfo("USERNAME", jobInfo.getUsername());
+    job.addInfo("JOB_QUEUE_NAME", jobInfo.getJobQueueName());
+    job.addInfo("SUBMIT_TIME", jobInfo.getSubmitTime());
+    job.addInfo("LAUNCH_TIME", jobInfo.getLaunchTime());
+    job.addInfo("FINISH_TIME", jobInfo.getFinishTime());
+    job.addInfo("JOB_STATUS", jobInfo.getJobStatus());
+    job.addInfo("PRIORITY", jobInfo.getPriority());
+    job.addInfo("TOTAL_MAPS", jobInfo.getTotalMaps());
+    job.addInfo("TOTAL_REDUCES", jobInfo.getTotalReduces());
+    job.addInfo("UBERIZED", jobInfo.getUberized());
+    job.addInfo("ERROR_INFO", jobInfo.getErrorInfo());
+
+    // add metrics from total counters
+    // we omit the map counters and reduce counters for now as it's kind of
+    // awkward to put them (map/reduce/total counters are really a group of
+    // related counters)
+    Counters totalCounters = jobInfo.getTotalCounters();
+    if (totalCounters != null) {
+      addMetrics(job, totalCounters);
+    }
+    // finally add configuration to the job
+    addConfiguration(job, conf);
+    LOG.info("converted job " + jobInfo.getJobId() + " to a timeline entity");
+    return job;
+  }
+
+  private void addConfiguration(TimelineEntity job, Configuration conf) {
+    for (Map.Entry<String, String> e: conf) {
+      job.addConfig(e.getKey(), e.getValue());
+    }
+  }
+
+  private void addMetrics(TimelineEntity entity, Counters counters) {
+    for (CounterGroup g: counters) {
+      String groupName = g.getName();
+      for (Counter c: g) {
+        String name = groupName + ":" + c.getName();
+        TimelineMetric metric = new TimelineMetric();
+        metric.setId(name);
+        metric.addValue(System.currentTimeMillis(), c.getValue());
+        entity.addMetric(metric);
+      }
+    }
+  }
+
+  private List<TimelineEntity> createTaskAndTaskAttemptEntities(
+      JobInfo jobInfo) {
+    List<TimelineEntity> entities = new ArrayList<>();
+    Map<TaskID, TaskInfo> taskInfoMap = jobInfo.getAllTasks();
+    LOG.info("job " + jobInfo.getJobId()+ " has " + taskInfoMap.size() +
+        " tasks");
+    for (TaskInfo taskInfo: taskInfoMap.values()) {
+      TimelineEntity task = createTaskEntity(taskInfo);
+      entities.add(task);
+      // add the task attempts from this task
+      Set<TimelineEntity> taskAttempts = createTaskAttemptEntities(taskInfo);
+      entities.addAll(taskAttempts);
+    }
+    return entities;
+  }
+
+  private TimelineEntity createTaskEntity(TaskInfo taskInfo) {
+    TimelineEntity task = new TimelineEntity();
+    task.setType(TASK);
+    task.setId(taskInfo.getTaskId().toString());
+    task.setCreatedTime(taskInfo.getStartTime());
+
+    task.addInfo("START_TIME", taskInfo.getStartTime());
+    task.addInfo("FINISH_TIME", taskInfo.getFinishTime());
+    task.addInfo("TASK_TYPE", taskInfo.getTaskType());
+    task.addInfo("TASK_STATUS", taskInfo.getTaskStatus());
+    task.addInfo("ERROR_INFO", taskInfo.getError());
+
+    // add metrics from counters
+    Counters counters = taskInfo.getCounters();
+    if (counters != null) {
+      addMetrics(task, counters);
+    }
+    LOG.info("converted task " + taskInfo.getTaskId() +
+        " to a timeline entity");
+    return task;
+  }
+
+  private Set<TimelineEntity> createTaskAttemptEntities(TaskInfo taskInfo) {
+    Set<TimelineEntity> taskAttempts = new HashSet<TimelineEntity>();
+    Map<TaskAttemptID, TaskAttemptInfo> taskAttemptInfoMap =
+        taskInfo.getAllTaskAttempts();
+    LOG.info("task " + taskInfo.getTaskId() + " has " +
+        taskAttemptInfoMap.size() + " task attempts");
+    for (TaskAttemptInfo taskAttemptInfo: taskAttemptInfoMap.values()) {
+      TimelineEntity taskAttempt = createTaskAttemptEntity(taskAttemptInfo);
+      taskAttempts.add(taskAttempt);
+    }
+    return taskAttempts;
+  }
+
+  private TimelineEntity createTaskAttemptEntity(
+      TaskAttemptInfo taskAttemptInfo) {
+    TimelineEntity taskAttempt = new TimelineEntity();
+    taskAttempt.setType(TASK_ATTEMPT);
+    taskAttempt.setId(taskAttemptInfo.getAttemptId().toString());
+    taskAttempt.setCreatedTime(taskAttemptInfo.getStartTime());
+
+    taskAttempt.addInfo("START_TIME", taskAttemptInfo.getStartTime());
+    taskAttempt.addInfo("FINISH_TIME", taskAttemptInfo.getFinishTime());
+    taskAttempt.addInfo("MAP_FINISH_TIME",
+        taskAttemptInfo.getMapFinishTime());
+    taskAttempt.addInfo("SHUFFLE_FINISH_TIME",
+        taskAttemptInfo.getShuffleFinishTime());
+    taskAttempt.addInfo("SORT_FINISH_TIME",
+        taskAttemptInfo.getSortFinishTime());
+    taskAttempt.addInfo("TASK_STATUS", taskAttemptInfo.getTaskStatus());
+    taskAttempt.addInfo("STATE", taskAttemptInfo.getState());
+    taskAttempt.addInfo("ERROR", taskAttemptInfo.getError());
+    taskAttempt.addInfo("CONTAINER_ID",
+        taskAttemptInfo.getContainerId().toString());
+
+    // add metrics from counters
+    Counters counters = taskAttemptInfo.getCounters();
+    if (counters != null) {
+      addMetrics(taskAttempt, counters);
+    }
+    LOG.info("converted task attempt " + taskAttemptInfo.getAttemptId() +
+        " to a timeline entity");
+    return taskAttempt;
+  }
+}

+ 83 - 47
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineServicePerformance.java

@@ -23,8 +23,6 @@ import java.util.Date;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.SleepJob.SleepInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.util.GenericOptionsParser;
@@ -46,15 +44,19 @@ public class TimelineServicePerformance extends Configured implements Tool {
     System.err.println(
         "Usage: [-m <maps>] number of mappers (default: " + NUM_MAPS_DEFAULT +
             ")\n" +
-        "     [-v] timeline service version\n" +
-        "     [-mtype <mapper type in integer>]\n" +
-        "          1. simple entity write mapper (default)\n" +
+        "     [-v] timeline service version (default: " +
+            TIMELINE_SERVICE_VERSION_1 + ")\n" +
+        "          1. version 1.x\n" +
+        "          2. version 2.x\n" +
+        "     [-mtype <mapper type in integer>] (default: " +
+            SIMPLE_ENTITY_WRITER + ")\n" +
+        "          1. simple entity write mapper\n" +
         "          2. jobhistory files replay mapper\n" +
         "     [-s <(KBs)test>] number of KB per put (mtype=1, default: " +
-             SimpleEntityWriterV1.KBS_SENT_DEFAULT + " KB)\n" +
+             SimpleEntityWriterConstants.KBS_SENT_DEFAULT + " KB)\n" +
         "     [-t] package sending iterations per mapper (mtype=1, default: " +
-             SimpleEntityWriterV1.TEST_TIMES_DEFAULT + ")\n" +
-        "     [-d <path>] root path of job history files (mtype=2)\n" +
+             SimpleEntityWriterConstants.TEST_TIMES_DEFAULT + ")\n" +
+        "     [-d <path>] hdfs root path of job history files (mtype=2)\n" +
         "     [-r <replay mode>] (mtype=2)\n" +
         "          1. write all entities for a job in one put (default)\n" +
         "          2. write one entity at a time\n");
@@ -78,8 +80,7 @@ public class TimelineServicePerformance extends Configured implements Tool {
       try {
         if ("-v".equals(args[i])) {
           timeline_service_version = Integer.parseInt(args[++i]);
-        }
-        if ("-m".equals(args[i])) {
+        } else if ("-m".equals(args[i])) {
           if (Integer.parseInt(args[++i]) > 0) {
             job.getConfiguration()
                 .setInt(MRJobConfig.NUM_MAPS, Integer.parseInt(args[i]));
@@ -88,11 +89,12 @@ public class TimelineServicePerformance extends Configured implements Tool {
           mapperType = Integer.parseInt(args[++i]);
         } else if ("-s".equals(args[i])) {
           if (Integer.parseInt(args[++i]) > 0) {
-            conf.setInt(SimpleEntityWriterV1.KBS_SENT, Integer.parseInt(args[i]));
+            conf.setInt(SimpleEntityWriterConstants.KBS_SENT,
+                Integer.parseInt(args[i]));
           }
         } else if ("-t".equals(args[i])) {
           if (Integer.parseInt(args[++i]) > 0) {
-            conf.setInt(SimpleEntityWriterV1.TEST_TIMES,
+            conf.setInt(SimpleEntityWriterConstants.TEST_TIMES,
                 Integer.parseInt(args[i]));
           }
         } else if ("-d".equals(args[i])) {
@@ -113,28 +115,41 @@ public class TimelineServicePerformance extends Configured implements Tool {
     }
 
     // handle mapper-specific settings
-    switch (timeline_service_version) {
-    case TIMELINE_SERVICE_VERSION_1:
-    default:
-      switch (mapperType) {
-      case JOB_HISTORY_FILE_REPLAY_MAPPER:
+    switch (mapperType) {
+    case JOB_HISTORY_FILE_REPLAY_MAPPER:
+      String processingPath =
+          conf.get(JobHistoryFileReplayHelper.PROCESSING_PATH);
+      if (processingPath == null || processingPath.isEmpty()) {
+        System.out.println("processing path is missing while mtype = 2");
+        return printUsage() == 0;
+      }
+      switch (timeline_service_version) {
+      case TIMELINE_SERVICE_VERSION_2:
+        job.setMapperClass(JobHistoryFileReplayMapperV2.class);
+        break;
+      case TIMELINE_SERVICE_VERSION_1:
+      default:
         job.setMapperClass(JobHistoryFileReplayMapperV1.class);
-        String processingPath =
-            conf.get(JobHistoryFileReplayHelper.PROCESSING_PATH);
-        if (processingPath == null || processingPath.isEmpty()) {
-          System.out.println("processing path is missing while mtype = 2");
-          return printUsage() == 0;
-        }
         break;
-      case SIMPLE_ENTITY_WRITER:
+      }
+      break;
+    case SIMPLE_ENTITY_WRITER:
+    default:
+      // use the current timestamp as the "run id" of the test: this will
+      // be used as simulating the cluster timestamp for apps
+      conf.setLong(
+          SimpleEntityWriterConstants.TIMELINE_SERVICE_PERFORMANCE_RUN_ID,
+          System.currentTimeMillis());
+      switch (timeline_service_version) {
+      case TIMELINE_SERVICE_VERSION_2:
+        job.setMapperClass(SimpleEntityWriterV2.class);
+        break;
+      case TIMELINE_SERVICE_VERSION_1:
       default:
         job.setMapperClass(SimpleEntityWriterV1.class);
-        // use the current timestamp as the "run id" of the test: this will
-        // be used as simulating the cluster timestamp for apps
-        conf.setLong(SimpleEntityWriterV1.TIMELINE_SERVICE_PERFORMANCE_RUN_ID,
-          System.currentTimeMillis());
         break;
       }
+      break;
     }
     return true;
   }
@@ -164,25 +179,46 @@ public class TimelineServicePerformance extends Configured implements Tool {
     Date startTime = new Date();
     System.out.println("Job started: " + startTime);
     int ret = job.waitForCompletion(true) ? 0 : 1;
-    org.apache.hadoop.mapreduce.Counters counters = job.getCounters();
-    long writetime =
-        counters.findCounter(PerfCounters.TIMELINE_SERVICE_WRITE_TIME).getValue();
-    long writecounts =
-        counters.findCounter(PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER).getValue();
-    long writesize =
-        counters.findCounter(PerfCounters.TIMELINE_SERVICE_WRITE_KBS).getValue();
-    double transacrate = writecounts * 1000 / (double)writetime;
-    double iorate = writesize * 1000 / (double)writetime;
-    int numMaps =
-        Integer.parseInt(job.getConfiguration().get(MRJobConfig.NUM_MAPS));
-
-    System.out.println("TRANSACTION RATE (per mapper): " + transacrate +
-        " ops/s");
-    System.out.println("IO RATE (per mapper): " + iorate + " KB/s");
-
-    System.out.println("TRANSACTION RATE (total): " + transacrate*numMaps +
-        " ops/s");
-    System.out.println("IO RATE (total): " + iorate*numMaps + " KB/s");
+    if (job.isSuccessful()) {
+      org.apache.hadoop.mapreduce.Counters counters = job.getCounters();
+      long writecounts =
+          counters.findCounter(
+              PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER).getValue();
+      long writefailures =
+          counters.findCounter(
+              PerfCounters.TIMELINE_SERVICE_WRITE_FAILURES).getValue();
+      if (writefailures > 0 && writefailures == writecounts) {
+        // see if we have a complete failure to write
+        System.out.println("Job failed: all writes failed!");
+      } else {
+        long writetime =
+            counters.findCounter(
+                PerfCounters.TIMELINE_SERVICE_WRITE_TIME).getValue();
+        long writesize =
+            counters.findCounter(
+                PerfCounters.TIMELINE_SERVICE_WRITE_KBS).getValue();
+        if (writetime == 0L) {
+          // see if write time is zero (normally shouldn't happen)
+          System.out.println("Job failed: write time is 0!");
+        } else {
+          double transacrate = writecounts * 1000 / (double)writetime;
+          double iorate = writesize * 1000 / (double)writetime;
+          int numMaps =
+              Integer.parseInt(
+                  job.getConfiguration().get(MRJobConfig.NUM_MAPS));
+
+          System.out.println("TRANSACTION RATE (per mapper): " + transacrate +
+              " ops/s");
+          System.out.println("IO RATE (per mapper): " + iorate + " KB/s");
+
+          System.out.println("TRANSACTION RATE (total): " +
+              transacrate*numMaps + " ops/s");
+          System.out.println("IO RATE (total): " + iorate*numMaps + " KB/s");
+        }
+      }
+    } else {
+      System.out.println("Job failed: " + job.getStatus().getFailureInfo());
+    }
 
     return ret;
   }

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

@@ -68,6 +68,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
   private static final Log LOG = LogFactory.getLog(MiniMRYarnCluster.class);
   private JobHistoryServer historyServer;
   private JobHistoryServerWrapper historyServerWrapper;
+  private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector";
 
   public MiniMRYarnCluster(String testName) {
     this(testName, 1);
@@ -169,8 +170,25 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
     conf.set(MRConfig.MASTER_ADDRESS, "test"); // The default is local because of
                                              // which shuffle doesn't happen
     //configure the shuffle service in NM
-    conf.setStrings(YarnConfiguration.NM_AUX_SERVICES,
-        new String[] { ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID });
+    String[] nmAuxServices = conf.getStrings(YarnConfiguration.NM_AUX_SERVICES);
+    // if need to enable TIMELINE_AUX_SERVICE_NAME
+    boolean enableTimelineAuxService = false;
+    if (nmAuxServices != null) {
+      for (String nmAuxService: nmAuxServices) {
+        if (nmAuxService.equals(TIMELINE_AUX_SERVICE_NAME)) {
+          enableTimelineAuxService = true;
+          break;
+        }
+      }
+    }
+    if (enableTimelineAuxService) {
+      conf.setStrings(YarnConfiguration.NM_AUX_SERVICES,
+          new String[] {ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID,
+              TIMELINE_AUX_SERVICE_NAME});
+    } else {
+      conf.setStrings(YarnConfiguration.NM_AUX_SERVICES,
+          new String[] {ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID});
+    }
     conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT,
         ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID), ShuffleHandler.class,
         Service.class);

+ 16 - 16
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java

@@ -18,6 +18,19 @@
 
 package org.apache.hadoop.test;
 
+import org.apache.hadoop.fs.DFSCIOTest;
+import org.apache.hadoop.fs.DistributedFSCheck;
+import org.apache.hadoop.fs.JHLogAnalyzer;
+import org.apache.hadoop.fs.TestDFSIO;
+import org.apache.hadoop.fs.TestFileSystem;
+import org.apache.hadoop.fs.loadGenerator.DataGenerator;
+import org.apache.hadoop.fs.loadGenerator.LoadGenerator;
+import org.apache.hadoop.fs.loadGenerator.LoadGeneratorMR;
+import org.apache.hadoop.fs.loadGenerator.StructureGenerator;
+import org.apache.hadoop.fs.slive.SliveTest;
+import org.apache.hadoop.hdfs.NNBench;
+import org.apache.hadoop.hdfs.NNBenchWithoutMR;
+import org.apache.hadoop.io.FileBench;
 import org.apache.hadoop.io.TestSequenceFile;
 import org.apache.hadoop.mapred.BigMapOutput;
 import org.apache.hadoop.mapred.GenericMRLoadGenerator;
@@ -28,28 +41,14 @@ import org.apache.hadoop.mapred.TestMapRed;
 import org.apache.hadoop.mapred.TestSequenceFileInputFormat;
 import org.apache.hadoop.mapred.TestTextInputFormat;
 import org.apache.hadoop.mapred.ThreadedMapBenchmark;
-import org.apache.hadoop.mapreduce.TimelineServicePerformance;
 import org.apache.hadoop.mapreduce.FailJob;
 import org.apache.hadoop.mapreduce.GrowingSleepJob;
 import org.apache.hadoop.mapreduce.LargeSorter;
 import org.apache.hadoop.mapreduce.MiniHadoopClusterManager;
 import org.apache.hadoop.mapreduce.SleepJob;
+import org.apache.hadoop.mapreduce.TimelineServicePerformance;
 import org.apache.hadoop.util.ProgramDriver;
 
-import org.apache.hadoop.hdfs.NNBench;
-import org.apache.hadoop.hdfs.NNBenchWithoutMR;
-import org.apache.hadoop.fs.TestFileSystem;
-import org.apache.hadoop.fs.TestDFSIO;
-import org.apache.hadoop.fs.DFSCIOTest;
-import org.apache.hadoop.fs.DistributedFSCheck;
-import org.apache.hadoop.io.FileBench;
-import org.apache.hadoop.fs.JHLogAnalyzer;
-import org.apache.hadoop.fs.loadGenerator.DataGenerator;
-import org.apache.hadoop.fs.loadGenerator.LoadGenerator;
-import org.apache.hadoop.fs.loadGenerator.LoadGeneratorMR;
-import org.apache.hadoop.fs.loadGenerator.StructureGenerator;
-import org.apache.hadoop.fs.slive.SliveTest;
-
 /**
  * Driver for Map-reduce tests.
  *
@@ -95,7 +94,8 @@ public class MapredTestDriver {
       pgd.addClass("gsleep", GrowingSleepJob.class,
           "A sleep job whose mappers create 1MB buffer for every record.");
       pgd.addClass("timelineperformance", TimelineServicePerformance.class,
-                   "A job that launches mappers to test timlineserver performance.");
+                   "A job that launches mappers to test timline service " +
+                   "performance.");
       pgd.addClass("nnbench", NNBench.class,
           "A benchmark that stresses the namenode w/ MR.");
       pgd.addClass("nnbenchWithoutMR", NNBenchWithoutMR.class,

+ 127 - 0
hadoop-project/pom.xml

@@ -48,6 +48,12 @@
     <!-- Version number for xerces used by JDiff -->
     <xerces.jdiff.version>2.11.0</xerces.jdiff.version>
 
+    <kafka.version>0.8.2.1</kafka.version>
+
+    <hbase.version>1.1.3</hbase.version>
+    <phoenix.version>4.7.0-HBase-1.1</phoenix.version>
+    <hbase-compatible-hadoop.version>2.5.1</hbase-compatible-hadoop.version>
+
     <hadoop.assemblies.version>${project.version}</hadoop.assemblies.version>
 
     <commons-daemon.version>1.0.13</commons-daemon.version>
@@ -319,6 +325,25 @@
         <type>test-jar</type>
       </dependency>
 
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+       <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+          <version>${project.version}</version>
+          <type>test-jar</type>
+      </dependency>
+
      <dependency>
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-yarn-applications-distributedshell</artifactId>
@@ -489,6 +514,11 @@
         <artifactId>commons-compress</artifactId>
         <version>1.4.1</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.commons</groupId>
+        <artifactId>commons-csv</artifactId>
+        <version>1.0</version>
+      </dependency>
       <dependency>
         <groupId>xmlenc</groupId>
         <artifactId>xmlenc</artifactId>
@@ -1110,6 +1140,103 @@
           <artifactId>mssql-jdbc</artifactId>
           <version>${mssql.version}</version>
         </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-common</artifactId>
+        <version>${hbase.version}</version>
+        <exclusions>
+          <exclusion>
+            <artifactId>jdk.tools</artifactId>
+           <groupId>jdk.tools</groupId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-common</artifactId>
+        <version>${hbase.version}</version>
+        <scope>test</scope>
+        <classifier>tests</classifier>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-client</artifactId>
+        <version>${hbase.version}</version>
+        <exclusions>
+          <!-- exclude jdk.tools (1.7) as we're not managing it -->
+          <exclusion>
+            <groupId>jdk.tools</groupId>
+            <artifactId>jdk.tools</artifactId>
+         </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-server</artifactId>
+        <version>${hbase.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-server</artifactId>
+        <version>${hbase.version}</version>
+        <scope>test</scope>
+        <classifier>tests</classifier>
+     </dependency>
+      <dependency>
+        <groupId>org.apache.phoenix</groupId>
+        <artifactId>phoenix-core</artifactId>
+        <version>${phoenix.version}</version>
+        <exclusions>
+          <!-- Exclude jline from here -->
+          <exclusion>
+            <artifactId>jline</artifactId>
+            <groupId>jline</groupId>
+          </exclusion>
+         <exclusion>
+            <artifactId>joda-time</artifactId>
+            <groupId>joda-time</groupId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.phoenix</groupId>
+        <artifactId>phoenix-core</artifactId>
+        <type>test-jar</type>
+        <version>${phoenix.version}</version>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-it</artifactId>
+        <version>${hbase.version}</version>
+        <scope>test</scope>
+        <classifier>tests</classifier>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-testing-util</artifactId>
+        <version>${hbase.version}</version>
+        <scope>test</scope>
+        <optional>true</optional>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jruby</groupId>
+            <artifactId>jruby-complete</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
+          </exclusion>
+          <exclusion>
+            <artifactId>jdk.tools</artifactId>
+            <groupId>jdk.tools</groupId>
+          </exclusion>
+        </exclusions>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 

+ 2 - 0
hadoop-project/src/site/site.xml

@@ -127,6 +127,7 @@
       <item name="Node Labels" href="hadoop-yarn/hadoop-yarn-site/NodeLabel.html"/>
       <item name="Web Application Proxy" href="hadoop-yarn/hadoop-yarn-site/WebApplicationProxy.html"/>
       <item name="Timeline Server" href="hadoop-yarn/hadoop-yarn-site/TimelineServer.html"/>
+      <item name="Timeline Service V.2" href="hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html"/>
       <item name="Writing YARN Applications" href="hadoop-yarn/hadoop-yarn-site/WritingYarnApplications.html"/>
       <item name="YARN Application Security" href="hadoop-yarn/hadoop-yarn-site/YarnApplicationSecurity.html"/>
       <item name="NodeManager" href="hadoop-yarn/hadoop-yarn-site/NodeManager.html"/>
@@ -147,6 +148,7 @@
       <item name="Resource Manager" href="hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html"/>
       <item name="Node Manager" href="hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html"/>
       <item name="Timeline Server" href="hadoop-yarn/hadoop-yarn-site/TimelineServer.html#Timeline_Server_REST_API_v1"/>
+      <item name="Timeline Service V.2" href="hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html#Timeline_Service_REST_API_v2"/>
     </menu>
     
     <menu name="Hadoop Compatible File Systems" inherit="top">

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/bin/yarn

@@ -69,6 +69,7 @@ function print_usage(){
   echo "                                        Use -remove-application-from-state-store <appId> for "
   echo "                                            removing application from RMStateStore."
   echo "  nodemanager                           run a nodemanager on each slave"
+  echo "  timelinereader                        run the timeline reader server"
   echo "  timelineserver                        run the timeline server"
   echo "  rmadmin                               admin tools"
   echo "  router                                run the Router daemon"
@@ -260,6 +261,8 @@ elif [ "$COMMAND" = "historyserver" ] ; then
   if [ "$YARN_HISTORYSERVER_HEAPSIZE" != "" ]; then
     JAVA_HEAP_MAX="-Xmx""$YARN_HISTORYSERVER_HEAPSIZE""m"
   fi
+elif [ "$COMMAND" = "timelinereader" ] ; then
+  CLASS='org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer'
 elif [ "$COMMAND" = "timelineserver" ] ; then
   CLASSPATH=${CLASSPATH}:$YARN_CONF_DIR/timelineserver-config/log4j.properties
   CLASS='org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer'

+ 8 - 1
hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd

@@ -155,7 +155,7 @@ if "%1" == "--loglevel" (
 
   set yarncommands=resourcemanager nodemanager proxyserver rmadmin version jar ^
      application applicationattempt cluster container node queue logs daemonlog historyserver ^
-     timelineserver router classpath
+     timelineserver router timelinereader classpath
   for %%i in ( %yarncommands% ) do (
     if %yarn-command% == %%i set yarncommand=true
   )
@@ -246,10 +246,16 @@ goto :eof
   )
   goto :eof
 
+<<<<<<< HEAD
 :router
   set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\router-config\log4j.properties
   set CLASS=org.apache.hadoop.yarn.server.router.Router
   set YARN_OPTS=%YARN_OPTS% %HADOOP_ROUTER_OPTS%
+=======
+:timelinereader
+  set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\timelineserver-config\log4j.properties
+  set CLASS=org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer
+>>>>>>> YARN-5355. Backported YARN-2928 into our branch-2 feature branch.
   goto :eof
 
 :nodemanager
@@ -328,6 +334,7 @@ goto :eof
   @echo   nodemanager          run a nodemanager on each slave
   @echo   router               run the Router daemon
   @echo   timelineserver       run the timeline server
+  @echo   timelinereader       run the timeline reader server
   @echo   rmadmin              admin tools
   @echo   version              print the version
   @echo   jar ^<jar^>          run a jar file

+ 27 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml

@@ -118,6 +118,23 @@
     <Bug pattern="BC_UNCONFIRMED_CAST" />
   </Match>
 
+  <!-- Object cast is based on the event type -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher" />
+    <Method name="publishApplicationEvent" />
+    <Bug pattern="BC_UNCONFIRMED_CAST" />
+  </Match>
+
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher" />
+    <Method name="publishLocalizationEvent" />
+    <Bug pattern="BC_UNCONFIRMED_CAST" />
+  </Match>
+
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Publisher$TimelineV2EventHandler" />
+     <Bug pattern="BC_UNCONFIRMED_CAST" />
+  </Match>
 
   <!-- Ignore intentional switch fallthroughs -->
   <Match>
@@ -517,6 +534,16 @@
     </Or>
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+  <!-- Ignore SQL_PREPARED_STATEMENT_GENERATED_FROM_NONCONSTANT_STRING warnings for Timeline Phoenix storage. -->
+  <!-- Since we're using dynamic columns, we have to generate SQL statements dynamically -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.timelineservice.storage.PhoenixOfflineAggregationWriterImpl" />
+    <Or>
+      <Method name="storeEntityVariableLengthFields" />
+      <Method name="writeAggregatedEntity" />
+    </Or>
+    <Bug pattern="SQL_PREPARED_STATEMENT_GENERATED_FROM_NONCONSTANT_STRING" />
+  </Match>
   
   <!-- Following fields are used in ErrorsAndWarningsBlock, which is not a part of analysis of findbugs -->
   <Match>

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml

@@ -47,6 +47,10 @@
       <groupId>commons-logging</groupId>
       <artifactId>commons-logging</artifactId>
     </dependency>
+    <dependency>
+      <groupId>javax.xml.bind</groupId>
+      <artifactId>jaxb-api</artifactId>
+    </dependency>
 
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>

+ 14 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java

@@ -567,4 +567,18 @@ public abstract class AllocateResponse {
    */
   @Deprecated
   public abstract List<ContainerResourceDecrease> getDecreasedContainers();
+
+  /**
+   * The address of collector that belong to this app
+   *
+   * @return The address of collector that belong to this attempt
+   */
+  @Public
+  @Unstable
+  public abstract String getCollectorAddr();
+
+  @Private
+  @Unstable
+  public abstract void setCollectorAddr(String collectorAddr);
+
 }

+ 6 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java

@@ -34,6 +34,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.util.TimelineServiceHelper;
 
 /**
  * <p>
@@ -231,11 +232,8 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
    */
   public void setRelatedEntities(
       Map<String, Set<String>> relatedEntities) {
-    if (relatedEntities != null && !(relatedEntities instanceof HashMap)) {
-      this.relatedEntities = new HashMap<String, Set<String>>(relatedEntities);
-    } else {
-      this.relatedEntities = (HashMap<String, Set<String>>) relatedEntities;
-    }
+    this.relatedEntities = TimelineServiceHelper.mapCastToHashMap(
+        relatedEntities);
   }
 
   /**
@@ -297,11 +295,8 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
    *          a map of primary filters
    */
   public void setPrimaryFilters(Map<String, Set<Object>> primaryFilters) {
-    if (primaryFilters != null && !(primaryFilters instanceof HashMap)) {
-      this.primaryFilters = new HashMap<String, Set<Object>>(primaryFilters);
-    } else {
-      this.primaryFilters = (HashMap<String, Set<Object>>) primaryFilters;
-    }
+    this.primaryFilters =
+        TimelineServiceHelper.mapCastToHashMap(primaryFilters);
   }
 
   /**
@@ -350,11 +345,7 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
    *          a map of other information
    */
   public void setOtherInfo(Map<String, Object> otherInfo) {
-    if (otherInfo != null && !(otherInfo instanceof HashMap)) {
-      this.otherInfo = new HashMap<String, Object>(otherInfo);
-    } else {
-      this.otherInfo = (HashMap<String, Object>) otherInfo;
-    }
+    this.otherInfo = TimelineServiceHelper.mapCastToHashMap(otherInfo);
   }
 
   /**

+ 3 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java

@@ -29,6 +29,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.util.TimelineServiceHelper;
 
 /**
  * The class that contains the information of an event that is related to some
@@ -135,11 +136,8 @@ public class TimelineEvent implements Comparable<TimelineEvent> {
    *          a map of of the information of the event
    */
   public void setEventInfo(Map<String, Object> eventInfo) {
-    if (eventInfo != null && !(eventInfo instanceof HashMap)) {
-      this.eventInfo = new HashMap<String, Object>(eventInfo);
-    } else {
-      this.eventInfo = (HashMap<String, Object>) eventInfo;
-    }
+    this.eventInfo = TimelineServiceHelper.mapCastToHashMap(
+        eventInfo);
   }
 
   @Override

+ 41 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationAttemptEntity.java

@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This entity represents an application attempt.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ApplicationAttemptEntity extends HierarchicalTimelineEntity {
+  public ApplicationAttemptEntity() {
+    super(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString());
+  }
+
+  public ApplicationAttemptEntity(TimelineEntity entity) {
+    super(entity);
+    if (!entity.getType().equals(
+        TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString())) {
+      throw new IllegalArgumentException("Incompatible entity type: "
+          + getId());
+    }
+  }
+}

+ 52 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationEntity.java

@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This entity represents an application.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ApplicationEntity extends HierarchicalTimelineEntity {
+  public static final String QUEUE_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "QUEUE";
+
+  public ApplicationEntity() {
+    super(TimelineEntityType.YARN_APPLICATION.toString());
+  }
+
+  public ApplicationEntity(TimelineEntity entity) {
+    super(entity);
+    if (!entity.getType().equals(
+        TimelineEntityType.YARN_APPLICATION.toString())) {
+      throw new IllegalArgumentException("Incompatible entity type: "
+          + getId());
+    }
+  }
+
+  public String getQueue() {
+    return getInfo().get(QUEUE_INFO_KEY).toString();
+  }
+
+  public void setQueue(String queue) {
+    addInfo(QUEUE_INFO_KEY, queue);
+  }
+}

+ 40 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ClusterEntity.java

@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This entity represents a YARN cluster.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ClusterEntity extends HierarchicalTimelineEntity {
+  public ClusterEntity() {
+    super(TimelineEntityType.YARN_CLUSTER.toString());
+  }
+
+  public ClusterEntity(TimelineEntity entity) {
+    super(entity);
+    if (!entity.getType().equals(TimelineEntityType.YARN_CLUSTER.toString())) {
+      throw new IllegalArgumentException("Incompatible entity type: "
+          + getId());
+    }
+  }
+}

+ 41 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ContainerEntity.java

@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This entity represents a container belonging to an application attempt.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ContainerEntity extends HierarchicalTimelineEntity {
+  public ContainerEntity() {
+    super(TimelineEntityType.YARN_CONTAINER.toString());
+  }
+
+  public ContainerEntity(TimelineEntity entity) {
+    super(entity);
+    if (!entity.getType().equals(
+        TimelineEntityType.YARN_CONTAINER.toString())) {
+      throw new IllegalArgumentException("Incompatible entity type: "
+          + getId());
+    }
+  }
+}

+ 191 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java

@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import java.util.Collection;
+import java.util.Date;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.xml.bind.annotation.XmlElement;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Entity that represents a record for flow activity. It's essentially a
+ * container entity for flow runs with limited information.
+ */
+@Public
+@Unstable
+public class FlowActivityEntity extends TimelineEntity {
+  public static final String CLUSTER_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "CLUSTER";
+  public static final String DATE_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "DATE";
+  public static final String USER_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "USER";
+  public static final String FLOW_NAME_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_NAME";
+
+  private final NavigableSet<FlowRunEntity> flowRuns = new TreeSet<>();
+
+  public FlowActivityEntity() {
+    super(TimelineEntityType.YARN_FLOW_ACTIVITY.toString());
+    // set config to null
+    setConfigs(null);
+  }
+
+  public FlowActivityEntity(String cluster, long time, String user,
+      String flowName) {
+    this();
+    setCluster(cluster);
+    setDate(time);
+    setUser(user);
+    setFlowName(flowName);
+  }
+
+  public FlowActivityEntity(TimelineEntity entity) {
+    super(entity);
+    if (!TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entity.getType())) {
+      throw new IllegalArgumentException("Incompatible entity type: " +
+          getId());
+    }
+    // set config to null
+    setConfigs(null);
+  }
+
+  @XmlElement(name = "id")
+  @Override
+  public String getId() {
+    // flow activity: cluster/day/user@flow_name
+    String id = super.getId();
+    if (id == null) {
+      StringBuilder sb = new StringBuilder();
+      sb.append(getCluster());
+      sb.append('/');
+      sb.append(getDate().getTime());
+      sb.append('/');
+      sb.append(getUser());
+      sb.append('@');
+      sb.append(getFlowName());
+      id = sb.toString();
+      setId(id);
+    }
+    return id;
+  }
+
+  @Override
+  public int compareTo(TimelineEntity entity) {
+    int comparison = getType().compareTo(entity.getType());
+    if (comparison == 0) {
+      // order by cluster, date (descending), user, and flow name
+      FlowActivityEntity other = (FlowActivityEntity)entity;
+      int clusterComparison = getCluster().compareTo(other.getCluster());
+      if (clusterComparison != 0) {
+        return clusterComparison;
+      }
+      int dateComparisonDescending =
+          (int)(other.getDate().getTime() - getDate().getTime()); // descending
+      if (dateComparisonDescending != 0) {
+        return dateComparisonDescending; // descending
+      }
+      int userComparison = getUser().compareTo(other.getUser());
+      if (userComparison != 0) {
+        return userComparison;
+      }
+      return getFlowName().compareTo(other.getFlowName());
+    } else {
+      return comparison;
+    }
+  }
+
+  /**
+   * Reuse the base class equals method.
+   */
+  @Override
+  public boolean equals(Object obj) {
+    return super.equals(obj);
+  }
+
+  /**
+   * Reuse the base class hashCode method.
+   */
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  public String getCluster() {
+    return (String)getInfo().get(CLUSTER_INFO_KEY);
+  }
+
+  public void setCluster(String cluster) {
+    addInfo(CLUSTER_INFO_KEY, cluster);
+  }
+
+  public Date getDate() {
+    Object date = getInfo().get(DATE_INFO_KEY);
+    if (date != null) {
+      if (date instanceof Long) {
+        return new Date((Long)date);
+      } else if (date instanceof Date) {
+        return (Date)date;
+      }
+    }
+    return null;
+  }
+
+  public void setDate(long time) {
+    Date date = new Date(time);
+    addInfo(DATE_INFO_KEY, date);
+  }
+
+  public String getUser() {
+    return (String)getInfo().get(USER_INFO_KEY);
+  }
+
+  public void setUser(String user) {
+    addInfo(USER_INFO_KEY, user);
+  }
+
+  public String getFlowName() {
+    return (String)getInfo().get(FLOW_NAME_INFO_KEY);
+  }
+
+  public void setFlowName(String flowName) {
+    addInfo(FLOW_NAME_INFO_KEY, flowName);
+  }
+
+  public void addFlowRun(FlowRunEntity run) {
+    flowRuns.add(run);
+  }
+
+  public void addFlowRuns(Collection<FlowRunEntity> runs) {
+    flowRuns.addAll(runs);
+  }
+
+  @XmlElement(name = "flowruns")
+  public NavigableSet<FlowRunEntity> getFlowRuns() {
+    return flowRuns;
+  }
+
+  public int getNumberOfRuns() {
+    return flowRuns.size();
+  }
+}

+ 126 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java

@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import javax.xml.bind.annotation.XmlElement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This entity represents a flow run.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class FlowRunEntity extends HierarchicalTimelineEntity {
+  public static final String USER_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "USER";
+  public static final String FLOW_NAME_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_NAME";
+  public static final String FLOW_VERSION_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "FLOW_VERSION";
+  public static final String FLOW_RUN_ID_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "FLOW_RUN_ID";
+  public static final String FLOW_RUN_END_TIME =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_RUN_END_TIME";
+
+  public FlowRunEntity() {
+    super(TimelineEntityType.YARN_FLOW_RUN.toString());
+    // set config to null
+    setConfigs(null);
+  }
+
+  public FlowRunEntity(TimelineEntity entity) {
+    super(entity);
+    if (!entity.getType().equals(
+        TimelineEntityType.YARN_FLOW_RUN.toString())) {
+      throw new IllegalArgumentException("Incompatible entity type: "
+          + getId());
+    }
+    // set config to null
+    setConfigs(null);
+  }
+
+  @XmlElement(name = "id")
+  @Override
+  public String getId() {
+    //Flow id schema: user@flow_name(or id)/run_id
+    String id = super.getId();
+    if (id == null) {
+      StringBuilder sb = new StringBuilder();
+      sb.append(getInfo().get(USER_INFO_KEY).toString());
+      sb.append('@');
+      sb.append(getInfo().get(FLOW_NAME_INFO_KEY).toString());
+      sb.append('/');
+      sb.append(getInfo().get(FLOW_RUN_ID_INFO_KEY).toString());
+      id = sb.toString();
+      setId(id);
+    }
+    return id;
+  }
+
+  public String getUser() {
+    return (String)getInfo().get(USER_INFO_KEY);
+  }
+
+  public void setUser(String user) {
+    addInfo(USER_INFO_KEY, user);
+  }
+
+  public String getName() {
+    return (String)getInfo().get(FLOW_NAME_INFO_KEY);
+  }
+
+  public void setName(String name) {
+    addInfo(FLOW_NAME_INFO_KEY, name);
+  }
+
+  public String getVersion() {
+    return (String)getInfo().get(FLOW_VERSION_INFO_KEY);
+  }
+
+  public void setVersion(String version) {
+    addInfo(FLOW_VERSION_INFO_KEY, version);
+  }
+
+  public long getRunId() {
+    Object runId = getInfo().get(FLOW_RUN_ID_INFO_KEY);
+    return runId == null ? 0L : (Long) runId;
+  }
+
+  public void setRunId(long runId) {
+    addInfo(FLOW_RUN_ID_INFO_KEY, runId);
+  }
+
+  public long getStartTime() {
+    return getCreatedTime();
+  }
+
+  public void setStartTime(long startTime) {
+    setCreatedTime(startTime);
+  }
+
+  public long getMaxEndTime() {
+    Object time = getInfo().get(FLOW_RUN_END_TIME);
+    return time == null ? 0L : (Long)time;
+  }
+
+  public void setMaxEndTime(long endTime) {
+    addInfo(FLOW_RUN_END_TIME, endTime);
+  }
+}

+ 133 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/HierarchicalTimelineEntity.java

@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * This class extends timeline entity and defines parent-child relationships
+ * with other entities.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class HierarchicalTimelineEntity extends TimelineEntity {
+  public static final String PARENT_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "PARENT_ENTITY";
+  public static final String CHILDREN_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "CHILDREN_ENTITY";
+
+  HierarchicalTimelineEntity(TimelineEntity entity) {
+    super(entity);
+  }
+
+  HierarchicalTimelineEntity(String type) {
+    super(type);
+  }
+
+  public Identifier getParent() {
+    Object obj = getInfo().get(PARENT_INFO_KEY);
+    if (obj != null) {
+      if (obj instanceof Identifier) {
+        return (Identifier) obj;
+      } else {
+        throw new YarnRuntimeException(
+            "Parent info is invalid identifier object");
+      }
+    }
+    return null;
+  }
+
+  public void setParent(Identifier parent) {
+    validateParent(parent.getType());
+    addInfo(PARENT_INFO_KEY, parent);
+  }
+
+  public void setParent(String type, String id) {
+    setParent(new Identifier(type, id));
+  }
+
+  @SuppressWarnings("unchecked")
+  public Set<Identifier> getChildren() {
+    Object identifiers = getInfo().get(CHILDREN_INFO_KEY);
+    if (identifiers == null) {
+      return new HashSet<>();
+    }
+    TimelineEntityType thisType = TimelineEntityType.valueOf(getType());
+    if (identifiers instanceof Set<?>) {
+      for (Object identifier : (Set<?>) identifiers) {
+        if (!(identifier instanceof Identifier)) {
+          throw new YarnRuntimeException(
+              "Children info contains invalid identifier object");
+        } else {
+          validateChild((Identifier) identifier, thisType);
+        }
+      }
+    } else {
+      throw new YarnRuntimeException(
+          "Children info is invalid identifier set");
+    }
+    Set<Identifier> children = (Set<Identifier>) identifiers;
+    return children;
+  }
+
+  public void setChildren(Set<Identifier> children) {
+    addInfo(CHILDREN_INFO_KEY, children);
+  }
+
+  public void addChildren(Set<Identifier> children) {
+    TimelineEntityType thisType = TimelineEntityType.valueOf(getType());
+    for (Identifier child : children) {
+      validateChild(child, thisType);
+    }
+    Set<Identifier> existingChildren = getChildren();
+    existingChildren.addAll(children);
+    setChildren(existingChildren);
+  }
+
+  public void addChild(Identifier child) {
+    addChildren(Collections.singleton(child));
+  }
+
+  public void addChild(String type, String id) {
+    addChild(new Identifier(type, id));
+  }
+
+  private void validateParent(String type) {
+    TimelineEntityType parentType = TimelineEntityType.valueOf(type);
+    TimelineEntityType thisType = TimelineEntityType.valueOf(getType());
+    if (!thisType.isParent(parentType)) {
+      throw new IllegalArgumentException(
+          type + " is not the acceptable parent of " + this.getType());
+    }
+  }
+
+  private void validateChild(Identifier child, TimelineEntityType thisType) {
+    TimelineEntityType childType = TimelineEntityType.valueOf(child.getType());
+    if (!thisType.isChild(childType)) {
+      throw new IllegalArgumentException(
+          child.getType() + " is not the acceptable child of " +
+              this.getType());
+    }
+  }
+}

+ 40 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/QueueEntity.java

@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This entity represents a queue.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class QueueEntity extends HierarchicalTimelineEntity {
+  public QueueEntity() {
+    super(TimelineEntityType.YARN_QUEUE.toString());
+  }
+
+  public QueueEntity(TimelineEntity entity) {
+    super(entity);
+    if (!entity.getType().equals(TimelineEntityType.YARN_QUEUE.toString())) {
+      throw new IllegalArgumentException("Incompatible entity type: "
+          + getId());
+    }
+  }
+}

+ 62 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntities.java

@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This class hosts a set of timeline entities.
+ */
+@XmlRootElement(name = "entities")
+@XmlAccessorType(XmlAccessType.NONE)
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TimelineEntities {
+
+  private List<TimelineEntity> entities = new ArrayList<>();
+
+  public TimelineEntities() {
+
+  }
+
+  @XmlElement(name = "entities")
+  public List<TimelineEntity> getEntities() {
+    return entities;
+  }
+
+  public void setEntities(List<TimelineEntity> timelineEntities) {
+    this.entities = timelineEntities;
+  }
+
+  public void addEntities(List<TimelineEntity> timelineEntities) {
+    this.entities.addAll(timelineEntities);
+  }
+
+  public void addEntity(TimelineEntity entity) {
+    entities.add(entity);
+  }
+}

+ 584 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java

@@ -0,0 +1,584 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.util.TimelineServiceHelper;
+import org.codehaus.jackson.annotate.JsonSetter;
+
+/**
+ * The basic timeline entity data structure for timeline service v2. Timeline
+ * entity objects are not thread safe and should not be accessed concurrently.
+ * All collection members will be initialized into empty collections. Two
+ * timeline entities are equal iff. their type and id are identical.
+ *
+ * All non-primitive type, non-collection members will be initialized into null.
+ * User should set the type and id of a timeline entity to make it valid (can be
+ * checked by using the {@link #isValid()} method). Callers to the getters
+ * should perform null checks for non-primitive type, non-collection members.
+ *
+ * Callers are recommended not to alter the returned collection objects from the
+ * getters.
+ */
+@XmlRootElement(name = "entity")
+@XmlAccessorType(XmlAccessType.NONE)
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TimelineEntity implements Comparable<TimelineEntity> {
+  protected final static String SYSTEM_INFO_KEY_PREFIX = "SYSTEM_INFO_";
+
+  /**
+   * Identifier of timeline entity(entity id + entity type).
+   */
+  @XmlRootElement(name = "identifier")
+  @XmlAccessorType(XmlAccessType.NONE)
+  public static class Identifier {
+    private String type;
+    private String id;
+
+    public Identifier(String type, String id) {
+      this.type = type;
+      this.id = id;
+    }
+
+    public Identifier() {
+
+    }
+
+    @XmlElement(name = "type")
+    public String getType() {
+      return type;
+    }
+
+    public void setType(String entityType) {
+      this.type = entityType;
+    }
+
+    @XmlElement(name = "id")
+    public String getId() {
+      return id;
+    }
+
+    public void setId(String entityId) {
+      this.id = entityId;
+    }
+
+    @Override
+    public String toString() {
+      return "TimelineEntity[" +
+          "type='" + type + '\'' +
+          ", id='" + id + '\'' + "]";
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((id == null) ? 0 : id.hashCode());
+      result =
+        prime * result + ((type == null) ? 0 : type.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (!(obj instanceof Identifier)) {
+        return false;
+      }
+      Identifier other = (Identifier) obj;
+      if (id == null) {
+        if (other.getId() != null) {
+          return false;
+        }
+      } else if (!id.equals(other.getId())) {
+        return false;
+      }
+      if (type == null) {
+        if (other.getType() != null) {
+          return false;
+        }
+      } else if (!type.equals(other.getType())) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  private TimelineEntity real;
+  private Identifier identifier;
+  private HashMap<String, Object> info = new HashMap<>();
+  private HashMap<String, String> configs = new HashMap<>();
+  private Set<TimelineMetric> metrics = new HashSet<>();
+  // events should be sorted by timestamp in descending order
+  private NavigableSet<TimelineEvent> events = new TreeSet<>();
+  private HashMap<String, Set<String>> isRelatedToEntities = new HashMap<>();
+  private HashMap<String, Set<String>> relatesToEntities = new HashMap<>();
+  private Long createdTime;
+
+  public TimelineEntity() {
+    identifier = new Identifier();
+  }
+
+  /**
+   * <p>
+   * The constuctor is used to construct a proxy {@link TimelineEntity} or its
+   * subclass object from the real entity object that carries information.
+   * </p>
+   *
+   * <p>
+   * It is usually used in the case where we want to recover class polymorphism
+   * after deserializing the entity from its JSON form.
+   * </p>
+   * @param entity the real entity that carries information
+   */
+  public TimelineEntity(TimelineEntity entity) {
+    real = entity.getReal();
+  }
+
+  protected TimelineEntity(String type) {
+    this();
+    identifier.type = type;
+  }
+
+  @XmlElement(name = "type")
+  public String getType() {
+    if (real == null) {
+      return identifier.type;
+    } else {
+      return real.getType();
+    }
+  }
+
+  public void setType(String type) {
+    if (real == null) {
+      identifier.type = type;
+    } else {
+      real.setType(type);
+    }
+  }
+
+  @XmlElement(name = "id")
+  public String getId() {
+    if (real == null) {
+      return identifier.id;
+    } else {
+      return real.getId();
+    }
+  }
+
+  public void setId(String id) {
+    if (real == null) {
+      identifier.id = id;
+    } else {
+      real.setId(id);
+    }
+  }
+
+  public Identifier getIdentifier() {
+    if (real == null) {
+      return identifier;
+    } else {
+      return real.getIdentifier();
+    }
+  }
+
+  public void setIdentifier(Identifier entityIdentifier) {
+    if (real == null) {
+      this.identifier = entityIdentifier;
+    } else {
+      real.setIdentifier(entityIdentifier);
+    }
+  }
+
+  // required by JAXB
+  @InterfaceAudience.Private
+  @XmlElement(name = "info")
+  public HashMap<String, Object> getInfoJAXB() {
+    if (real == null) {
+      return info;
+    } else {
+      return real.getInfoJAXB();
+    }
+  }
+
+  public Map<String, Object> getInfo() {
+    if (real == null) {
+      return info;
+    } else {
+      return real.getInfo();
+    }
+  }
+
+  public void setInfo(Map<String, Object> entityInfos) {
+    if (real == null) {
+      this.info = TimelineServiceHelper.mapCastToHashMap(entityInfos);
+    } else {
+      real.setInfo(entityInfos);
+    }
+  }
+
+  public void addInfo(Map<String, Object> entityInfos) {
+    if (real == null) {
+      this.info.putAll(entityInfos);
+    } else {
+      real.addInfo(entityInfos);
+    }
+  }
+
+  public void addInfo(String key, Object value) {
+    if (real == null) {
+      info.put(key, value);
+    } else {
+      real.addInfo(key, value);
+    }
+  }
+
+  // required by JAXB
+  @InterfaceAudience.Private
+  @XmlElement(name = "configs")
+  public HashMap<String, String> getConfigsJAXB() {
+    if (real == null) {
+      return configs;
+    } else {
+      return real.getConfigsJAXB();
+    }
+  }
+
+  public Map<String, String> getConfigs() {
+    if (real == null) {
+      return configs;
+    } else {
+      return real.getConfigs();
+    }
+  }
+
+  public void setConfigs(Map<String, String> entityConfigs) {
+    if (real == null) {
+      this.configs = TimelineServiceHelper.mapCastToHashMap(entityConfigs);
+    } else {
+      real.setConfigs(entityConfigs);
+    }
+  }
+
+  public void addConfigs(Map<String, String> entityConfigs) {
+    if (real == null) {
+      this.configs.putAll(entityConfigs);
+    } else {
+      real.addConfigs(entityConfigs);
+    }
+  }
+
+  public void addConfig(String key, String value) {
+    if (real == null) {
+      configs.put(key, value);
+    } else {
+      real.addConfig(key, value);
+    }
+  }
+
+  @XmlElement(name = "metrics")
+  public Set<TimelineMetric> getMetrics() {
+    if (real == null) {
+      return metrics;
+    } else {
+      return real.getMetrics();
+    }
+  }
+
+  public void setMetrics(Set<TimelineMetric> entityMetrics) {
+    if (real == null) {
+      this.metrics = entityMetrics;
+    } else {
+      real.setMetrics(entityMetrics);
+    }
+  }
+
+  public void addMetrics(Set<TimelineMetric> entityMetrics) {
+    if (real == null) {
+      this.metrics.addAll(entityMetrics);
+    } else {
+      real.addMetrics(entityMetrics);
+    }
+  }
+
+  public void addMetric(TimelineMetric metric) {
+    if (real == null) {
+      metrics.add(metric);
+    } else {
+      real.addMetric(metric);
+    }
+  }
+
+  @XmlElement(name = "events")
+  public NavigableSet<TimelineEvent> getEvents() {
+    if (real == null) {
+      return events;
+    } else {
+      return real.getEvents();
+    }
+  }
+
+  public void setEvents(NavigableSet<TimelineEvent> entityEvents) {
+    if (real == null) {
+      this.events = entityEvents;
+    } else {
+      real.setEvents(entityEvents);
+    }
+  }
+
+  public void addEvents(Set<TimelineEvent> entityEvents) {
+    if (real == null) {
+      this.events.addAll(entityEvents);
+    } else {
+      real.addEvents(entityEvents);
+    }
+  }
+
+  public void addEvent(TimelineEvent event) {
+    if (real == null) {
+      events.add(event);
+    } else {
+      real.addEvent(event);
+    }
+  }
+
+  public Map<String, Set<String>> getIsRelatedToEntities() {
+    if (real == null) {
+      return isRelatedToEntities;
+    } else {
+      return real.getIsRelatedToEntities();
+    }
+  }
+
+  // required by JAXB
+  @InterfaceAudience.Private
+  @XmlElement(name = "isrelatedto")
+  public HashMap<String, Set<String>> getIsRelatedToEntitiesJAXB() {
+    if (real == null) {
+      return isRelatedToEntities;
+    } else {
+      return real.getIsRelatedToEntitiesJAXB();
+    }
+  }
+
+  @JsonSetter("isrelatedto")
+  public void setIsRelatedToEntities(
+      Map<String, Set<String>> isRelatedTo) {
+    if (real == null) {
+      this.isRelatedToEntities =
+          TimelineServiceHelper.mapCastToHashMap(isRelatedTo);
+    } else {
+      real.setIsRelatedToEntities(isRelatedTo);
+    }
+  }
+
+  public void addIsRelatedToEntities(
+      Map<String, Set<String>> isRelatedTo) {
+    if (real == null) {
+      for (Map.Entry<String, Set<String>> entry : isRelatedTo.entrySet()) {
+        Set<String> ids = this.isRelatedToEntities.get(entry.getKey());
+        if (ids == null) {
+          ids = new HashSet<>();
+          this.isRelatedToEntities.put(entry.getKey(), ids);
+        }
+        ids.addAll(entry.getValue());
+      }
+    } else {
+      real.addIsRelatedToEntities(isRelatedTo);
+    }
+  }
+
+  public void addIsRelatedToEntity(String type, String id) {
+    if (real == null) {
+      Set<String> ids = isRelatedToEntities.get(type);
+      if (ids == null) {
+        ids = new HashSet<>();
+        isRelatedToEntities.put(type, ids);
+      }
+      ids.add(id);
+    } else {
+      real.addIsRelatedToEntity(type, id);
+    }
+  }
+
+  // required by JAXB
+  @InterfaceAudience.Private
+  @XmlElement(name = "relatesto")
+  public HashMap<String, Set<String>> getRelatesToEntitiesJAXB() {
+    if (real == null) {
+      return relatesToEntities;
+    } else {
+      return real.getRelatesToEntitiesJAXB();
+    }
+  }
+
+  public Map<String, Set<String>> getRelatesToEntities() {
+    if (real == null) {
+      return relatesToEntities;
+    } else {
+      return real.getRelatesToEntities();
+    }
+  }
+
+  public void addRelatesToEntities(Map<String, Set<String>> relatesTo) {
+    if (real == null) {
+      for (Map.Entry<String, Set<String>> entry : relatesTo.entrySet()) {
+        Set<String> ids = this.relatesToEntities.get(entry.getKey());
+        if (ids == null) {
+          ids = new HashSet<>();
+          this.relatesToEntities.put(entry.getKey(), ids);
+        }
+        ids.addAll(entry.getValue());
+      }
+    } else {
+      real.addRelatesToEntities(relatesTo);
+    }
+  }
+
+  public void addRelatesToEntity(String type, String id) {
+    if (real == null) {
+      Set<String> ids = relatesToEntities.get(type);
+      if (ids == null) {
+        ids = new HashSet<>();
+        relatesToEntities.put(type, ids);
+      }
+      ids.add(id);
+    } else {
+      real.addRelatesToEntity(type, id);
+    }
+  }
+
+  @JsonSetter("relatesto")
+  public void setRelatesToEntities(Map<String, Set<String>> relatesTo) {
+    if (real == null) {
+      this.relatesToEntities =
+          TimelineServiceHelper.mapCastToHashMap(relatesTo);
+    } else {
+      real.setRelatesToEntities(relatesTo);
+    }
+  }
+
+  @XmlElement(name = "createdtime")
+  public Long getCreatedTime() {
+    if (real == null) {
+      return createdTime;
+    } else {
+      return real.getCreatedTime();
+    }
+  }
+
+  @JsonSetter("createdtime")
+  public void setCreatedTime(Long createdTs) {
+    if (real == null) {
+      this.createdTime = createdTs;
+    } else {
+      real.setCreatedTime(createdTs);
+    }
+  }
+
+  /**
+   * Set UID in info which will be then used for query by UI.
+   * @param uidKey key for UID in info.
+   * @param uId UID to be set for the key.
+   */
+  public void setUID(String uidKey, String uId) {
+    if (real == null) {
+      info.put(uidKey, uId);
+    } else {
+      real.addInfo(uidKey, uId);
+    }
+  }
+
+  public boolean isValid() {
+    return (getId() != null && getType() != null);
+  }
+
+  // When get hashCode for a timeline entity, or check if two timeline entities
+  // are equal, we only compare their identifiers (id and type)
+  @Override
+  public int hashCode() {
+    return getIdentifier().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof TimelineEntity)) {
+      return false;
+    }
+    TimelineEntity other = (TimelineEntity) obj;
+    return getIdentifier().equals(other.getIdentifier());
+  }
+
+  @Override
+  public int compareTo(TimelineEntity other) {
+    int comparison = getType().compareTo(other.getType());
+    if (comparison == 0) {
+      if (getCreatedTime() == null) {
+        if (other.getCreatedTime() == null) {
+          return getId().compareTo(other.getId());
+        } else {
+          return 1;
+        }
+      }
+      if (other.getCreatedTime() == null) {
+        return -1;
+      }
+      if (getCreatedTime() > other.getCreatedTime()) {
+        // Order by created time desc
+        return -1;
+      } else if (getCreatedTime() < other.getCreatedTime()) {
+        return 1;
+      } else {
+        return getId().compareTo(other.getId());
+      }
+    } else {
+      return comparison;
+    }
+  }
+
+  protected TimelineEntity getReal() {
+    return real == null ? this : real;
+  }
+
+  public String toString() {
+    if (real == null) {
+      return identifier.toString();
+    } else {
+      return real.toString();
+    }
+  }
+}

+ 101 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java

@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Defines type of entity.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum TimelineEntityType {
+  YARN_CLUSTER,
+  YARN_FLOW_RUN,
+  YARN_APPLICATION,
+  YARN_APPLICATION_ATTEMPT,
+  YARN_CONTAINER,
+  YARN_USER,
+  YARN_QUEUE,
+  YARN_FLOW_ACTIVITY;
+
+  /**
+   * Whether the input type can be a parent of this entity.
+   *
+   * @param type entity type.
+   * @return true, if this entity type is parent of passed entity type, false
+   *     otherwise.
+   */
+  public boolean isParent(TimelineEntityType type) {
+    switch (this) {
+    case YARN_CLUSTER:
+      return false;
+    case YARN_FLOW_RUN:
+      return YARN_FLOW_RUN == type || YARN_CLUSTER == type;
+    case YARN_APPLICATION:
+      return YARN_FLOW_RUN == type || YARN_CLUSTER == type;
+    case YARN_APPLICATION_ATTEMPT:
+      return YARN_APPLICATION == type;
+    case YARN_CONTAINER:
+      return YARN_APPLICATION_ATTEMPT == type;
+    case YARN_QUEUE:
+      return YARN_QUEUE == type;
+    default:
+      return false;
+    }
+  }
+
+  /**
+   * Whether the input type can be a child of this entity.
+   *
+   * @param type entity type.
+   * @return true, if this entity type is child of passed entity type, false
+   *     otherwise.
+   */
+  public boolean isChild(TimelineEntityType type) {
+    switch (this) {
+    case YARN_CLUSTER:
+      return YARN_FLOW_RUN == type || YARN_APPLICATION == type;
+    case YARN_FLOW_RUN:
+      return YARN_FLOW_RUN == type || YARN_APPLICATION == type;
+    case YARN_APPLICATION:
+      return YARN_APPLICATION_ATTEMPT == type;
+    case YARN_APPLICATION_ATTEMPT:
+      return YARN_CONTAINER == type;
+    case YARN_CONTAINER:
+      return false;
+    case YARN_QUEUE:
+      return YARN_QUEUE == type;
+    default:
+      return false;
+    }
+  }
+
+  /**
+   * Whether the type of this entity matches the type indicated by the input
+   * argument.
+   *
+   * @param typeString entity type represented as a string.
+   * @return true, if string representation of this entity type matches the
+   *     entity type passed.
+   */
+  public boolean matches(String typeString) {
+    return toString().equals(typeString);
+  }
+}

+ 133 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java

@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.util.TimelineServiceHelper;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * This class contains the information of an event that belongs to an entity.
+ * Users are free to define what the event means, such as starting an
+ * application, container being allocated, etc.
+ */
+@XmlRootElement(name = "event")
+@XmlAccessorType(XmlAccessType.NONE)
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TimelineEvent implements Comparable<TimelineEvent> {
+  public static final long INVALID_TIMESTAMP = 0L;
+
+  private String id;
+  private HashMap<String, Object> info = new HashMap<>();
+  private long timestamp;
+
+  public TimelineEvent() {
+
+  }
+
+  @XmlElement(name = "id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String eventId) {
+    this.id = eventId;
+  }
+
+  // required by JAXB
+  @InterfaceAudience.Private
+  @XmlElement(name = "info")
+  public HashMap<String, Object> getInfoJAXB() {
+    return info;
+  }
+
+  public Map<String, Object> getInfo() {
+    return info;
+  }
+
+  public void setInfo(Map<String, Object> infos) {
+    this.info = TimelineServiceHelper.mapCastToHashMap(infos);
+  }
+
+  public void addInfo(Map<String, Object> infos) {
+    this.info.putAll(infos);
+  }
+
+  public void addInfo(String key, Object value) {
+    info.put(key, value);
+  }
+
+  @XmlElement(name = "timestamp")
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long ts) {
+    this.timestamp = ts;
+  }
+
+  public boolean isValid() {
+    return (id != null && timestamp != INVALID_TIMESTAMP);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = (int) (timestamp ^ (timestamp >>> 32));
+    result = 31 * result + id.hashCode();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof TimelineEvent)) {
+      return false;
+    }
+
+    TimelineEvent event = (TimelineEvent) o;
+
+    if (timestamp != event.timestamp) {
+      return false;
+    }
+    if (!id.equals(event.id)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public int compareTo(TimelineEvent other) {
+    if (timestamp > other.timestamp) {
+      return -1;
+    } else if (timestamp < other.timestamp) {
+      return 1;
+    } else {
+      return id.compareTo(other.id);
+    }
+  }
+}

+ 289 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java

@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.TreeMap;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * This class contains the information of a metric that is related to some
+ * entity. Metric can either be a time series or single value.
+ */
+@XmlRootElement(name = "metric")
+@XmlAccessorType(XmlAccessType.NONE)
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TimelineMetric {
+
+  /**
+   * Type of metric.
+   */
+  public static enum Type {
+    SINGLE_VALUE,
+    TIME_SERIES
+  }
+
+  private Type type;
+  private String id;
+  // By default, not to do any aggregation operations. This field will NOT be
+  // persisted (like a "transient" member).
+  private TimelineMetricOperation realtimeAggregationOp
+      = TimelineMetricOperation.NOP;
+
+  private TreeMap<Long, Number> values
+      = new TreeMap<>(Collections.reverseOrder());
+
+  public TimelineMetric() {
+    this(Type.SINGLE_VALUE);
+  }
+
+  public TimelineMetric(Type type) {
+    this.type = type;
+  }
+
+
+  @XmlElement(name = "type")
+  public Type getType() {
+    return type;
+  }
+
+  public void setType(Type metricType) {
+    this.type = metricType;
+  }
+
+  @XmlElement(name = "id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String metricId) {
+    this.id = metricId;
+  }
+
+  /**
+   * Get the real time aggregation operation of this metric.
+   *
+   * @return Real time aggregation operation
+   */
+  // required by JAXB
+  @XmlElement(name = "aggregationOp")
+  public TimelineMetricOperation getRealtimeAggregationOp() {
+    return realtimeAggregationOp;
+  }
+
+  /**
+   * Set the real time aggregation operation of this metric.
+   *
+   * @param op A timeline metric operation that the metric should perform on
+   *           real time aggregations
+   */
+  public void setRealtimeAggregationOp(
+      final TimelineMetricOperation op) {
+    this.realtimeAggregationOp = op;
+  }
+
+  // required by JAXB
+  @InterfaceAudience.Private
+  @XmlElement(name = "values")
+  public TreeMap<Long, Number> getValuesJAXB() {
+    return values;
+  }
+
+  public Map<Long, Number> getValues() {
+    return values;
+  }
+
+  public void setValues(Map<Long, Number> vals) {
+    if (type == Type.SINGLE_VALUE) {
+      overwrite(vals);
+    } else {
+      if (vals != null) {
+        this.values = new TreeMap<>(Collections.reverseOrder());
+        this.values.putAll(vals);
+      } else {
+        this.values = null;
+      }
+    }
+  }
+
+  public void addValues(Map<Long, Number> vals) {
+    if (type == Type.SINGLE_VALUE) {
+      overwrite(vals);
+    } else {
+      this.values.putAll(vals);
+    }
+  }
+
+  public void addValue(long timestamp, Number value) {
+    if (type == Type.SINGLE_VALUE) {
+      values.clear();
+    }
+    values.put(timestamp, value);
+  }
+
+  private void overwrite(Map<Long, Number> vals) {
+    if (vals.size() > 1) {
+      throw new IllegalArgumentException(
+          "Values cannot contain more than one point in " +
+              Type.SINGLE_VALUE + " mode");
+    }
+    this.values.clear();
+    this.values.putAll(vals);
+  }
+
+  public boolean isValid() {
+    return (id != null);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = id.hashCode();
+    result = 31 * result + type.hashCode();
+    return result;
+  }
+
+  // Only check if type and id are equal
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof TimelineMetric)) {
+      return false;
+    }
+
+    TimelineMetric m = (TimelineMetric) o;
+
+    if (!id.equals(m.id)) {
+      return false;
+    }
+    if (type != m.type) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "{id: " + id + ", type: " + type +
+        ", realtimeAggregationOp: " +
+        realtimeAggregationOp + "; " + values.toString() +
+        "}";
+  }
+
+  /**
+   * Get the latest timeline metric as single value type.
+   *
+   * @param metric Incoming timeline metric
+   * @return The latest metric in the incoming metric
+   */
+  public static TimelineMetric getLatestSingleValueMetric(
+      TimelineMetric metric) {
+    if (metric.getType() == Type.SINGLE_VALUE) {
+      return metric;
+    } else {
+      TimelineMetric singleValueMetric = new TimelineMetric(Type.SINGLE_VALUE);
+      Long firstKey = metric.values.firstKey();
+      if (firstKey != null) {
+        Number firstValue = metric.values.get(firstKey);
+        singleValueMetric.addValue(firstKey, firstValue);
+      }
+      return singleValueMetric;
+    }
+  }
+
+  /**
+   * Get single data timestamp of the metric.
+   *
+   * @return the single data timestamp
+   */
+  public long getSingleDataTimestamp() {
+    if (this.type == Type.SINGLE_VALUE) {
+      if (values.size() == 0) {
+        throw new YarnRuntimeException("Values for this timeline metric is " +
+            "empty.");
+      } else {
+        return values.firstKey();
+      }
+    } else {
+      throw new YarnRuntimeException("Type for this timeline metric is not " +
+          "SINGLE_VALUE.");
+    }
+  }
+
+  /**
+   * Get single data value of the metric.
+   *
+   * @return the single data value
+   */
+  public Number getSingleDataValue() {
+    if (this.type == Type.SINGLE_VALUE) {
+      if (values.size() == 0) {
+        return null;
+      } else {
+        return values.get(values.firstKey());
+      }
+    } else {
+      throw new YarnRuntimeException("Type for this timeline metric is not " +
+          "SINGLE_VALUE.");
+    }
+  }
+
+  /**
+   * Aggregate an incoming metric to the base aggregated metric with the given
+   * operation state in a stateless fashion. The assumption here is
+   * baseAggregatedMetric and latestMetric should be single value data if not
+   * null.
+   *
+   * @param incomingMetric Incoming timeline metric to aggregate
+   * @param baseAggregatedMetric Base timeline metric
+   * @return Result metric after aggregation
+   */
+  public static TimelineMetric aggregateTo(TimelineMetric incomingMetric,
+      TimelineMetric baseAggregatedMetric) {
+    return aggregateTo(incomingMetric, baseAggregatedMetric, null);
+  }
+
+  /**
+   * Aggregate an incoming metric to the base aggregated metric with the given
+   * operation state. The assumption here is baseAggregatedMetric and
+   * latestMetric should be single value data if not null.
+   *
+   * @param incomingMetric Incoming timeline metric to aggregate
+   * @param baseAggregatedMetric Base timeline metric
+   * @param state Operation state
+   * @return Result metric after aggregation
+   */
+  public static TimelineMetric aggregateTo(TimelineMetric incomingMetric,
+      TimelineMetric baseAggregatedMetric, Map<Object, Object> state) {
+    TimelineMetricOperation operation
+        = incomingMetric.getRealtimeAggregationOp();
+    return operation.aggregate(incomingMetric, baseAggregatedMetric, state);
+  }
+
+}

+ 115 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricCalculator.java

@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * A calculator for timeline metrics.
+ */
+public final class TimelineMetricCalculator {
+
+  private TimelineMetricCalculator() {
+    // do nothing.
+  }
+
+  /**
+   * Compare two not-null numbers.
+   * @param n1 Number n1
+   * @param n2 Number n2
+   * @return 0 if n1 equals n2, a negative int if n1 is less than n2, a
+   * positive int otherwise.
+   */
+  public static int compare(Number n1, Number n2) {
+    if (n1 == null || n2 == null) {
+      throw new YarnRuntimeException(
+          "Number to be compared shouldn't be null.");
+    }
+
+    if (n1 instanceof Integer || n1 instanceof Long) {
+      if (n1.longValue() == n2.longValue()) {
+        return 0;
+      } else {
+        return (n1.longValue() < n2.longValue()) ? -1 : 1;
+      }
+    }
+
+    if (n1 instanceof Float || n1 instanceof Double) {
+      if (n1.doubleValue() == n2.doubleValue()) {
+        return 0;
+      } else {
+        return (n1.doubleValue() < n2.doubleValue()) ? -1 : 1;
+      }
+    }
+
+    // TODO throw warnings/exceptions for other types of number.
+    throw new YarnRuntimeException("Unsupported types for number comparison: "
+        + n1.getClass().getName() + ", " + n2.getClass().getName());
+  }
+
+  /**
+   * Subtract operation between two Numbers.
+   * @param n1 Number n1
+   * @param n2 Number n2
+   * @return Number represent to (n1 - n2).
+   */
+  public static Number sub(Number n1, Number n2) {
+    if (n1 == null) {
+      throw new YarnRuntimeException(
+          "Number to be subtracted shouldn't be null.");
+    } else if (n2 == null) {
+      return n1;
+    }
+
+    if (n1 instanceof Integer || n1 instanceof Long) {
+      return n1.longValue() - n2.longValue();
+    }
+
+    if (n1 instanceof Float || n1 instanceof Double) {
+      return n1.doubleValue() - n2.doubleValue();
+    }
+
+    // TODO throw warnings/exceptions for other types of number.
+    return null;
+  }
+
+  /**
+   * Sum up two Numbers.
+   * @param n1 Number n1
+   * @param n2 Number n2
+   * @return Number represent to (n1 + n2).
+   */
+  public static Number sum(Number n1, Number n2) {
+    if (n1 == null) {
+      return n2;
+    } else if (n2 == null) {
+      return n1;
+    }
+
+    if (n1 instanceof Integer || n1 instanceof Long) {
+      return n1.longValue() + n2.longValue();
+    }
+
+    if (n1 instanceof Float || n1 instanceof Double) {
+      return n1.doubleValue() + n2.doubleValue();
+    }
+
+    // TODO throw warnings/exceptions for other types of number.
+    return null;
+  }
+}

+ 167 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricOperation.java

@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import java.util.Map;
+
+/**
+ * Aggregation operations.
+ */
+public enum TimelineMetricOperation {
+  NOP("NOP") {
+    /**
+     * Do nothing on the base metric.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state (not used)
+     * @return Metric b
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming,
+        TimelineMetric base, Map<Object, Object> state) {
+      return base;
+    }
+  },
+  MAX("MAX") {
+    /**
+     * Keep the greater value of incoming and base. Stateless operation.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state (not used)
+     * @return the greater value of a and b
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming,
+        TimelineMetric base, Map<Object, Object> state) {
+      if (base == null) {
+        return incoming;
+      }
+      Number incomingValue = incoming.getSingleDataValue();
+      Number aggregateValue = base.getSingleDataValue();
+      if (aggregateValue == null) {
+        aggregateValue = Long.MIN_VALUE;
+      }
+      if (TimelineMetricCalculator.compare(incomingValue, aggregateValue) > 0) {
+        base.addValue(incoming.getSingleDataTimestamp(), incomingValue);
+      }
+      return base;
+    }
+  },
+  REPLACE("REPLACE") {
+    /**
+     * Replace the base metric with the incoming value. Stateless operation.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state (not used)
+     * @return Metric a
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming,
+        TimelineMetric base,
+        Map<Object, Object> state) {
+      return incoming;
+    }
+  },
+  SUM("SUM") {
+    /**
+     * Return the sum of the incoming metric and the base metric if the
+     * operation is stateless. For stateful operations, also subtract the
+     * value of the timeline metric mapped to the PREV_METRIC_STATE_KEY
+     * in the state object.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state (PREV_METRIC_STATE_KEY's value as Metric p)
+     * @return A metric with value a + b - p
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming, TimelineMetric base,
+        Map<Object, Object> state) {
+      if (base == null) {
+        return incoming;
+      }
+      Number incomingValue = incoming.getSingleDataValue();
+      Number aggregateValue = base.getSingleDataValue();
+      Number result
+          = TimelineMetricCalculator.sum(incomingValue, aggregateValue);
+
+      // If there are previous value in the state, we will take it off from the
+      // sum
+      if (state != null) {
+        Object prevMetric = state.get(PREV_METRIC_STATE_KEY);
+        if (prevMetric instanceof TimelineMetric) {
+          result = TimelineMetricCalculator.sub(result,
+              ((TimelineMetric) prevMetric).getSingleDataValue());
+        }
+      }
+      base.addValue(incoming.getSingleDataTimestamp(), result);
+      return base;
+    }
+  },
+  AVG("AVERAGE") {
+    /**
+     * Return the average value of the incoming metric and the base metric,
+     * with a given state. Not supported yet.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state
+     * @return Not finished yet
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming, TimelineMetric base,
+        Map<Object, Object> state) {
+      // Not supported yet
+      throw new UnsupportedOperationException(
+          "Unsupported aggregation operation: AVERAGE");
+    }
+  };
+
+  public static final String PREV_METRIC_STATE_KEY = "PREV_METRIC";
+
+  /**
+   * Perform the aggregation operation.
+   *
+   * @param incoming Incoming metric
+   * @param aggregate Base aggregation metric
+   * @param state Operation state
+   * @return Result metric for this aggregation operation
+   */
+  public TimelineMetric aggregate(TimelineMetric incoming,
+      TimelineMetric aggregate, Map<Object, Object> state) {
+    return exec(incoming, aggregate, state);
+  }
+
+  private final String opName;
+
+  TimelineMetricOperation(String opString) {
+    opName = opString;
+  }
+
+  @Override
+  public String toString() {
+    return this.opName;
+  }
+
+  abstract TimelineMetric exec(TimelineMetric incoming, TimelineMetric base,
+      Map<Object, Object> state);
+}

+ 167 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineWriteResponse.java

@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A class that holds a list of put errors. This is the response returned when a
+ * list of {@link TimelineEntity} objects is added to the timeline. If there are
+ * errors in storing individual entity objects, they will be indicated in the
+ * list of errors.
+ */
+@XmlRootElement(name = "response")
+@XmlAccessorType(XmlAccessType.NONE)
+@Public
+@Unstable
+public class TimelineWriteResponse {
+
+  private List<TimelineWriteError> errors = new ArrayList<TimelineWriteError>();
+
+  public TimelineWriteResponse() {
+
+  }
+
+  /**
+   * Get a list of {@link TimelineWriteError} instances.
+   *
+   * @return a list of {@link TimelineWriteError} instances
+   */
+  @XmlElement(name = "errors")
+  public List<TimelineWriteError> getErrors() {
+    return errors;
+  }
+
+  /**
+   * Add a single {@link TimelineWriteError} instance into the existing list.
+   *
+   * @param error
+   *          a single {@link TimelineWriteError} instance
+   */
+  public void addError(TimelineWriteError error) {
+    errors.add(error);
+  }
+
+  /**
+   * Add a list of {@link TimelineWriteError} instances into the existing list.
+   *
+   * @param writeErrors
+   *          a list of {@link TimelineWriteError} instances
+   */
+  public void addErrors(List<TimelineWriteError> writeErrors) {
+    this.errors.addAll(writeErrors);
+  }
+
+  /**
+   * Set the list to the given list of {@link TimelineWriteError} instances.
+   *
+   * @param writeErrors
+   *          a list of {@link TimelineWriteError} instances
+   */
+  public void setErrors(List<TimelineWriteError> writeErrors) {
+    this.errors.clear();
+    this.errors.addAll(writeErrors);
+  }
+
+  /**
+   * A class that holds the error code for one entity.
+   */
+  @XmlRootElement(name = "error")
+  @XmlAccessorType(XmlAccessType.NONE)
+  @Public
+  @Unstable
+  public static class TimelineWriteError {
+
+    /**
+     * Error code returned if an IOException is encountered when storing an
+     * entity.
+     */
+    public static final int IO_EXCEPTION = 1;
+
+    private String entityId;
+    private String entityType;
+    private int errorCode;
+
+    /**
+     * Get the entity Id.
+     *
+     * @return the entity Id
+     */
+    @XmlElement(name = "entity")
+    public String getEntityId() {
+      return entityId;
+    }
+
+    /**
+     * Set the entity Id.
+     *
+     * @param id the entity Id.
+     */
+    public void setEntityId(String id) {
+      this.entityId = id;
+    }
+
+    /**
+     * Get the entity type.
+     *
+     * @return the entity type
+     */
+    @XmlElement(name = "entitytype")
+    public String getEntityType() {
+      return entityType;
+    }
+
+    /**
+     * Set the entity type.
+     *
+     * @param type the entity type.
+     */
+    public void setEntityType(String type) {
+      this.entityType = type;
+    }
+
+    /**
+     * Get the error code.
+     *
+     * @return an error code
+     */
+    @XmlElement(name = "errorcode")
+    public int getErrorCode() {
+      return errorCode;
+    }
+
+    /**
+     * Set the error code to the given error code.
+     *
+     * @param code an error code.
+     */
+    public void setErrorCode(int code) {
+      this.errorCode = code;
+    }
+
+  }
+
+}

+ 40 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/UserEntity.java

@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This entity represents a user.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class UserEntity extends TimelineEntity {
+  public UserEntity() {
+    super(TimelineEntityType.YARN_USER.toString());
+  }
+
+  public UserEntity(TimelineEntity entity) {
+    super(entity);
+    if (!entity.getType().equals(TimelineEntityType.YARN_USER.toString())) {
+      throw new IllegalArgumentException("Incompatible entity type: "
+          + getId());
+    }
+  }
+}

+ 26 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/package-info.java

@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Package org.apache.hadoop.yarn.api.records.timelineservice contains classes
+ * which define the data model for ATSv2.
+ */
+@InterfaceAudience.Public
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.classification.InterfaceAudience;

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

@@ -96,6 +96,10 @@ public class YarnConfiguration extends Configuration {
         new DeprecationDelta(RM_ZK_RETRY_INTERVAL_MS,
             CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS),
     });
+    Configuration.addDeprecations(new DeprecationDelta[] {
+        new DeprecationDelta(RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
+            SYSTEM_METRICS_PUBLISHER_ENABLED)
+    });
   }
 
   //Configurations
@@ -146,6 +150,7 @@ public class YarnConfiguration extends Configuration {
   public static final String RM_PREFIX = "yarn.resourcemanager.";
 
   public static final String RM_CLUSTER_ID = RM_PREFIX + "cluster-id";
+  public static final String DEFAULT_RM_CLUSTER_ID = "yarn_cluster";
 
   public static final String RM_HOSTNAME = RM_PREFIX + "hostname";
 
@@ -500,16 +505,37 @@ public class YarnConfiguration extends Configuration {
 
   /**
    *  The setting that controls whether yarn system metrics is published on the
-   *  timeline server or not by RM.
+   *  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
+      + "system-metrics-publisher.enabled";
+  public static final boolean DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED =
+      false;
+
+  /**
+   *  The setting that controls whether yarn system metrics is published on the
+   *  timeline server or not by RM and NM. This configuration setting is for
+   *  ATS v2.
    */
-  public static final String RM_SYSTEM_METRICS_PUBLISHER_ENABLED =
-      RM_PREFIX + "system-metrics-publisher.enabled";
-  public static final boolean DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED = false;
+  public static final String SYSTEM_METRICS_PUBLISHER_ENABLED = YARN_PREFIX
+      + "system-metrics-publisher.enabled";
+  public static final boolean DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED = false;
+
+  /**
+   * The setting that controls whether yarn container events are published to
+   * the timeline service or not by RM. This configuration setting is for ATS
+   * V2
+   */
+  public static final String RM_PUBLISH_CONTAINER_EVENTS_ENABLED = YARN_PREFIX
+      + "rm.system-metrics-publisher.emit-container-events";
+  public static final boolean DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED =
+      false;
 
   public static final String RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE =
       RM_PREFIX + "system-metrics-publisher.dispatcher.pool-size";
-  public static final int DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE =
-      10;
+  public static final int
+      DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE = 10;
 
   //RM delegation token related keys
   public static final String RM_DELEGATION_KEY_UPDATE_INTERVAL_KEY =
@@ -956,6 +982,11 @@ public class YarnConfiguration extends Configuration {
     NM_PREFIX + "container-manager.thread-count";
   public static final int DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT = 20;
   
+  /** Number of threads container manager uses.*/
+  public static final String NM_COLLECTOR_SERVICE_THREAD_COUNT =
+      NM_PREFIX + "collector-service.thread-count";
+  public static final int DEFAULT_NM_COLLECTOR_SERVICE_THREAD_COUNT = 5;
+
   /** Number of threads used in cleanup.*/
   public static final String NM_DELETE_THREAD_COUNT = 
     NM_PREFIX +  "delete.thread-count";
@@ -983,6 +1014,13 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_NM_LOCALIZER_ADDRESS = "0.0.0.0:" +
     DEFAULT_NM_LOCALIZER_PORT;
   
+  /** Address where the collector service IPC is.*/
+  public static final String NM_COLLECTOR_SERVICE_ADDRESS =
+      NM_PREFIX + "collector-service.address";
+  public static final int DEFAULT_NM_COLLECTOR_SERVICE_PORT = 8048;
+  public static final String DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS =
+      "0.0.0.0:" + DEFAULT_NM_LOCALIZER_PORT;
+
   /** Interval in between cache cleanups.*/
   public static final String NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS =
     NM_PREFIX + "localizer.cache.cleanup.interval-ms";
@@ -2011,7 +2049,7 @@ public class YarnConfiguration extends Configuration {
   public static final String TIMELINE_SERVICE_UI_WEB_PATH_PREFIX =
       TIMELINE_SERVICE_PREFIX + "ui-web-path.";
 
-  /** Timeline client settings */
+  /** Timeline client settings. */
   public static final String TIMELINE_SERVICE_CLIENT_PREFIX =
       TIMELINE_SERVICE_PREFIX + "client.";
 
@@ -2150,6 +2188,53 @@ public class YarnConfiguration extends Configuration {
       = TIMELINE_SERVICE_PREFIX
       + "entity-file.fs-support-append";
 
+  /**
+   * Settings for timeline service v2.0
+   */
+  public static final String TIMELINE_SERVICE_WRITER_CLASS =
+      TIMELINE_SERVICE_PREFIX + "writer.class";
+
+  public static final String TIMELINE_SERVICE_READER_CLASS =
+      TIMELINE_SERVICE_PREFIX + "reader.class";
+
+  /** The setting that controls how often the timeline collector flushes the
+   * timeline writer.
+   */
+  public static final String TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS =
+      TIMELINE_SERVICE_PREFIX + "writer.flush-interval-seconds";
+
+  public static final int
+      DEFAULT_TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS = 60;
+
+  /**
+   * The name for setting that controls how long the final value of
+   * a metric of a completed app is retained before merging
+   * into the flow sum.
+   */
+  public static final String APP_FINAL_VALUE_RETENTION_THRESHOLD =
+      TIMELINE_SERVICE_PREFIX
+      + "hbase.coprocessor.app-final-value-retention-milliseconds";
+
+  /**
+   * The setting that controls how long the final value of a metric of a
+   * completed app is retained before merging into the flow sum. Up to this time
+   * after an application is completed out-of-order values that arrive can be
+   * recognized and discarded at the cost of increased storage.
+   */
+  public static final long DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD = 3 * 24
+      * 60 * 60 * 1000L;
+
+  public static final String ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS =
+      TIMELINE_SERVICE_PREFIX + "app-collector.linger-period.ms";
+
+  public static final int DEFAULT_ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS = 1000;
+
+  public static final String NUMBER_OF_ASYNC_ENTITIES_TO_MERGE =
+      TIMELINE_SERVICE_PREFIX
+          + "timeline-client.number-of-async-entities-to-merge";
+
+  public static final int DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE = 10;
+
   // mark app-history related configs @Private as application history is going
   // to be integrated into the timeline service
   @Private
@@ -2208,6 +2293,7 @@ public class YarnConfiguration extends Configuration {
   /** The listening endpoint for the timeline service application.*/
   public static final String TIMELINE_SERVICE_BIND_HOST =
       TIMELINE_SERVICE_PREFIX + "bind-host";
+  public static final String DEFAULT_TIMELINE_SERVICE_BIND_HOST = "0.0.0.0";
 
   /** The number of threads to handle client RPC API requests. */
   public static final String TIMELINE_SERVICE_HANDLER_THREAD_COUNT =
@@ -2407,6 +2493,16 @@ public class YarnConfiguration extends Configuration {
   public static final long    DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME =
       7*24*60*60*1000; // 7 days
 
+  // Timeline service v2 offlien aggregation related keys
+  public static final String TIMELINE_OFFLINE_AGGREGATION_PREFIX =
+      YarnConfiguration.TIMELINE_SERVICE_PREFIX + "aggregation.offline.";
+  public static final String PHOENIX_OFFLINE_STORAGE_CONN_STR
+      = TIMELINE_OFFLINE_AGGREGATION_PREFIX
+          + "phoenix.connectionString";
+
+  public static final String PHOENIX_OFFLINE_STORAGE_CONN_STR_DEFAULT
+      = "jdbc:phoenix:localhost:2181:/hbase";
+
   // ///////////////////////////////
   // Shared Cache Configs
   // ///////////////////////////////
@@ -3148,6 +3244,53 @@ public class YarnConfiguration extends Configuration {
         YarnConfiguration.DEFAULT_OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED);
   }
 
+  // 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,
+      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. */
   public static void main(String[] args) throws Exception {
     new YarnConfiguration(new Configuration()).writeXml(System.out);

+ 49 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/TimelineServiceHelper.java

@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.util;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+
+/**
+ * Helper class for Timeline service.
+ */
+@LimitedPrivate({ "MapReduce", "YARN" })
+public final class TimelineServiceHelper {
+
+  private TimelineServiceHelper() {
+    // Utility classes should not have a public or default constructor.
+  }
+
+  /**
+   * Cast map to HashMap for generic type.
+   * @param originalMap the map need to be casted
+   * @param <E> key type
+   * @param <V> value type
+   * @return casted HashMap object
+   */
+  public static <E, V> HashMap<E, V> mapCastToHashMap(
+      Map<E, V> originalMap) {
+    return originalMap == null ? null : originalMap instanceof HashMap ?
+        (HashMap<E, V>) originalMap : new HashMap<E, V>(originalMap);
+  }
+
+}

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

@@ -115,6 +115,7 @@ message AllocateResponseProto {
   repeated ContainerResourceDecreaseProto decreased_containers = 11;
   optional hadoop.common.TokenProto am_rm_token = 12;
   optional PriorityProto application_priority = 13;
+  optional string collector_addr = 14;
   repeated UpdateContainerErrorProto update_errors = 15;
   repeated UpdatedContainerProto updated_containers = 16;
 }

+ 100 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java

@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
+
+import org.junit.Test;
+
+public class TestTimelineMetric {
+
+  @Test
+  public void testTimelineMetricAggregation() {
+    long ts = System.currentTimeMillis();
+    // single_value metric add against null metric
+    TimelineMetric m1 = getSingleValueMetric("MEGA_BYTES_MILLIS",
+        TimelineMetricOperation.SUM, ts, 10000L);
+    TimelineMetric aggregatedMetric = TimelineMetric.aggregateTo(m1, null);
+    assertEquals(10000L, aggregatedMetric.getSingleDataValue());
+
+    TimelineMetric m2 = getSingleValueMetric("MEGA_BYTES_MILLIS",
+        TimelineMetricOperation.SUM, ts, 20000L);
+    aggregatedMetric = TimelineMetric.aggregateTo(m2, aggregatedMetric);
+    assertEquals(30000L, aggregatedMetric.getSingleDataValue());
+
+    // stateful sum test
+    Map<Object, Object> state = new HashMap<>();
+    state.put(TimelineMetricOperation.PREV_METRIC_STATE_KEY, m2);
+    TimelineMetric m2New = getSingleValueMetric("MEGA_BYTES_MILLIS",
+        TimelineMetricOperation.SUM, ts, 10000L);
+    aggregatedMetric = TimelineMetric.aggregateTo(m2New, aggregatedMetric,
+        state);
+    assertEquals(20000L, aggregatedMetric.getSingleDataValue());
+
+    // single_value metric max against single_value metric
+    TimelineMetric m3 = getSingleValueMetric("TRANSFER_RATE",
+        TimelineMetricOperation.MAX, ts, 150L);
+    TimelineMetric aggregatedMax = TimelineMetric.aggregateTo(m3, null);
+    assertEquals(150L, aggregatedMax.getSingleDataValue());
+
+    TimelineMetric m4 = getSingleValueMetric("TRANSFER_RATE",
+        TimelineMetricOperation.MAX, ts, 170L);
+    aggregatedMax = TimelineMetric.aggregateTo(m4, aggregatedMax);
+    assertEquals(170L, aggregatedMax.getSingleDataValue());
+
+    // single_value metric avg against single_value metric
+    TimelineMetric m5 = getSingleValueMetric("TRANSFER_RATE",
+        TimelineMetricOperation.AVG, ts, 150L);
+    try {
+      TimelineMetric.aggregateTo(m5, null);
+      fail("Taking average among metrics is not supported! ");
+    } catch (UnsupportedOperationException e) {
+      // Expected
+    }
+
+  }
+
+  private static TimelineMetric getSingleValueMetric(String id,
+      TimelineMetricOperation op, long timestamp, long value) {
+    TimelineMetric m = new TimelineMetric();
+    m.setId(id);
+    m.setType(Type.SINGLE_VALUE);
+    m.setRealtimeAggregationOp(op);
+    Map<Long, Number> metricValues = new HashMap<Long, Number>();
+    metricValues.put(timestamp, value);
+    m.setValues(metricValues);
+    return m;
+  }
+
+  private static TimelineMetric getTimeSeriesMetric(String id,
+      TimelineMetricOperation op, Map<Long, Number> metricValues) {
+    TimelineMetric m = new TimelineMetric();
+    m.setId(id);
+    m.setType(Type.TIME_SERIES);
+    m.setRealtimeAggregationOp(op);
+    m.setValues(metricValues);
+    return m;
+  }
+
+}

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java

@@ -125,6 +125,9 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
     // Ignore all YARN Application Timeline Service (version 1) properties
     configurationPrefixToSkipCompare.add("yarn.timeline-service.");
+    // skip deprecated RM_SYSTEM_METRICS_PUBLISHER_ENABLED
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED);
 
     // skip deprecated ZooKeeper settings
     configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_ADDRESS);

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml

@@ -70,6 +70,16 @@
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>
       <groupId>org.apache.hadoop</groupId>

+ 160 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -103,7 +103,6 @@ import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.log4j.LogManager;
 
@@ -220,6 +219,8 @@ public class ApplicationMaster {
   // Tracking url to which app master publishes info for clients to monitor
   private String appMasterTrackingUrl = "";
 
+  private boolean timelineServiceV2 = false;
+
   // App Master configuration
   // No. of containers to run shell command on
   @VisibleForTesting
@@ -553,6 +554,14 @@ public class ApplicationMaster {
         cliParser.getOptionValue("container_max_retries", "0"));
     containrRetryInterval = Integer.parseInt(cliParser.getOptionValue(
         "container_retry_interval", "0"));
+
+    if (YarnConfiguration.timelineServiceEnabled(conf)) {
+      timelineServiceV2 = YarnConfiguration.timelineServiceV2Enabled(conf);
+    } else {
+      timelineClient = null;
+      LOG.warn("Timeline service is not enabled");
+    }
+
     return true;
   }
 
@@ -600,7 +609,6 @@ public class ApplicationMaster {
         UserGroupInformation.createRemoteUser(appSubmitterUserName);
     appSubmitterUgi.addCredentials(credentials);
 
-
     AMRMClientAsync.AbstractCallbackHandler allocListener =
         new RMCallbackHandler();
     amRMClient = AMRMClientAsync.createAMRMClientAsync(1000, allocListener);
@@ -613,9 +621,18 @@ public class ApplicationMaster {
     nmClientAsync.start();
 
     startTimelineClient(conf);
+    if (timelineServiceV2) {
+      // need to bind timelineClient
+      amRMClient.registerTimelineClient(timelineClient);
+    }
     if(timelineClient != null) {
-      publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-          DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi);
+      if (timelineServiceV2) {
+        publishApplicationAttemptEventOnTimelineServiceV2(
+            DSEvent.DS_APP_ATTEMPT_START);
+      } else {
+        publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
+            DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi);
+      }
     }
 
     // Setup local RPC Server to accept status requests directly from clients
@@ -685,10 +702,16 @@ public class ApplicationMaster {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Void>() {
         @Override
         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
-            timelineClient = TimelineClient.createTimelineClient();
+            if (timelineServiceV2) {
+              timelineClient = TimelineClient.createTimelineClient(
+                  appAttemptID.getApplicationId());
+              LOG.info("Timeline service V2 client is enabled");
+            } else {
+              timelineClient = TimelineClient.createTimelineClient();
+              LOG.info("Timeline service V1 client is enabled");
+            }
             timelineClient.init(conf);
             timelineClient.start();
           } else {
@@ -718,9 +741,14 @@ public class ApplicationMaster {
       } catch (InterruptedException ex) {}
     }
 
-    if(timelineClient != null) {
-      publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-          DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
+    if (timelineClient != null) {
+      if (timelineServiceV2) {
+        publishApplicationAttemptEventOnTimelineServiceV2(
+            DSEvent.DS_APP_ATTEMPT_END);
+      } else {
+        publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
+            DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
+      }
     }
 
     // Join all launched threads
@@ -826,8 +854,12 @@ public class ApplicationMaster {
               + containerStatus.getContainerId());
         }
         if(timelineClient != null) {
-          publishContainerEndEvent(
-              timelineClient, containerStatus, domainId, appSubmitterUgi);
+          if (timelineServiceV2) {
+            publishContainerEndEventOnTimelineServiceV2(containerStatus);
+          } else {
+            publishContainerEndEvent(
+                timelineClient, containerStatus, domainId, appSubmitterUgi);
+          }
         }
       }
       
@@ -948,12 +980,18 @@ public class ApplicationMaster {
       }
       Container container = containers.get(containerId);
       if (container != null) {
-        applicationMaster.nmClientAsync.getContainerStatusAsync(containerId, container.getNodeId());
+        applicationMaster.nmClientAsync.getContainerStatusAsync(
+            containerId, container.getNodeId());
       }
       if(applicationMaster.timelineClient != null) {
-        applicationMaster.publishContainerStartEvent(
-            applicationMaster.timelineClient, container,
-            applicationMaster.domainId, applicationMaster.appSubmitterUgi);
+        if (applicationMaster.timelineServiceV2) {
+          applicationMaster.publishContainerStartEventOnTimelineServiceV2(
+              container);
+        } else {
+          applicationMaster.publishContainerStartEvent(
+              applicationMaster.timelineClient, container,
+              applicationMaster.domainId, applicationMaster.appSubmitterUgi);
+        }
       }
     }
 
@@ -1272,7 +1310,7 @@ public class ApplicationMaster {
       LOG.error("App Attempt "
           + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end")
           + " event could not be published for "
-          + appAttemptId.toString(), e);
+          + appAttemptID, e);
     }
   }
 
@@ -1319,4 +1357,109 @@ public class ApplicationMaster {
             shellId);
     return new Thread(runnableLaunchContainer);
   }
+
+  private void publishContainerStartEventOnTimelineServiceV2(
+      Container container) {
+    final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+        entity =
+            new org.apache.hadoop.yarn.api.records.timelineservice.
+            TimelineEntity();
+    entity.setId(container.getId().toString());
+    entity.setType(DSEntity.DS_CONTAINER.toString());
+    long ts = System.currentTimeMillis();
+    entity.setCreatedTime(ts);
+    entity.addInfo("user", appSubmitterUgi.getShortUserName());
+
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
+        new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
+    event.setTimestamp(ts);
+    event.setId(DSEvent.DS_CONTAINER_START.toString());
+    event.addInfo("Node", container.getNodeId().toString());
+    event.addInfo("Resources", container.getResource().toString());
+    entity.addEvent(event);
+
+    try {
+      appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public TimelinePutResponse run() throws Exception {
+          timelineClient.putEntities(entity);
+          return null;
+        }
+      });
+    } catch (Exception e) {
+      LOG.error("Container start event could not be published for "
+          + container.getId().toString(),
+          e instanceof UndeclaredThrowableException ? e.getCause() : e);
+    }
+  }
+
+  private void publishContainerEndEventOnTimelineServiceV2(
+      final ContainerStatus container) {
+    final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+        entity =
+            new org.apache.hadoop.yarn.api.records.timelineservice.
+            TimelineEntity();
+    entity.setId(container.getContainerId().toString());
+    entity.setType(DSEntity.DS_CONTAINER.toString());
+    //entity.setDomainId(domainId);
+    entity.addInfo("user", appSubmitterUgi.getShortUserName());
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
+        new  org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
+    event.setTimestamp(System.currentTimeMillis());
+    event.setId(DSEvent.DS_CONTAINER_END.toString());
+    event.addInfo("State", container.getState().name());
+    event.addInfo("Exit Status", container.getExitStatus());
+    entity.addEvent(event);
+
+    try {
+      appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public TimelinePutResponse run() throws Exception {
+          timelineClient.putEntities(entity);
+          return null;
+        }
+      });
+    } catch (Exception e) {
+      LOG.error("Container end event could not be published for "
+          + container.getContainerId().toString(),
+          e instanceof UndeclaredThrowableException ? e.getCause() : e);
+    }
+  }
+
+  private void publishApplicationAttemptEventOnTimelineServiceV2(
+      DSEvent appEvent) {
+    final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+        entity =
+            new org.apache.hadoop.yarn.api.records.timelineservice.
+            TimelineEntity();
+    entity.setId(appAttemptID.toString());
+    entity.setType(DSEntity.DS_APP_ATTEMPT.toString());
+    long ts = System.currentTimeMillis();
+    if (appEvent == DSEvent.DS_APP_ATTEMPT_START) {
+      entity.setCreatedTime(ts);
+    }
+    entity.addInfo("user", appSubmitterUgi.getShortUserName());
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
+        new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
+    event.setId(appEvent.toString());
+    event.setTimestamp(ts);
+    entity.addEvent(event);
+
+    try {
+      appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public TimelinePutResponse run() throws Exception {
+          timelineClient.putEntitiesAsync(entity);
+          return null;
+        }
+      });
+    } catch (Exception e) {
+      LOG.error("App Attempt "
+          + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end")
+          + " event could not be published for "
+          + appAttemptID,
+          e instanceof UndeclaredThrowableException ? e.getCause() : e);
+    }
+  }
+
 }

+ 42 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -22,8 +22,10 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.Vector;
 
 import org.apache.commons.cli.CommandLine;
@@ -187,6 +189,10 @@ public class Client {
   // Timeline domain writer access control
   private String modifyACLs = null;
 
+  private String flowName = null;
+  private String flowVersion = null;
+  private long flowRunId = 0L;
+
   // Command line options
   private Options opts;
 
@@ -258,7 +264,8 @@ public class Client {
     opts.addOption("shell_args", true, "Command line args for the shell script." +
         "Multiple args can be separated by empty space.");
     opts.getOption("shell_args").setArgs(Option.UNLIMITED_VALUES);
-    opts.addOption("shell_env", true, "Environment for shell script. Specified as env_key=env_val pairs");
+    opts.addOption("shell_env", true,
+        "Environment for shell script. Specified as env_key=env_val pairs");
     opts.addOption("shell_cmd_priority", true, "Priority for the shell command containers");
     opts.addOption("container_memory", true, "Amount of memory in MB 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");
@@ -284,6 +291,12 @@ public class Client {
         + "modify the timeline entities in the given domain");
     opts.addOption("create", false, "Flag to indicate whether to create the "
         + "domain specified with -domain.");
+    opts.addOption("flow_name", true, "Flow name which the distributed shell "
+        + "app belongs to");
+    opts.addOption("flow_version", true, "Flow version which the distributed "
+        + "shell app belongs to");
+    opts.addOption("flow_run_id", true, "Flow run ID which the distributed "
+        + "shell app belongs to");
     opts.addOption("help", false, "Print usage");
     opts.addOption("node_label_expression", true,
         "Node label expression to determine the nodes"
@@ -463,6 +476,20 @@ public class Client {
           + cliParser.getOptionValue("container_retry_interval"));
     }
 
+    if (cliParser.hasOption("flow_name")) {
+      flowName = cliParser.getOptionValue("flow_name");
+    }
+    if (cliParser.hasOption("flow_version")) {
+      flowVersion = cliParser.getOptionValue("flow_version");
+    }
+    if (cliParser.hasOption("flow_run_id")) {
+      try {
+        flowRunId = Long.parseLong(cliParser.getOptionValue("flow_run_id"));
+      } catch (NumberFormatException e) {
+        throw new IllegalArgumentException(
+            "Flow run is not a valid long value", e);
+      }
+    }
     return true;
   }
 
@@ -554,6 +581,18 @@ public class Client {
         .setAttemptFailuresValidityInterval(attemptFailuresValidityInterval);
     }
 
+    Set<String> tags = new HashSet<String>();
+    if (flowName != null) {
+      tags.add(TimelineUtils.generateFlowNameTag(flowName));
+    }
+    if (flowVersion != null) {
+      tags.add(TimelineUtils.generateFlowVersionTag(flowVersion));
+    }
+    if (flowRunId != 0) {
+      tags.add(TimelineUtils.generateFlowRunIdTag(flowRunId));
+    }
+    appContext.setApplicationTags(tags);
+
     // set local resources for the application master
     // local files or archives as needed
     // In this scenario, the jar file for the application master is part of the local resources			
@@ -667,7 +706,7 @@ public class Client {
 
     for (Map.Entry<String, String> entry : shellEnv.entrySet()) {
       vargs.add("--shell_env " + entry.getKey() + "=" + entry.getValue());
-    }			
+    }
     if (debugFlag) {
       vargs.add("--debug");
     }
@@ -683,7 +722,7 @@ public class Client {
       command.append(str).append(" ");
     }
 
-    LOG.info("Completed setting up app master command " + command.toString());	   
+    LOG.info("Completed setting up app master command " + command.toString());
     List<String> commands = new ArrayList<String>();
     commands.add(command.toString());		
 

+ 281 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -38,6 +38,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -55,29 +56,37 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
-import org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.impl.DirectTimelineWriter;
+import org.apache.hadoop.yarn.client.api.impl.TestTimelineClient;
 import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
 import org.apache.hadoop.yarn.client.api.impl.TimelineWriter;
-import org.apache.hadoop.yarn.client.api.impl.TestTimelineClient;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
+import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.timeline.PluginStoreTestUtils;
 import org.apache.hadoop.yarn.server.timeline.NameValuePair;
+import org.apache.hadoop.yarn.server.timeline.PluginStoreTestUtils;
 import org.apache.hadoop.yarn.server.timeline.TimelineVersion;
 import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher;
+import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
+import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
+import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -85,8 +94,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 
-import com.sun.jersey.api.client.ClientHandlerException;
-
 public class TestDistributedShell {
 
   private static final Log LOG =
@@ -99,6 +106,7 @@ public class TestDistributedShell {
   protected YarnConfiguration conf = null;
   private static final int NUM_NMS = 1;
   private static final float DEFAULT_TIMELINE_VERSION = 1.0f;
+  private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector";
 
   protected final static String APPMASTER_JAR =
       JarFinder.getJar(ApplicationMaster.class);
@@ -120,17 +128,36 @@ public class TestDistributedShell {
 
   private void setupInternal(int numNodeManager, float timelineVersion)
       throws Exception {
-
     LOG.info("Starting up YARN cluster");
-    
+
     conf = new YarnConfiguration();
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128);
+    // reduce the teardown waiting time
+    conf.setLong(YarnConfiguration.DISPATCHER_DRAIN_EVENTS_TIMEOUT, 1000);
     conf.set("yarn.log.dir", "target");
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    // mark if we need to launch the v1 timeline server
+    // disable aux-service based timeline aggregators
+    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.RM_SCHEDULER, CapacityScheduler.class.getName());
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
     conf.set("mapreduce.jobhistory.address",
         "0.0.0.0:" + ServerSocketUtil.getPort(10021, 10));
+    // Enable ContainersMonitorImpl
+    conf.set(YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
+        LinuxResourceCalculatorPlugin.class.getName());
+    conf.set(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE,
+        ProcfsBasedProcessTree.class.getName());
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, true);
+    conf.setBoolean(
+        YarnConfiguration.YARN_MINICLUSTER_CONTROL_RESOURCE_MONITORING,
+        true);
+    conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
+          true);
 
     // ATS version specific settings
     if (timelineVersion == 1.0f) {
@@ -148,6 +175,19 @@ public class TestDistributedShell {
       PluginStoreTestUtils.prepareConfiguration(conf, hdfsCluster);
       conf.set(YarnConfiguration.TIMELINE_SERVICE_ENTITY_GROUP_PLUGIN_CLASSES,
           DistributedShellTimelinePlugin.class.getName());
+    } else if (timelineVersion == 2.0f) {
+      // set version to 2
+      conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+      // disable v1 timeline server since we no longer have a server here
+      // 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 + ".class",
+          PerNodeTimelineCollectorsAuxService.class.getName());
+      conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
+          FileSystemTimelineWriterImpl.class,
+          org.apache.hadoop.yarn.server.timelineservice.storage.
+              TimelineWriter.class);
     } else {
       Assert.fail("Wrong timeline version number: " + timelineVersion);
     }
@@ -241,7 +281,30 @@ public class TestDistributedShell {
     testDSShell(true);
   }
 
+  @Test
+  @TimelineVersion(2.0f)
+  public void testDSShellWithoutDomainV2() throws Exception {
+    testDSShell(false);
+  }
+
   public void testDSShell(boolean haveDomain) throws Exception {
+    testDSShell(haveDomain, true);
+  }
+
+  @Test
+  @TimelineVersion(2.0f)
+  public void testDSShellWithoutDomainV2DefaultFlow() throws Exception {
+    testDSShell(false, true);
+  }
+
+  @Test
+  @TimelineVersion(2.0f)
+  public void testDSShellWithoutDomainV2CustomizedFlow() throws Exception {
+    testDSShell(false, false);
+  }
+
+  public void testDSShell(boolean haveDomain, boolean defaultFlow)
+      throws Exception {
     String[] args = {
         "--jar",
         APPMASTER_JAR,
@@ -268,9 +331,23 @@ public class TestDistributedShell {
           "writer_user writer_group",
           "--create"
       };
-      List<String> argsList = new ArrayList<String>(Arrays.asList(args));
-      argsList.addAll(Arrays.asList(domainArgs));
-      args = argsList.toArray(new String[argsList.size()]);
+      args = mergeArgs(args, domainArgs);
+    }
+    boolean isTestingTimelineV2 = false;
+    if (timelineVersionWatcher.getTimelineVersion() == 2.0f) {
+      isTestingTimelineV2 = true;
+      if (!defaultFlow) {
+        String[] flowArgs = {
+            "--flow_name",
+            "test_flow_name",
+            "--flow_version",
+            "test_flow_version",
+            "--flow_run_id",
+            "12345678"
+        };
+        args = mergeArgs(args, flowArgs);
+      }
+      LOG.info("Setup: Using timeline v2!");
     }
 
     LOG.info("Initializing DS Client");
@@ -297,13 +374,16 @@ public class TestDistributedShell {
 
     boolean verified = false;
     String errorMessage = "";
+    ApplicationId appId = null;
+    ApplicationReport appReport = null;
     while(!verified) {
       List<ApplicationReport> apps = yarnClient.getApplications();
       if (apps.size() == 0 ) {
         Thread.sleep(10);
         continue;
       }
-      ApplicationReport appReport = apps.get(0);
+      appReport = apps.get(0);
+      appId = appReport.getApplicationId();
       if(appReport.getHost().equals("N/A")) {
         Thread.sleep(10);
         continue;
@@ -315,13 +395,16 @@ public class TestDistributedShell {
       if (checkHostname(appReport.getHost()) && appReport.getRpcPort() == -1) {
         verified = true;
       }
-      if (appReport.getYarnApplicationState() == YarnApplicationState.FINISHED) {
+
+      if (appReport.getYarnApplicationState() == YarnApplicationState.FINISHED
+          && appReport.getFinalApplicationStatus() !=
+              FinalApplicationStatus.UNDEFINED) {
         break;
       }
     }
     Assert.assertTrue(errorMessage, verified);
     t.join();
-    LOG.info("Client run completed. Result=" + result);
+    LOG.info("Client run completed for testDSShell. Result=" + result);
     Assert.assertTrue(result.get());
 
     if (timelineVersionWatcher.getTimelineVersion() == 1.5f) {
@@ -342,6 +425,15 @@ public class TestDistributedShell {
       }
     }
 
+    TimelineDomain domain = null;
+    if (!isTestingTimelineV2) {
+      checkTimelineV1(haveDomain);
+    } else {
+      checkTimelineV2(haveDomain, appId, defaultFlow, appReport);
+    }
+  }
+
+  private void checkTimelineV1(boolean haveDomain) throws Exception {
     TimelineDomain domain = null;
     if (haveDomain) {
       domain = yarnCluster.getApplicationHistoryServer()
@@ -393,6 +485,179 @@ public class TestDistributedShell {
     }
   }
 
+  private void checkTimelineV2(boolean haveDomain, ApplicationId appId,
+      boolean defaultFlow, ApplicationReport appReport) throws Exception {
+    LOG.info("Started checkTimelineV2 ");
+    // For PoC check in /tmp/timeline_service_data YARN-3264
+    String tmpRoot =
+        FileSystemTimelineWriterImpl.DEFAULT_TIMELINE_SERVICE_STORAGE_DIR_ROOT
+            + "/entities/";
+
+    File tmpRootFolder = new File(tmpRoot);
+    try {
+      Assert.assertTrue(tmpRootFolder.isDirectory());
+      String basePath = tmpRoot +
+          YarnConfiguration.DEFAULT_RM_CLUSTER_ID + "/" +
+          UserGroupInformation.getCurrentUser().getShortUserName() +
+          (defaultFlow ?
+              "/" + appReport.getName() + "/" +
+                  TimelineUtils.DEFAULT_FLOW_VERSION +"/" +
+                  appReport.getStartTime() +"/" :
+              "/test_flow_name/test_flow_version/12345678/") +
+          appId.toString();
+      LOG.info("basePath: " + basePath);
+      // for this test, we expect DS_APP_ATTEMPT AND DS_CONTAINER dirs
+
+      // Verify DS_APP_ATTEMPT entities posted by the client
+      // there will be at least one attempt, look for that file
+      String appTimestampFileName =
+          "appattempt_" + appId.getClusterTimestamp() + "_000" + appId.getId()
+              + "_000001"
+              + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+      verifyEntityTypeFileExists(basePath, "DS_APP_ATTEMPT",
+          appTimestampFileName);
+
+      // Verify DS_CONTAINER entities posted by the client
+      String containerTimestampFileName =
+          "container_" + appId.getClusterTimestamp() + "_000" + appId.getId()
+              + "_01_000002.thist";
+      verifyEntityTypeFileExists(basePath, "DS_CONTAINER",
+          containerTimestampFileName);
+
+      // Verify NM posting container metrics info.
+      String containerMetricsTimestampFileName =
+          "container_" + appId.getClusterTimestamp() + "_000" + appId.getId()
+              + "_01_000001"
+              + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+      File containerEntityFile = verifyEntityTypeFileExists(basePath,
+          TimelineEntityType.YARN_CONTAINER.toString(),
+          containerMetricsTimestampFileName);
+      Assert.assertEquals(
+          "Container created event needs to be published atleast once",
+          1,
+          getNumOfStringOccurences(containerEntityFile,
+              ContainerMetricsConstants.CREATED_EVENT_TYPE));
+
+      // to avoid race condition of testcase, atleast check 4 times with sleep
+      // of 500ms
+      long numOfContainerFinishedOccurences = 0;
+      for (int i = 0; i < 4; i++) {
+        numOfContainerFinishedOccurences =
+            getNumOfStringOccurences(containerEntityFile,
+                ContainerMetricsConstants.FINISHED_EVENT_TYPE);
+        if (numOfContainerFinishedOccurences > 0) {
+          break;
+        } else {
+          Thread.sleep(500L);
+        }
+      }
+      Assert.assertEquals(
+          "Container finished event needs to be published atleast once",
+          1,
+          numOfContainerFinishedOccurences);
+
+      // Verify RM posting Application life cycle Events are getting published
+      String appMetricsTimestampFileName =
+          "application_" + appId.getClusterTimestamp() + "_000" + appId.getId()
+              + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+      File appEntityFile =
+          verifyEntityTypeFileExists(basePath,
+              TimelineEntityType.YARN_APPLICATION.toString(),
+              appMetricsTimestampFileName);
+      Assert.assertEquals(
+          "Application created event should be published atleast once",
+          1,
+          getNumOfStringOccurences(appEntityFile,
+              ApplicationMetricsConstants.CREATED_EVENT_TYPE));
+
+      // to avoid race condition of testcase, atleast check 4 times with sleep
+      // of 500ms
+      long numOfStringOccurences = 0;
+      for (int i = 0; i < 4; i++) {
+        numOfStringOccurences =
+            getNumOfStringOccurences(appEntityFile,
+                ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+        if (numOfStringOccurences > 0) {
+          break;
+        } else {
+          Thread.sleep(500L);
+        }
+      }
+      Assert.assertEquals(
+          "Application finished event should be published atleast once",
+          1,
+          numOfStringOccurences);
+
+      // Verify RM posting AppAttempt life cycle Events are getting published
+      String appAttemptMetricsTimestampFileName =
+          "appattempt_" + appId.getClusterTimestamp() + "_000" + appId.getId()
+              + "_000001"
+              + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+      File appAttemptEntityFile =
+          verifyEntityTypeFileExists(basePath,
+              TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(),
+              appAttemptMetricsTimestampFileName);
+      Assert.assertEquals(
+          "AppAttempt register event should be published atleast once",
+          1,
+          getNumOfStringOccurences(appAttemptEntityFile,
+              AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE));
+
+      Assert.assertEquals(
+          "AppAttempt finished event should be published atleast once",
+          1,
+          getNumOfStringOccurences(appAttemptEntityFile,
+              AppAttemptMetricsConstants.FINISHED_EVENT_TYPE));
+    } finally {
+      FileUtils.deleteDirectory(tmpRootFolder.getParentFile());
+    }
+  }
+
+  private File verifyEntityTypeFileExists(String basePath, String entityType,
+      String entityfileName) {
+    String outputDirPathForEntity = basePath + "/" + entityType + "/";
+    File outputDirForEntity = new File(outputDirPathForEntity);
+    Assert.assertTrue(outputDirForEntity.isDirectory());
+
+    String entityFilePath = outputDirPathForEntity + entityfileName;
+
+    File entityFile = new File(entityFilePath);
+    Assert.assertTrue(entityFile.exists());
+    return entityFile;
+  }
+
+  private long getNumOfStringOccurences(File entityFile, String searchString)
+      throws IOException {
+    BufferedReader reader = null;
+    String strLine;
+    long actualCount = 0;
+    try {
+      reader = new BufferedReader(new FileReader(entityFile));
+      while ((strLine = reader.readLine()) != null) {
+        if (strLine.trim().contains(searchString)) {
+          actualCount++;
+        }
+      }
+    } finally {
+      reader.close();
+    }
+    return actualCount;
+  }
+
+  /**
+   * Utility function to merge two String arrays to form a new String array for
+   * our argumemts.
+   *
+   * @param args
+   * @param newArgs
+   * @return a String array consists of {args, newArgs}
+   */
+  private String[] mergeArgs(String[] args, String[] newArgs) {
+    List<String> argsList = new ArrayList<String>(Arrays.asList(args));
+    argsList.addAll(Arrays.asList(newArgs));
+    return argsList.toArray(new String[argsList.size()]);
+  }
+
   /*
    * NetUtils.getHostname() returns a string in the form "hostname/ip".
    * Sometimes the hostname we get is the FQDN and sometimes the short name. In
@@ -1052,4 +1317,3 @@ public class TestDistributedShell {
     return numOfWords;
   }
 }
-

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java

@@ -41,7 +41,7 @@ public class TestDistributedShellWithNodeLabels {
   
   static final int NUM_NMS = 2;
   TestDistributedShell distShellTest;
- 
+
   @Before
   public void setup() throws Exception {
     distShellTest = new TestDistributedShell();

+ 19 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java

@@ -54,6 +54,8 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     AbstractService {
   private static final Log LOG = LogFactory.getLog(AMRMClient.class);
 
+  private TimelineClient timelineClient;
+
   /**
    * Create a new instance of AMRMClient.
    * For usage:
@@ -680,11 +682,27 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     return nmTokenCache;
   }
 
+  /**
+   * Register TimelineClient to AMRMClient.
+   * @param client the timeline client to register
+   */
+  public void registerTimelineClient(TimelineClient client) {
+    this.timelineClient = client;
+  }
+
+  /**
+   * Get registered timeline client.
+   * @return the registered timeline client
+   */
+  public TimelineClient getRegisteredTimeineClient() {
+    return this.timelineClient;
+  }
+
   /**
    * Wait for <code>check</code> to return true for each 1000 ms.
    * See also {@link #waitFor(com.google.common.base.Supplier, int)}
    * and {@link #waitFor(com.google.common.base.Supplier, int, int)}
-   * @param check
+   * @param check the condition for which it should wait
    */
   public void waitFor(Supplier<Boolean> check) throws InterruptedException {
     waitFor(check, 1000);

+ 18 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -342,6 +343,22 @@ extends AbstractService {
    */
   public abstract int getClusterNodeCount();
 
+  /**
+   * Register TimelineClient to AMRMClient.
+   * @param timelineClient
+   */
+  public void registerTimelineClient(TimelineClient timelineClient) {
+    client.registerTimelineClient(timelineClient);
+  }
+
+  /**
+   * Get registered timeline client.
+   * @return the registered timeline client
+   */
+  public TimelineClient getRegisteredTimeineClient() {
+    return client.getRegisteredTimeineClient();
+  }
+
   /**
    * Update application's blacklist with addition or removal resources.
    *
@@ -357,7 +374,7 @@ extends AbstractService {
    * Wait for <code>check</code> to return true for each 1000 ms.
    * See also {@link #waitFor(com.google.common.base.Supplier, int)}
    * and {@link #waitFor(com.google.common.base.Supplier, int, int)}
-   * @param check
+   * @param check the condition for which it should wait
    */
   public void waitFor(Supplier<Boolean> check) throws InterruptedException {
     waitFor(check, 1000);

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
@@ -67,6 +68,10 @@ extends AMRMClientAsync<T> {
   
   private volatile boolean keepRunning;
   private volatile float progress;
+  
+  private volatile String collectorAddr;
+
+  private volatile Throwable savedException;
 
   /**
    *
@@ -318,6 +323,19 @@ extends AMRMClientAsync<T> {
           }
 
           AllocateResponse response = (AllocateResponse) object;
+
+          String collectorAddress = response.getCollectorAddr();
+          TimelineClient timelineClient = client.getRegisteredTimeineClient();
+          if (timelineClient != null && collectorAddress != null
+              && !collectorAddress.isEmpty()) {
+            if (collectorAddr == null
+                || !collectorAddr.equals(collectorAddress)) {
+              collectorAddr = collectorAddress;
+              timelineClient.setTimelineServiceAddress(collectorAddress);
+              LOG.info("collectorAddress " + collectorAddress);
+            }
+          }
+
           List<NodeReport> updatedNodes = response.getUpdatedNodes();
           if (!updatedNodes.isEmpty()) {
             handler.onNodesUpdated(updatedNodes);

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml

@@ -215,6 +215,7 @@
             <exclude>src/main/resources/webapps/yarn/.keep</exclude>
             <exclude>src/main/resources/webapps/applicationhistory/.keep</exclude>
             <exclude>src/main/resources/webapps/sharedcache/.keep</exclude>
+            <exclude>src/main/resources/webapps/timeline/.keep</exclude>
             <exclude>src/main/resources/webapps/cluster/.keep</exclude>
             <exclude>src/main/resources/webapps/test/.keep</exclude>
             <exclude>src/main/resources/webapps/proxy/.keep</exclude>

+ 17 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java

@@ -408,6 +408,23 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     this.amrmToken = amRMToken;
   }
 
+
+  @Override
+  public synchronized String getCollectorAddr() {
+    AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getCollectorAddr();
+  }
+
+  @Override
+  public synchronized void setCollectorAddr(String collectorAddr) {
+    maybeInitBuilder();
+    if (collectorAddr == null) {
+      builder.clearCollectorAddr();
+      return;
+    }
+    builder.setCollectorAddr(collectorAddr);
+  }
+
   @Override
   public synchronized Priority getApplicationPriority() {
     AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;

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

@@ -28,8 +28,9 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
@@ -51,8 +52,13 @@ public abstract class TimelineClient extends AbstractService implements
    * current user may use {@link UserGroupInformation#doAs} another user to
    * construct and initialize a timeline client if the following operations are
    * supposed to be conducted by that user.
+   */
+  private ApplicationId contextAppId;
+
+  /**
+   * Creates an instance of the timeline v.1.x client.
    *
-   * @return a timeline client
+   * @return the created timeline client instance
    */
   @Public
   public static TimelineClient createTimelineClient() {
@@ -60,9 +66,23 @@ public abstract class TimelineClient extends AbstractService implements
     return client;
   }
 
+  /**
+   * Creates an instance of the timeline v.2 client.
+   *
+   * @param appId the application id with which the timeline client is
+   * associated
+   * @return the created timeline client instance
+   */
+  @Public
+  public static TimelineClient createTimelineClient(ApplicationId appId) {
+    TimelineClient client = new TimelineClientImpl(appId);
+    return client;
+  }
+
   @Private
-  protected TimelineClient(String name) {
+  protected TimelineClient(String name, ApplicationId appId) {
     super(name);
+    setContextAppId(appId);
   }
 
   /**
@@ -75,8 +95,8 @@ public abstract class TimelineClient extends AbstractService implements
    * @param entities
    *          the collection of {@link TimelineEntity}
    * @return the error information if the sent entities are not correctly stored
-   * @throws IOException
-   * @throws YarnException
+   * @throws IOException if there are I/O errors
+   * @throws YarnException if entities are incomplete/invalid
    */
   @Public
   public abstract TimelinePutResponse putEntities(
@@ -96,8 +116,8 @@ public abstract class TimelineClient extends AbstractService implements
    * @param entities
    *          the collection of {@link TimelineEntity}
    * @return the error information if the sent entities are not correctly stored
-   * @throws IOException
-   * @throws YarnException
+   * @throws IOException if there are I/O errors
+   * @throws YarnException if entities are incomplete/invalid
    */
   @Public
   public abstract TimelinePutResponse putEntities(
@@ -187,4 +207,57 @@ public abstract class TimelineClient extends AbstractService implements
   public abstract void cancelDelegationToken(
       Token<TimelineDelegationTokenIdentifier> timelineDT)
           throws IOException, YarnException;
+
+  /**
+   * <p>
+   * Send the information of a number of conceptual entities to the timeline
+   * 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>
+   *
+   * @param entities the collection of {@link
+   * org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
+   * @throws IOException
+   * @throws YarnException
+   */
+  @Public
+  public abstract void putEntities(
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
+          entities) throws IOException, YarnException;
+
+  /**
+   * <p>
+   * Send the information of a number of conceptual entities to the timeline
+   * 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>
+   *
+   * @param entities the collection of {@link
+   * org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
+   * @throws IOException
+   * @throws YarnException
+   */
+  @Public
+  public abstract void putEntitiesAsync(
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
+          entities) throws IOException, YarnException;
+
+  /**
+   * <p>
+   * Update the timeline service address where the request will be sent to.
+   * </p>
+   * @param address
+   *          the timeline service address
+   */
+  public abstract void setTimelineServiceAddress(String address);
+
+  protected ApplicationId getContextAppId() {
+    return contextAppId;
+  }
+
+  protected void setContextAppId(ApplicationId appId) {
+    this.contextAppId = appId;
+  }
 }

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

@@ -31,10 +31,21 @@ import java.net.URL;
 import java.net.URLConnection;
 import java.security.GeneralSecurityException;
 import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import javax.net.ssl.HostnameVerifier;
 import javax.net.ssl.HttpsURLConnection;
 import javax.net.ssl.SSLSocketFactory;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.HelpFormatter;
@@ -44,8 +55,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
@@ -55,6 +66,7 @@ import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthentica
 import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
 import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
@@ -67,6 +79,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.codehaus.jackson.map.ObjectMapper;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -79,6 +92,7 @@ import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.api.client.filter.ClientFilter;
 import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
 import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
 
 @Private
 @Evolving
@@ -87,6 +101,8 @@ public class TimelineClientImpl extends TimelineClient {
   private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);
   private static final String RESOURCE_URI_STR = "/ws/v1/timeline/";
   private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final String RESOURCE_URI_STR_V1 = "/ws/v1/timeline/";
+  private static final String RESOURCE_URI_STR_V2 = "/ws/v2/timeline/";
   private static final Joiner JOINER = Joiner.on("");
   public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
 
@@ -107,7 +123,6 @@ public class TimelineClientImpl extends TimelineClient {
   private ConnectionConfigurator connConfigurator;
   private DelegationTokenAuthenticator authenticator;
   private DelegationTokenAuthenticatedURL.Token token;
-  private URI resURI;
   private UserGroupInformation authUgi;
   private String doAsUser;
   private Configuration configuration;
@@ -115,10 +130,20 @@ public class TimelineClientImpl extends TimelineClient {
   private TimelineWriter timelineWriter;
   private SSLFactory sslFactory;
 
+  private volatile String timelineServiceAddress;
+
+  // Retry parameters for identifying new timeline service
+  // TODO consider to merge with connection retry
+  private int maxServiceRetries;
+  private long serviceRetryInterval;
+  private boolean timelineServiceV2 = false;
+
   @Private
   @VisibleForTesting
   TimelineClientConnectionRetry connectionRetry;
 
+  private TimelineEntityDispatcher entityDispatcher;
+
   // Abstract class for an operation that should be retried by timeline client
   @Private
   @VisibleForTesting
@@ -258,7 +283,12 @@ public class TimelineClientImpl extends TimelineClient {
   }
 
   public TimelineClientImpl() {
-    super(TimelineClientImpl.class.getName());
+    super(TimelineClientImpl.class.getName(), null);
+  }
+
+  public TimelineClientImpl(ApplicationId applicationId) {
+    super(TimelineClientImpl.class.getName(), applicationId);
+    this.timelineServiceV2 = true;
   }
 
   protected void serviceInit(Configuration conf) throws Exception {
@@ -287,31 +317,47 @@ public class TimelineClientImpl extends TimelineClient {
     client = new Client(new URLConnectionClientHandler(
         new TimelineURLConnectionFactory()), cc);
     TimelineJerseyRetryFilter retryFilter = new TimelineJerseyRetryFilter();
-    client.addFilter(retryFilter);
-
-    if (YarnConfiguration.useHttps(conf)) {
-      resURI = URI
-          .create(JOINER.join("https://", conf.get(
-              YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
-              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS),
-              RESOURCE_URI_STR));
+    // TODO need to cleanup filter retry later.
+    if (!timelineServiceV2) {
+      client.addFilter(retryFilter);
+    }
+
+    // old version timeline service need to get address from configuration
+    // while new version need to auto discovery (with retry).
+    if (timelineServiceV2) {
+      maxServiceRetries = conf.getInt(
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
+      serviceRetryInterval = conf.getLong(
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
+      entityDispatcher = new TimelineEntityDispatcher(conf);
     } else {
-      resURI = URI.create(JOINER.join("http://", conf.get(
-          YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS),
-          RESOURCE_URI_STR));
-    }
-    LOG.info("Timeline service address: " + resURI);
-    timelineServiceVersion =
-        conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
+      if (YarnConfiguration.useHttps(conf)) {
+        setTimelineServiceAddress(conf.get(
+            YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS));
+      } else {
+        setTimelineServiceAddress(conf.get(
+            YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS));
+      }
+      timelineServiceVersion =
+          conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
+      LOG.info("Timeline service address: " + getTimelineServiceAddress());
+    }
     super.serviceInit(conf);
   }
 
   @Override
   protected void serviceStart() throws Exception {
-    timelineWriter = createTimelineWriter(
-        configuration, authUgi, client, resURI);
+    if (timelineServiceV2) {
+      entityDispatcher.start();
+    } else {
+      timelineWriter = createTimelineWriter(configuration, authUgi, client,
+          constructResURI(getConfig(), timelineServiceAddress, false));
+    }
   }
 
   protected TimelineWriter createTimelineWriter(Configuration conf,
@@ -333,6 +379,9 @@ public class TimelineClientImpl extends TimelineClient {
     if (this.sslFactory != null) {
       this.sslFactory.destroy();
     }
+    if (timelineServiceV2) {
+      entityDispatcher.stop();
+    }
     super.serviceStop();
   }
 
@@ -349,6 +398,25 @@ public class TimelineClientImpl extends TimelineClient {
     return timelineWriter.putEntities(entities);
   }
 
+  @Override
+  public void putEntities(
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
+          entities) throws IOException, YarnException {
+    if (!timelineServiceV2) {
+      throw new YarnException("v.2 method is invoked on a v.1.x client");
+    }
+    entityDispatcher.dispatchEntities(true, entities);
+  }
+
+  @Override
+  public void putEntitiesAsync(
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
+          entities) throws IOException, YarnException {
+    if (!timelineServiceV2) {
+      throw new YarnException("v.2 method is invoked on a v.1.x client");
+    }
+    entityDispatcher.dispatchEntities(false, entities);
+  }
 
   @Override
   public void putDomain(TimelineDomain domain) throws IOException,
@@ -356,11 +424,110 @@ public class TimelineClientImpl extends TimelineClient {
     timelineWriter.putDomain(domain);
   }
 
+  // Used for new timeline service only
+  @Private
+  protected void putObjects(String path, MultivaluedMap<String, String> params,
+      Object obj) throws IOException, YarnException {
+
+    int retries = verifyRestEndPointAvailable();
+
+    // timelineServiceAddress could be stale, add retry logic here.
+    boolean needRetry = true;
+    while (needRetry) {
+      try {
+        URI uri = constructResURI(getConfig(), timelineServiceAddress, true);
+        putObjects(uri, path, params, obj);
+        needRetry = false;
+      } catch (IOException e) {
+        // handle exception for timelineServiceAddress being updated.
+        checkRetryWithSleep(retries, e);
+        retries--;
+      }
+    }
+  }
+
+  private int verifyRestEndPointAvailable() throws YarnException {
+    // timelineServiceAddress could haven't be initialized yet
+    // or stale (only for new timeline service)
+    int retries = pollTimelineServiceAddress(this.maxServiceRetries);
+    if (timelineServiceAddress == null) {
+      String errMessage = "TimelineClient has reached to max retry times : "
+          + this.maxServiceRetries
+          + ", but failed to fetch timeline service address. Please verify"
+          + " Timeline Auxillary Service is configured in all the NMs";
+      LOG.error(errMessage);
+      throw new YarnException(errMessage);
+    }
+    return retries;
+  }
+
+  /**
+   * Check if reaching to maximum of retries.
+   * @param retries
+   * @param e
+   */
+  private void checkRetryWithSleep(int retries, IOException e)
+      throws YarnException, IOException {
+    if (retries > 0) {
+      try {
+        Thread.sleep(this.serviceRetryInterval);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw new YarnException("Interrupted while retrying to connect to ATS");
+      }
+    } else {
+      StringBuilder msg =
+          new StringBuilder("TimelineClient has reached to max retry times : ");
+      msg.append(this.maxServiceRetries);
+      msg.append(" for service address: ");
+      msg.append(timelineServiceAddress);
+      LOG.error(msg.toString());
+      throw new IOException(msg.toString(), e);
+    }
+  }
+
+  protected void putObjects(
+      URI base, String path, MultivaluedMap<String, String> params, Object obj)
+          throws IOException, YarnException {
+    ClientResponse resp;
+    try {
+      resp = client.resource(base).path(path).queryParams(params)
+          .accept(MediaType.APPLICATION_JSON)
+          .type(MediaType.APPLICATION_JSON)
+          .put(ClientResponse.class, obj);
+    } catch (RuntimeException re) {
+      // runtime exception is expected if the client cannot connect the server
+      String msg =
+          "Failed to get the response from the timeline server.";
+      LOG.error(msg, re);
+      throw new IOException(re);
+    }
+    if (resp == null ||
+        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
+      String msg = "Response from the timeline server is " +
+          ((resp == null) ? "null":
+          "not successful," + " HTTP error code: " + resp.getStatus()
+          + ", Server response:\n" + resp.getEntity(String.class));
+      LOG.error(msg);
+      throw new YarnException(msg);
+    }
+  }
+
+  @Override
+  public void setTimelineServiceAddress(String address) {
+    this.timelineServiceAddress = address;
+  }
+
+  private String getTimelineServiceAddress() {
+    return this.timelineServiceAddress;
+  }
+
   @SuppressWarnings("unchecked")
   @Override
   public Token<TimelineDelegationTokenIdentifier> getDelegationToken(
       final String renewer) throws IOException, YarnException {
-    PrivilegedExceptionAction<Token<TimelineDelegationTokenIdentifier>> getDTAction =
+    PrivilegedExceptionAction<Token<TimelineDelegationTokenIdentifier>>
+        getDTAction =
         new PrivilegedExceptionAction<Token<TimelineDelegationTokenIdentifier>>() {
 
           @Override
@@ -369,8 +536,12 @@ public class TimelineClientImpl extends TimelineClient {
             DelegationTokenAuthenticatedURL authUrl =
                 new DelegationTokenAuthenticatedURL(authenticator,
                     connConfigurator);
+            // TODO we should add retry logic here if timelineServiceAddress is
+            // not available immediately.
             return (Token) authUrl.getDelegationToken(
-                resURI.toURL(), token, renewer, doAsUser);
+                constructResURI(getConfig(),
+                    getTimelineServiceAddress(), false).toURL(),
+                token, renewer, doAsUser);
           }
         };
     return (Token<TimelineDelegationTokenIdentifier>) operateDelegationToken(getDTAction);
@@ -393,8 +564,8 @@ public class TimelineClientImpl extends TimelineClient {
           @Override
           public Long run() throws Exception {
             // If the timeline DT to renew is different than cached, replace it.
-            // Token to set every time for retry, because when exception happens,
-            // DelegationTokenAuthenticatedURL will reset it to null;
+            // Token to set every time for retry, because when exception
+            // happens, DelegationTokenAuthenticatedURL will reset it to null;
             if (!timelineDT.equals(token.getDelegationToken())) {
               token.setDelegationToken((Token) timelineDT);
             }
@@ -403,9 +574,10 @@ public class TimelineClientImpl extends TimelineClient {
                     connConfigurator);
             // If the token service address is not available, fall back to use
             // the configured service address.
-            final URI serviceURI = isTokenServiceAddrEmpty ? resURI
+            final URI serviceURI = isTokenServiceAddrEmpty ?
+                constructResURI(getConfig(), getTimelineServiceAddress(), false)
                 : new URI(scheme, null, address.getHostName(),
-                address.getPort(), RESOURCE_URI_STR, null, null);
+                address.getPort(), RESOURCE_URI_STR_V1, null, null);
             return authUrl
                 .renewDelegationToken(serviceURI.toURL(), token, doAsUser);
           }
@@ -429,9 +601,10 @@ public class TimelineClientImpl extends TimelineClient {
 
           @Override
           public Void run() throws Exception {
-            // If the timeline DT to cancel is different than cached, replace it.
-            // Token to set every time for retry, because when exception happens,
-            // DelegationTokenAuthenticatedURL will reset it to null;
+            // If the timeline DT to cancel is different than cached, replace
+            // it.
+            // Token to set every time for retry, because when exception
+            // happens, DelegationTokenAuthenticatedURL will reset it to null;
             if (!timelineDT.equals(token.getDelegationToken())) {
               token.setDelegationToken((Token) timelineDT);
             }
@@ -440,9 +613,10 @@ public class TimelineClientImpl extends TimelineClient {
                     connConfigurator);
             // If the token service address is not available, fall back to use
             // the configured service address.
-            final URI serviceURI = isTokenServiceAddrEmpty ? resURI
+            final URI serviceURI = isTokenServiceAddrEmpty ?
+                constructResURI(getConfig(), getTimelineServiceAddress(), false)
                 : new URI(scheme, null, address.getHostName(),
-                address.getPort(), RESOURCE_URI_STR, null, null);
+                address.getPort(), RESOURCE_URI_STR_V1, null, null);
             authUrl.cancelDelegationToken(serviceURI.toURL(), token, doAsUser);
             return null;
           }
@@ -452,7 +626,8 @@ public class TimelineClientImpl extends TimelineClient {
 
   @Override
   public String toString() {
-    return super.toString() + " with timeline server " + resURI
+    return super.toString() + " with timeline server "
+        + constructResURI(getConfig(), getTimelineServiceAddress(), false)
         + " and writer " + timelineWriter;
   }
 
@@ -466,6 +641,26 @@ public class TimelineClientImpl extends TimelineClient {
     return connectionRetry.retryOn(tokenRetryOp);
   }
 
+  /**
+   * Poll TimelineServiceAddress for maximum of retries times if it is null.
+   *
+   * @param retries
+   * @return the left retry times
+   * @throws IOException
+   */
+  private int pollTimelineServiceAddress(int retries) throws YarnException {
+    while (timelineServiceAddress == null && retries > 0) {
+      try {
+        Thread.sleep(this.serviceRetryInterval);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new YarnException("Interrupted while trying to connect ATS");
+      }
+      retries--;
+    }
+    return retries;
+  }
+
   private class TimelineURLConnectionFactory
       implements HttpURLConnectionFactory {
 
@@ -535,6 +730,13 @@ public class TimelineClientImpl extends TimelineClient {
     connection.setReadTimeout(socketTimeout);
   }
 
+  private static URI constructResURI(
+      Configuration conf, String address, boolean v2) {
+    return URI.create(
+        JOINER.join(YarnConfiguration.useHttps(conf) ? "https://" : "http://",
+            address, v2 ? RESOURCE_URI_STR_V2 : RESOURCE_URI_STR_V1));
+  }
+
   public static void main(String[] argv) throws Exception {
     CommandLine cliParser = new GnuParser().parse(opts, argv);
     if (cliParser.hasOption("put")) {
@@ -716,4 +918,220 @@ public class TimelineClientImpl extends TimelineClient {
     }
   }
 
+  private final class EntitiesHolder extends FutureTask<Void> {
+    private final
+        org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
+            entities;
+    private final boolean isSync;
+
+    EntitiesHolder(
+        final
+            org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
+                entities,
+        final boolean isSync) {
+      super(new Callable<Void>() {
+        // publishEntities()
+        public Void call() throws Exception {
+          MultivaluedMap<String, String> params = new MultivaluedMapImpl();
+          params.add("appid", getContextAppId().toString());
+          params.add("async", Boolean.toString(!isSync));
+          putObjects("entities", params, entities);
+          return null;
+        }
+      });
+      this.entities = entities;
+      this.isSync = isSync;
+    }
+
+    public boolean isSync() {
+      return isSync;
+    }
+
+    public org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
+        getEntities() {
+      return entities;
+    }
+  }
+
+  /**
+   * This class is responsible for collecting the timeline entities and
+   * publishing them in async.
+   */
+  private class TimelineEntityDispatcher {
+    /**
+     * Time period for which the timelineclient will wait for draining after
+     * stop.
+     */
+    private static final long DRAIN_TIME_PERIOD = 2000L;
+
+    private int numberOfAsyncsToMerge;
+    private final BlockingQueue<EntitiesHolder> timelineEntityQueue;
+    private ExecutorService executor;
+
+    TimelineEntityDispatcher(Configuration conf) {
+      timelineEntityQueue = new LinkedBlockingQueue<EntitiesHolder>();
+      numberOfAsyncsToMerge =
+          conf.getInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE,
+              YarnConfiguration.DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE);
+    }
+
+    Runnable createRunnable() {
+      return new Runnable() {
+        @Override
+        public void run() {
+          try {
+            EntitiesHolder entitiesHolder;
+            while (!Thread.currentThread().isInterrupted()) {
+              // Merge all the async calls and make one push, but if its sync
+              // call push immediately
+              try {
+                entitiesHolder = timelineEntityQueue.take();
+              } catch (InterruptedException ie) {
+                LOG.info("Timeline dispatcher thread was interrupted ");
+                Thread.currentThread().interrupt();
+                return;
+              }
+              if (entitiesHolder != null) {
+                publishWithoutBlockingOnQueue(entitiesHolder);
+              }
+            }
+          } finally {
+            if (!timelineEntityQueue.isEmpty()) {
+              LOG.info("Yet to publish " + timelineEntityQueue.size()
+                  + " timelineEntities, draining them now. ");
+            }
+            // Try to drain the remaining entities to be published @ the max for
+            // 2 seconds
+            long timeTillweDrain =
+                System.currentTimeMillis() + DRAIN_TIME_PERIOD;
+            while (!timelineEntityQueue.isEmpty()) {
+              publishWithoutBlockingOnQueue(timelineEntityQueue.poll());
+              if (System.currentTimeMillis() > timeTillweDrain) {
+                // time elapsed stop publishing further....
+                if (!timelineEntityQueue.isEmpty()) {
+                  LOG.warn("Time to drain elapsed! Remaining "
+                      + timelineEntityQueue.size() + "timelineEntities will not"
+                      + " be published");
+                  // if some entities were not drained then we need interrupt
+                  // the threads which had put sync EntityHolders to the queue.
+                  EntitiesHolder nextEntityInTheQueue = null;
+                  while ((nextEntityInTheQueue =
+                      timelineEntityQueue.poll()) != null) {
+                    nextEntityInTheQueue.cancel(true);
+                  }
+                }
+                break;
+              }
+            }
+          }
+        }
+
+        /**
+         * Publishes the given EntitiesHolder and return immediately if sync
+         * call, else tries to fetch the EntitiesHolder from the queue in non
+         * blocking fashion and collate the Entities if possible before
+         * publishing through REST.
+         *
+         * @param entitiesHolder
+         */
+        private void publishWithoutBlockingOnQueue(
+            EntitiesHolder entitiesHolder) {
+          if (entitiesHolder.isSync()) {
+            entitiesHolder.run();
+            return;
+          }
+          int count = 1;
+          while (true) {
+            // loop till we find a sync put Entities or there is nothing
+            // to take
+            EntitiesHolder nextEntityInTheQueue = timelineEntityQueue.poll();
+            if (nextEntityInTheQueue == null) {
+              // Nothing in the queue just publish and get back to the
+              // blocked wait state
+              entitiesHolder.run();
+              break;
+            } else if (nextEntityInTheQueue.isSync()) {
+              // flush all the prev async entities first
+              entitiesHolder.run();
+              // and then flush the sync entity
+              nextEntityInTheQueue.run();
+              break;
+            } else {
+              // append all async entities together and then flush
+              entitiesHolder.getEntities().addEntities(
+                  nextEntityInTheQueue.getEntities().getEntities());
+              count++;
+              if (count == numberOfAsyncsToMerge) {
+                // Flush the entities if the number of the async
+                // putEntites merged reaches the desired limit. To avoid
+                // collecting multiple entities and delaying for a long
+                // time.
+                entitiesHolder.run();
+                break;
+              }
+            }
+          }
+        }
+      };
+    }
+
+    public void dispatchEntities(boolean sync,
+        org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity[]
+            entitiesTobePublished) throws YarnException {
+      if (executor.isShutdown()) {
+        throw new YarnException("Timeline client is in the process of stopping,"
+            + " not accepting any more TimelineEntities");
+      }
+
+      // wrap all TimelineEntity into TimelineEntities object
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
+          entities =
+              new org.apache.hadoop.yarn.api.records.timelineservice.
+                  TimelineEntities();
+      for (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+               entity : entitiesTobePublished) {
+        entities.addEntity(entity);
+      }
+
+      // created a holder and place it in queue
+      EntitiesHolder entitiesHolder = new EntitiesHolder(entities, sync);
+      try {
+        timelineEntityQueue.put(entitiesHolder);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new YarnException(
+            "Failed while adding entity to the queue for publishing", e);
+      }
+
+      if (sync) {
+        // In sync call we need to wait till its published and if any error then
+        // throw it back
+        try {
+          entitiesHolder.get();
+        } catch (ExecutionException e) {
+          throw new YarnException("Failed while publishing entity",
+              e.getCause());
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new YarnException("Interrupted while publishing entity", e);
+        }
+      }
+    }
+
+    public void start() {
+      executor = Executors.newSingleThreadExecutor();
+      executor.execute(createRunnable());
+    }
+
+    public void stop() {
+      LOG.info("Stopping TimelineClient.");
+      executor.shutdownNow();
+      try {
+        executor.awaitTermination(DRAIN_TIME_PERIOD, TimeUnit.MILLISECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        e.printStackTrace();
+      }
+    }
+  }
 }

+ 49 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
@@ -43,6 +44,13 @@ import org.codehaus.jackson.map.ObjectMapper;
 @Evolving
 public class TimelineUtils {
 
+  public static final String FLOW_NAME_TAG_PREFIX = "TIMELINE_FLOW_NAME_TAG";
+  public static final String FLOW_VERSION_TAG_PREFIX =
+      "TIMELINE_FLOW_VERSION_TAG";
+  public static final String FLOW_RUN_ID_TAG_PREFIX =
+      "TIMELINE_FLOW_RUN_ID_TAG";
+  public final static String DEFAULT_FLOW_VERSION = "1";
+
   private static ObjectMapper mapper;
 
   static {
@@ -154,4 +162,45 @@ public class TimelineUtils {
         getTimelineTokenServiceAddress(conf);
     return SecurityUtil.buildTokenService(timelineServiceAddr);
   }
+
+  public static String generateDefaultFlowName(String appName,
+      ApplicationId appId) {
+    return (appName != null &&
+        !appName.equals(YarnConfiguration.DEFAULT_APPLICATION_NAME)) ?
+        appName :
+        "flow_" + appId.getClusterTimestamp() + "_" + appId.getId();
+  }
+
+  /**
+   * Generate flow name tag.
+   *
+   * @param flowName flow name that identifies a distinct flow application which
+   *                 can be run repeatedly over time
+   * @return flow name tag.
+   */
+  public static String generateFlowNameTag(String flowName) {
+    return FLOW_NAME_TAG_PREFIX + ":" + flowName;
+  }
+
+  /**
+   * Generate flow version tag.
+   *
+   * @param flowVersion flow version that keeps track of the changes made to the
+   *                    flow
+   * @return flow version tag.
+   */
+  public static String generateFlowVersionTag(String flowVersion) {
+    return FLOW_VERSION_TAG_PREFIX + ":" + flowVersion;
+  }
+
+  /**
+   * Generate flow run ID tag.
+   *
+   * @param flowRunId flow run ID that identifies one instance (or specific
+   *                  execution) of that flow
+   * @return flow run id tag.
+   */
+  public static String generateFlowRunIdTag(long flowRunId) {
+    return FLOW_RUN_ID_TAG_PREFIX + ":" + flowRunId;
+  }
 }

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java

@@ -252,7 +252,7 @@ public class WebAppUtils {
     return getResolvedAddress(address);
   }
 
-  private static String getResolvedAddress(InetSocketAddress address) {
+  public static String getResolvedAddress(InetSocketAddress address) {
     address = NetUtils.getConnectAddress(address);
     StringBuilder sb = new StringBuilder();
     InetAddress resolved = address.getAddress();
@@ -314,6 +314,10 @@ public class WebAppUtils {
   }
 
   public static String getAHSWebAppURLWithoutScheme(Configuration conf) {
+    return getTimelineReaderWebAppURL(conf);
+  }
+
+  public static String getTimelineReaderWebAppURL(Configuration conf) {
     if (YarnConfiguration.useHttps(conf)) {
       return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
         YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
@@ -322,7 +326,7 @@ public class WebAppUtils {
         YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
     }
   }
-  
+
   /**
    * if url has scheme then it will be returned as it is else it will return
    * url with scheme.

+ 0 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/timeline/.keep


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

@@ -790,11 +790,29 @@
 
   <property>
     <description>The setting that controls whether yarn system metrics is
-    published on the timeline server or not by RM.</description>
+    published to the Timeline server (version one) or not, by RM.
+    This configuration is now deprecated in favor of
+    yarn.system-metrics-publisher.enabled.</description>
     <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
     <value>false</value>
   </property>
 
+  <property>
+    <description>The setting that controls whether yarn system metrics is
+    published on the Timeline service or not by RM And NM.</description>
+    <name>yarn.system-metrics-publisher.enabled</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>The setting that controls whether yarn container events are
+    published to the timeline service or not by RM. This configuration setting
+    is for ATS V2.</description>
+    <name>yarn.rm.system-metrics-publisher.emit-container-events</name>
+    <value>false</value>
+  </property>
+
+
   <property>
     <description>Number of worker threads that send the yarn system metrics
     data.</description>
@@ -978,6 +996,12 @@
     <value>20</value>
   </property>
 
+    <property>
+    <description>Number of threads collector service uses.</description>
+    <name>yarn.nodemanager.collector-service.thread-count</name>
+    <value>5</value>
+  </property>
+
   <property>
     <description>Number of threads used in cleanup.</description>
     <name>yarn.nodemanager.delete.thread-count</name>
@@ -1053,6 +1077,13 @@
     <value>${yarn.nodemanager.hostname}:8040</value>
   </property>
 
+
+  <property>
+    <description>Address where the collector service IPC is.</description>
+    <name>yarn.nodemanager.collector-service.address</name>
+    <value>${yarn.nodemanager.hostname}:8048</value>
+  </property>
+
   <property>
     <description>Interval in between cache cleanups.</description>
     <name>yarn.nodemanager.localizer.cache.cleanup.interval-ms</name>
@@ -1937,7 +1968,7 @@
     <description>Indicate what is the current version of the running
     timeline service. For example, if "yarn.timeline-service.version" is 1.5,
     and "yarn.timeline-service.enabled" is true, it means the cluster will and
-    should bring up the timeline service v.1.5.
+    should bring up the timeline service v.1.5 (and nothing else).
     On the client side, if the client uses the same version of timeline service,
     it should succeed. If the client chooses to use a smaller version in spite of this,
     then depending on how robust the compatibility story is between versions,
@@ -2261,6 +2292,23 @@
     <value>300</value>
   </property>
 
+  <!-- Timeline Service v2 Configuration -->
+  <property>
+    <name>yarn.timeline-service.writer.class</name>
+    <description>
+      Storage implementation ATS v2 will use for the TimelineWriter service.
+    </description>
+    <value>org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl</value>
+  </property>
+
+  <property>
+    <name>yarn.timeline-service.reader.class</name>
+    <description>
+      Storage implementation ATS v2 will use for the TimelineReader service.
+    </description>
+    <value>org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl</value>
+  </property>
+
   <property>
     <name>yarn.timeline-service.client.internal-timers-ttl-secs</name>
     <description>
@@ -2271,6 +2319,41 @@
     <value>420</value>
   </property>
 
+  <property>
+    <description>The setting that controls how often the timeline collector
+    flushes the timeline writer.</description>
+    <name>yarn.timeline-service.writer.flush-interval-seconds</name>
+    <value>60</value>
+  </property>
+
+  <property>
+    <description>Time period till which the application collector will be alive
+     in NM, after the  application master container finishes.</description>
+    <name>yarn.timeline-service.app-collector.linger-period.ms</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>Time line V2 client tries to merge these many number of
+    async entities (if available) and then call the REST ATS V2 API to submit.
+    </description>
+    <name>yarn.timeline-service.timeline-client.number-of-async-entities-to-merge</name>
+    <value>10</value>
+  </property>
+
+  <property>
+    <description>
+    The setting that controls how long the final value
+    of a metric of a completed app is retained before merging into
+    the flow sum. Up to this time after an application is completed
+    out-of-order values that arrive can be recognized and discarded at the
+    cost of increased storage.
+    </description>
+    <name>yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds
+    </name>
+    <value>259200000</value>
+  </property>
+
   <!--  Shared Cache Configuration -->
 
   <property>

+ 15 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java

@@ -29,6 +29,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
@@ -118,7 +119,7 @@ public class TestContainerLaunchRPC {
             resource, System.currentTimeMillis() + 10000, 42, 42,
             Priority.newInstance(0), 0);
       Token containerToken =
-          TestRPC.newContainerToken(nodeId, "password".getBytes(),
+          newContainerToken(nodeId, "password".getBytes(),
             containerTokenIdentifier);
 
       StartContainerRequest scRequest =
@@ -144,6 +145,19 @@ public class TestContainerLaunchRPC {
     Assert.fail("timeout exception should have occurred!");
   }
 
+  public static Token newContainerToken(NodeId nodeId, byte[] password,
+      ContainerTokenIdentifier tokenIdentifier) {
+    // RPC layer client expects ip:port as service for tokens
+    InetSocketAddress addr =
+        NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort());
+    // NOTE: use SecurityUtil.setTokenService if this becomes a "real" token
+    Token containerToken =
+        Token.newInstance(tokenIdentifier.getBytes(),
+          ContainerTokenIdentifier.KIND.toString(), password, SecurityUtil
+            .buildTokenService(addr).toString());
+    return containerToken;
+  }
+
   public class DummyContainerManager implements ContainerManagementProtocol {
 
     private ContainerStatus status = null;

+ 15 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java

@@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
@@ -107,7 +108,7 @@ public class TestContainerResourceIncreaseRPC {
               resource, System.currentTimeMillis() + 10000, 42, 42,
                   Priority.newInstance(0), 0);
       Token containerToken =
-          TestRPC.newContainerToken(nodeId, "password".getBytes(),
+          newContainerToken(nodeId, "password".getBytes(),
               containerTokenIdentifier);
       // Construct container resource increase request,
       List<Token> increaseTokens = new ArrayList<>();
@@ -130,6 +131,19 @@ public class TestContainerResourceIncreaseRPC {
     Assert.fail("timeout exception should have occurred!");
   }
 
+  public static Token newContainerToken(NodeId nodeId, byte[] password,
+      ContainerTokenIdentifier tokenIdentifier) {
+    // RPC layer client expects ip:port as service for tokens
+    InetSocketAddress addr =
+        NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort());
+    // NOTE: use SecurityUtil.setTokenService if this becomes a "real" token
+    Token containerToken =
+        Token.newInstance(tokenIdentifier.getBytes(),
+          ContainerTokenIdentifier.KIND.toString(), password, SecurityUtil
+            .buildTokenService(addr).toString());
+    return containerToken;
+  }
+
   public class DummyContainerManager implements ContainerManagementProtocol {
 
     @Override

+ 312 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java

@@ -0,0 +1,312 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
+import org.junit.Test;
+import org.junit.Assert;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+
+public class TestTimelineServiceRecords {
+  private static final Log LOG =
+      LogFactory.getLog(TestTimelineServiceRecords.class);
+
+  @Test
+  public void testTimelineEntities() throws Exception {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setType("test type 1");
+    entity.setId("test id 1");
+    entity.addInfo("test info key 1", "test info value 1");
+    entity.addInfo("test info key 2",
+        Arrays.asList("test info value 2", "test info value 3"));
+    entity.addInfo("test info key 3", true);
+    Assert.assertTrue(
+        entity.getInfo().get("test info key 3") instanceof Boolean);
+    entity.addConfig("test config key 1", "test config value 1");
+    entity.addConfig("test config key 2", "test config value 2");
+
+    TimelineMetric metric1 =
+        new TimelineMetric(TimelineMetric.Type.TIME_SERIES);
+    metric1.setId("test metric id 1");
+    metric1.addValue(1L, 1.0F);
+    metric1.addValue(3L, 3.0D);
+    metric1.addValue(2L, 2);
+    Assert.assertEquals(TimelineMetric.Type.TIME_SERIES, metric1.getType());
+    Iterator<Map.Entry<Long, Number>> itr =
+        metric1.getValues().entrySet().iterator();
+    Map.Entry<Long, Number> entry = itr.next();
+    Assert.assertEquals(new Long(3L), entry.getKey());
+    Assert.assertEquals(3.0D, entry.getValue());
+    entry = itr.next();
+    Assert.assertEquals(new Long(2L), entry.getKey());
+    Assert.assertEquals(2, entry.getValue());
+    entry = itr.next();
+    Assert.assertEquals(new Long(1L), entry.getKey());
+    Assert.assertEquals(1.0F, entry.getValue());
+    Assert.assertFalse(itr.hasNext());
+    entity.addMetric(metric1);
+
+    TimelineMetric metric2 =
+        new TimelineMetric(TimelineMetric.Type.SINGLE_VALUE);
+    metric2.setId("test metric id 1");
+    metric2.addValue(3L, (short) 3);
+    Assert.assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric2.getType());
+    Assert.assertTrue(
+        metric2.getValues().values().iterator().next() instanceof Short);
+    Map<Long, Number> points = new HashMap<>();
+    points.put(4L, 4.0D);
+    points.put(5L, 5.0D);
+    try {
+      metric2.setValues(points);
+      Assert.fail();
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "Values cannot contain more than one point in"));
+    }
+    try {
+      metric2.addValues(points);
+      Assert.fail();
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "Values cannot contain more than one point in"));
+    }
+    entity.addMetric(metric2);
+
+    TimelineMetric metric3 =
+        new TimelineMetric(TimelineMetric.Type.SINGLE_VALUE);
+    metric3.setId("test metric id 1");
+    metric3.addValue(4L, (short) 4);
+    Assert.assertEquals("metric3 should equal to metric2! ", metric3, metric2);
+    Assert.assertNotEquals("metric1 should not equal to metric2! ",
+        metric1, metric2);
+
+    TimelineEvent event1 = new TimelineEvent();
+    event1.setId("test event id 1");
+    event1.addInfo("test info key 1", "test info value 1");
+    event1.addInfo("test info key 2",
+        Arrays.asList("test info value 2", "test info value 3"));
+    event1.addInfo("test info key 3", true);
+    Assert.assertTrue(
+        event1.getInfo().get("test info key 3") instanceof Boolean);
+    event1.setTimestamp(1L);
+    entity.addEvent(event1);
+
+    TimelineEvent event2 = new TimelineEvent();
+    event2.setId("test event id 2");
+    event2.addInfo("test info key 1", "test info value 1");
+    event2.addInfo("test info key 2",
+        Arrays.asList("test info value 2", "test info value 3"));
+    event2.addInfo("test info key 3", true);
+    Assert.assertTrue(
+        event2.getInfo().get("test info key 3") instanceof Boolean);
+    event2.setTimestamp(2L);
+    entity.addEvent(event2);
+
+    Assert.assertFalse("event1 should not equal to event2! ",
+        event1.equals(event2));
+    TimelineEvent event3 = new TimelineEvent();
+    event3.setId("test event id 1");
+    event3.setTimestamp(1L);
+    Assert.assertEquals("event1 should equal to event3! ", event3, event1);
+    Assert.assertNotEquals("event1 should not equal to event2! ",
+        event1, event2);
+
+    entity.setCreatedTime(0L);
+    entity.addRelatesToEntity("test type 2", "test id 2");
+    entity.addRelatesToEntity("test type 3", "test id 3");
+    entity.addIsRelatedToEntity("test type 4", "test id 4");
+    entity.addIsRelatedToEntity("test type 5", "test id 5");
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(entity, true));
+
+    TimelineEntities entities = new TimelineEntities();
+    TimelineEntity entity1 = new TimelineEntity();
+    entities.addEntity(entity1);
+    TimelineEntity entity2 = new TimelineEntity();
+    entities.addEntity(entity2);
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(entities, true));
+
+    Assert.assertFalse("entity 1 should not be valid without type and id",
+        entity1.isValid());
+    entity1.setId("test id 2");
+    entity1.setType("test type 2");
+    entity2.setId("test id 1");
+    entity2.setType("test type 1");
+
+    Assert.assertEquals("Timeline entity should equal to entity2! ",
+        entity, entity2);
+    Assert.assertNotEquals("entity1 should not equal to entity! ",
+        entity1, entity);
+    Assert.assertEquals("entity should be less than entity1! ",
+        entity1.compareTo(entity), 1);
+    Assert.assertEquals("entity's hash code should be -28727840 but not "
+        + entity.hashCode(), entity.hashCode(), -28727840);
+  }
+
+  @Test
+  public void testFirstClassCitizenEntities() throws Exception {
+    UserEntity user = new UserEntity();
+    user.setId("test user id");
+
+    QueueEntity queue = new QueueEntity();
+    queue.setId("test queue id");
+
+
+    ClusterEntity cluster = new ClusterEntity();
+    cluster.setId("test cluster id");
+
+    FlowRunEntity flow1 = new FlowRunEntity();
+    //flow1.setId("test flow id 1");
+    flow1.setUser(user.getId());
+    flow1.setName("test flow name 1");
+    flow1.setVersion("test flow version 1");
+    flow1.setRunId(1L);
+
+    FlowRunEntity flow2 = new FlowRunEntity();
+    //flow2.setId("test flow run id 2");
+    flow2.setUser(user.getId());
+    flow2.setName("test flow name 2");
+    flow2.setVersion("test flow version 2");
+    flow2.setRunId(2L);
+
+    ApplicationEntity app1 = new ApplicationEntity();
+    app1.setId(ApplicationId.newInstance(0, 1).toString());
+    app1.setQueue(queue.getId());
+
+    ApplicationEntity app2 = new ApplicationEntity();
+    app2.setId(ApplicationId.newInstance(0, 2).toString());
+    app2.setQueue(queue.getId());
+
+    ApplicationAttemptEntity appAttempt = new ApplicationAttemptEntity();
+    appAttempt.setId(ApplicationAttemptId.newInstance(
+        ApplicationId.newInstance(0, 1), 1).toString());
+
+    ContainerEntity container = new ContainerEntity();
+    container.setId(ContainerId.newContainerId(
+        ApplicationAttemptId.newInstance(
+            ApplicationId.newInstance(0, 1), 1), 1).toString());
+
+    cluster.addChild(TimelineEntityType.YARN_FLOW_RUN.toString(),
+        flow1.getId());
+    flow1
+        .setParent(TimelineEntityType.YARN_CLUSTER.toString(), cluster.getId());
+    flow1.addChild(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId());
+    flow2.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow1.getId());
+    flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(),
+        app1.getId());
+    flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(),
+        app2.getId());
+    app1.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId());
+    app1.addChild(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(),
+        appAttempt.getId());
+    appAttempt
+        .setParent(TimelineEntityType.YARN_APPLICATION.toString(),
+            app1.getId());
+    app2.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId());
+    appAttempt.addChild(TimelineEntityType.YARN_CONTAINER.toString(),
+        container.getId());
+    container.setParent(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(),
+        appAttempt.getId());
+
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(cluster, true));
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(flow1, true));
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(flow2, true));
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(app1, true));
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(app2, true));
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(appAttempt, true));
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(container, true));
+
+
+    // Check parent/children APIs
+    Assert.assertNotNull(app1.getParent());
+    Assert.assertEquals(flow2.getType(), app1.getParent().getType());
+    Assert.assertEquals(flow2.getId(), app1.getParent().getId());
+    app1.addInfo(ApplicationEntity.PARENT_INFO_KEY, "invalid parent object");
+    try {
+      app1.getParent();
+      Assert.fail();
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof YarnRuntimeException);
+      Assert.assertTrue(e.getMessage().contains(
+          "Parent info is invalid identifier object"));
+    }
+
+    Assert.assertNotNull(app1.getChildren());
+    Assert.assertEquals(1, app1.getChildren().size());
+    Assert.assertEquals(
+        appAttempt.getType(), app1.getChildren().iterator().next().getType());
+    Assert.assertEquals(
+        appAttempt.getId(), app1.getChildren().iterator().next().getId());
+    app1.addInfo(ApplicationEntity.CHILDREN_INFO_KEY,
+        Collections.singletonList("invalid children set"));
+    try {
+      app1.getChildren();
+      Assert.fail();
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof YarnRuntimeException);
+      Assert.assertTrue(e.getMessage().contains(
+          "Children info is invalid identifier set"));
+    }
+    app1.addInfo(ApplicationEntity.CHILDREN_INFO_KEY,
+        Collections.singleton("invalid child object"));
+    try {
+      app1.getChildren();
+      Assert.fail();
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof YarnRuntimeException);
+      Assert.assertTrue(e.getMessage().contains(
+          "Children info contains invalid identifier object"));
+    }
+  }
+
+  @Test
+  public void testUser() throws Exception {
+    UserEntity user = new UserEntity();
+    user.setId("test user id");
+    user.addInfo("test info key 1", "test info value 1");
+    user.addInfo("test info key 2", "test info value 2");
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(user, true));
+  }
+
+  @Test
+  public void testQueue() throws Exception {
+    QueueEntity queue = new QueueEntity();
+    queue.setId("test queue id");
+    queue.addInfo("test info key 1", "test info value 1");
+    queue.addInfo("test info key 2", "test info value 2");
+    queue.setParent(TimelineEntityType.YARN_QUEUE.toString(),
+        "test parent queue id");
+    queue.addChild(TimelineEntityType.YARN_QUEUE.toString(),
+        "test child queue id 1");
+    queue.addChild(TimelineEntityType.YARN_QUEUE.toString(),
+        "test child queue id 2");
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(queue, true));
+  }
+}

+ 378 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java

@@ -0,0 +1,378 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client.api.impl;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.ws.rs.core.MultivaluedMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+public class TestTimelineClientV2Impl {
+  private static final Log LOG =
+      LogFactory.getLog(TestTimelineClientV2Impl.class);
+  private TestV2TimelineClient client;
+  private static final long TIME_TO_SLEEP = 150L;
+  private static final String EXCEPTION_MSG = "Exception in the content";
+
+  @Before
+  public void setup() {
+    conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.0f);
+    conf.setInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE, 3);
+    if (!currTestName.getMethodName()
+        .contains("testRetryOnConnectionFailure")) {
+      client = createTimelineClient(conf);
+    }
+  }
+
+  @Rule
+  public TestName currTestName = new TestName();
+  private YarnConfiguration conf;
+
+  private TestV2TimelineClient createTimelineClient(YarnConfiguration config) {
+    ApplicationId id = ApplicationId.newInstance(0, 0);
+    TestV2TimelineClient tc = new TestV2TimelineClient(id);
+    tc.init(config);
+    tc.start();
+    return tc;
+  }
+
+  private class TestV2TimelineClientForExceptionHandling
+      extends TimelineClientImpl {
+    public TestV2TimelineClientForExceptionHandling(ApplicationId id) {
+      super(id);
+    }
+
+    private boolean throwYarnException;
+
+    public void setThrowYarnException(boolean throwYarnException) {
+      this.throwYarnException = throwYarnException;
+    }
+
+    public boolean isThrowYarnException() {
+      return throwYarnException;
+    }
+
+    @Override
+    protected void putObjects(URI base, String path,
+        MultivaluedMap<String, String> params, Object obj)
+            throws IOException, YarnException {
+      if (throwYarnException) {
+        throw new YarnException(EXCEPTION_MSG);
+      } else {
+        throw new IOException(
+            "Failed to get the response from the timeline server.");
+      }
+    }
+  }
+
+  private class TestV2TimelineClient
+      extends TestV2TimelineClientForExceptionHandling {
+    private boolean sleepBeforeReturn;
+
+    private List<TimelineEntities> publishedEntities;
+
+    public TimelineEntities getPublishedEntities(int putIndex) {
+      Assert.assertTrue("Not So many entities Published",
+          putIndex < publishedEntities.size());
+      return publishedEntities.get(putIndex);
+    }
+
+    public void setSleepBeforeReturn(boolean sleepBeforeReturn) {
+      this.sleepBeforeReturn = sleepBeforeReturn;
+    }
+
+    public int getNumOfTimelineEntitiesPublished() {
+      return publishedEntities.size();
+    }
+
+    public TestV2TimelineClient(ApplicationId id) {
+      super(id);
+      publishedEntities = new ArrayList<TimelineEntities>();
+    }
+
+    protected void putObjects(String path,
+        MultivaluedMap<String, String> params, Object obj)
+            throws IOException, YarnException {
+      if (isThrowYarnException()) {
+        throw new YarnException("ActualException");
+      }
+      publishedEntities.add((TimelineEntities) obj);
+      if (sleepBeforeReturn) {
+        try {
+          Thread.sleep(TIME_TO_SLEEP);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testExceptionMultipleRetry() {
+    TestV2TimelineClientForExceptionHandling c =
+        new TestV2TimelineClientForExceptionHandling(
+            ApplicationId.newInstance(0, 0));
+    int maxRetries = 2;
+    conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
+        maxRetries);
+    c.init(conf);
+    c.start();
+    c.setTimelineServiceAddress("localhost:12345");
+    try {
+      c.putEntities(new TimelineEntity());
+    } catch (IOException e) {
+      Assert.fail("YARN exception is expected");
+    } catch (YarnException e) {
+      Throwable cause = e.getCause();
+      Assert.assertTrue("IOException is expected",
+          cause instanceof IOException);
+      Assert.assertTrue("YARN exception is expected",
+          cause.getMessage().contains(
+              "TimelineClient has reached to max retry times : " + maxRetries));
+    }
+
+    c.setThrowYarnException(true);
+    try {
+      c.putEntities(new TimelineEntity());
+    } catch (IOException e) {
+      Assert.fail("YARN exception is expected");
+    } catch (YarnException e) {
+      Throwable cause = e.getCause();
+      Assert.assertTrue("YARN exception is expected",
+          cause instanceof YarnException);
+      Assert.assertTrue("YARN exception is expected",
+          cause.getMessage().contains(EXCEPTION_MSG));
+    }
+    c.stop();
+  }
+
+  @Test
+  public void testPostEntities() throws Exception {
+    try {
+      client.putEntities(generateEntity("1"));
+    } catch (YarnException e) {
+      Assert.fail("Exception is not expected");
+    }
+  }
+
+  @Test
+  public void testASyncCallMerge() throws Exception {
+    client.setSleepBeforeReturn(true);
+    try {
+      client.putEntitiesAsync(generateEntity("1"));
+      Thread.sleep(TIME_TO_SLEEP / 2);
+      // by the time first put response comes push 2 entities in the queue
+      client.putEntitiesAsync(generateEntity("2"));
+      client.putEntitiesAsync(generateEntity("3"));
+    } catch (YarnException e) {
+      Assert.fail("Exception is not expected");
+    }
+    for (int i = 0; i < 4; i++) {
+      if (client.getNumOfTimelineEntitiesPublished() == 2) {
+        break;
+      }
+      Thread.sleep(TIME_TO_SLEEP);
+    }
+    Assert.assertEquals("two merged TimelineEntities needs to be published", 2,
+        client.getNumOfTimelineEntitiesPublished());
+    TimelineEntities secondPublishedEntities = client.getPublishedEntities(1);
+    Assert.assertEquals(
+        "Merged TimelineEntities Object needs to 2 TimelineEntity Object", 2,
+        secondPublishedEntities.getEntities().size());
+    Assert.assertEquals("Order of Async Events Needs to be FIFO", "2",
+        secondPublishedEntities.getEntities().get(0).getId());
+    Assert.assertEquals("Order of Async Events Needs to be FIFO", "3",
+        secondPublishedEntities.getEntities().get(1).getId());
+  }
+
+  @Test
+  public void testSyncCall() throws Exception {
+    try {
+      // sync entity should not be be merged with Async
+      client.putEntities(generateEntity("1"));
+      client.putEntitiesAsync(generateEntity("2"));
+      client.putEntitiesAsync(generateEntity("3"));
+      // except for the sync call above 2 should be merged
+      client.putEntities(generateEntity("4"));
+    } catch (YarnException e) {
+      Assert.fail("Exception is not expected");
+    }
+    for (int i = 0; i < 4; i++) {
+      if (client.getNumOfTimelineEntitiesPublished() == 3) {
+        break;
+      }
+      Thread.sleep(TIME_TO_SLEEP);
+    }
+    printReceivedEntities();
+    Assert.assertEquals("TimelineEntities not published as desired", 3,
+        client.getNumOfTimelineEntitiesPublished());
+    TimelineEntities firstPublishedEntities = client.getPublishedEntities(0);
+    Assert.assertEquals("sync entities should not be merged with async", 1,
+        firstPublishedEntities.getEntities().size());
+
+    // test before pushing the sync entities asyncs are merged and pushed
+    TimelineEntities secondPublishedEntities = client.getPublishedEntities(1);
+    Assert.assertEquals(
+        "async entities should be merged before publishing sync", 2,
+        secondPublishedEntities.getEntities().size());
+    Assert.assertEquals("Order of Async Events Needs to be FIFO", "2",
+        secondPublishedEntities.getEntities().get(0).getId());
+    Assert.assertEquals("Order of Async Events Needs to be FIFO", "3",
+        secondPublishedEntities.getEntities().get(1).getId());
+
+    // test the last entity published is sync put
+    TimelineEntities thirdPublishedEntities = client.getPublishedEntities(2);
+    Assert.assertEquals("sync entities had to be published at the last", 1,
+        thirdPublishedEntities.getEntities().size());
+    Assert.assertEquals("Expected last sync Event is not proper", "4",
+        thirdPublishedEntities.getEntities().get(0).getId());
+  }
+
+  @Test
+  public void testExceptionCalls() throws Exception {
+    client.setThrowYarnException(true);
+    try {
+      client.putEntitiesAsync(generateEntity("1"));
+    } catch (YarnException e) {
+      Assert.fail("Async calls are not expected to throw exception");
+    }
+
+    try {
+      client.putEntities(generateEntity("2"));
+      Assert.fail("Sync calls are expected to throw exception");
+    } catch (YarnException e) {
+      Assert.assertEquals("Same exception needs to be thrown",
+          "ActualException", e.getCause().getMessage());
+    }
+  }
+
+  @Test
+  public void testConfigurableNumberOfMerges() throws Exception {
+    client.setSleepBeforeReturn(true);
+    try {
+      // At max 3 entities need to be merged
+      client.putEntitiesAsync(generateEntity("1"));
+      client.putEntitiesAsync(generateEntity("2"));
+      client.putEntitiesAsync(generateEntity("3"));
+      client.putEntitiesAsync(generateEntity("4"));
+      client.putEntities(generateEntity("5"));
+      client.putEntitiesAsync(generateEntity("6"));
+      client.putEntitiesAsync(generateEntity("7"));
+      client.putEntitiesAsync(generateEntity("8"));
+      client.putEntitiesAsync(generateEntity("9"));
+      client.putEntitiesAsync(generateEntity("10"));
+    } catch (YarnException e) {
+      Assert.fail("No exception expected");
+    }
+    // not having the same logic here as it doesn't depend on how many times
+    // events are published.
+    Thread.sleep(2 * TIME_TO_SLEEP);
+    printReceivedEntities();
+    for (TimelineEntities publishedEntities : client.publishedEntities) {
+      Assert.assertTrue(
+          "Number of entities should not be greater than 3 for each publish,"
+              + " but was " + publishedEntities.getEntities().size(),
+          publishedEntities.getEntities().size() <= 3);
+    }
+  }
+
+  @Test
+  public void testAfterStop() throws Exception {
+    client.setSleepBeforeReturn(true);
+    try {
+      // At max 3 entities need to be merged
+      client.putEntities(generateEntity("1"));
+      for (int i = 2; i < 20; i++) {
+        client.putEntitiesAsync(generateEntity("" + i));
+      }
+      client.stop();
+      try {
+        client.putEntitiesAsync(generateEntity("50"));
+        Assert.fail("Exception expected");
+      } catch (YarnException e) {
+        // expected
+      }
+    } catch (YarnException e) {
+      Assert.fail("No exception expected");
+    }
+    // not having the same logic here as it doesn't depend on how many times
+    // events are published.
+    for (int i = 0; i < 5; i++) {
+      TimelineEntities publishedEntities =
+          client.publishedEntities.get(client.publishedEntities.size() - 1);
+      TimelineEntity timelineEntity = publishedEntities.getEntities()
+          .get(publishedEntities.getEntities().size() - 1);
+      if (!timelineEntity.getId().equals("19")) {
+        Thread.sleep(2 * TIME_TO_SLEEP);
+      }
+    }
+    printReceivedEntities();
+    TimelineEntities publishedEntities =
+        client.publishedEntities.get(client.publishedEntities.size() - 1);
+    TimelineEntity timelineEntity = publishedEntities.getEntities()
+        .get(publishedEntities.getEntities().size() - 1);
+    Assert.assertEquals("", "19", timelineEntity.getId());
+  }
+
+  private void printReceivedEntities() {
+    for (int i = 0; i < client.getNumOfTimelineEntitiesPublished(); i++) {
+      TimelineEntities publishedEntities = client.getPublishedEntities(i);
+      StringBuilder entitiesPerPublish = new StringBuilder();
+      for (TimelineEntity entity : publishedEntities.getEntities()) {
+        entitiesPerPublish.append(entity.getId());
+        entitiesPerPublish.append(",");
+      }
+      LOG.info("Entities Published @ index " + i + " : "
+          + entitiesPerPublish.toString());
+    }
+  }
+
+  private static TimelineEntity generateEntity(String id) {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setId(id);
+    entity.setType("testEntity");
+    entity.setCreatedTime(System.currentTimeMillis());
+    return entity;
+  }
+
+  @After
+  public void tearDown() {
+    if (client != null) {
+      client.stop();
+    }
+  }
+}

+ 83 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java

@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.util;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestTimelineServiceHelper {
+
+  @Test
+  public void testMapCastToHashMap() {
+
+    // Test null map be casted to null
+    Map<String, String> nullMap = null;
+    Assert.assertNull(TimelineServiceHelper.mapCastToHashMap(nullMap));
+
+    // Test empty hashmap be casted to a empty hashmap
+    Map<String, String> emptyHashMap = new HashMap<String, String>();
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(emptyHashMap).size(), 0);
+
+    // Test empty non-hashmap be casted to a empty hashmap
+    Map<String, String> emptyTreeMap = new TreeMap<String, String>();
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(emptyTreeMap).size(), 0);
+
+    // Test non-empty hashmap be casted to hashmap correctly
+    Map<String, String> firstHashMap = new HashMap<String, String>();
+    String key = "KEY";
+    String value = "VALUE";
+    firstHashMap.put(key, value);
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(firstHashMap), firstHashMap);
+
+    // Test non-empty non-hashmap is casted correctly.
+    Map<String, String> firstTreeMap = new TreeMap<String, String>();
+    firstTreeMap.put(key, value);
+    HashMap<String, String> alternateHashMap =
+        TimelineServiceHelper.mapCastToHashMap(firstTreeMap);
+    Assert.assertEquals(firstTreeMap.size(), alternateHashMap.size());
+    Assert.assertEquals(alternateHashMap.get(key), value);
+
+    // Test complicated hashmap be casted correctly
+    Map<String, Set<String>> complicatedHashMap =
+        new HashMap<String, Set<String>>();
+    Set<String> hashSet = new HashSet<String>();
+    hashSet.add(value);
+    complicatedHashMap.put(key, hashSet);
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(complicatedHashMap),
+        complicatedHashMap);
+
+    // Test complicated non-hashmap get casted correctly
+    Map<String, Set<String>> complicatedTreeMap =
+        new TreeMap<String, Set<String>>();
+    complicatedTreeMap.put(key, hashSet);
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(complicatedTreeMap).get(key),
+        hashSet);
+  }
+
+}

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml

@@ -190,6 +190,7 @@
                   <include>yarn_server_federation_protos.proto</include>
                   <include>ResourceTracker.proto</include>
                   <include>SCMUploader.proto</include>
+                  <include>collectornodemanager_protocol.proto</include>
                 </includes>
               </source>
             </configuration>

+ 73 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocol.java

@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+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.ReportNewCollectorInfoRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
+
+/**
+ * <p>The protocol between an <code>TimelineCollectorManager</code> and a
+ * <code>NodeManager</code> to report a new application collector get launched.
+ * </p>
+ *
+ */
+@Private
+public interface CollectorNodemanagerProtocol {
+
+  /**
+   *
+   * <p>
+   * The <code>TimelineCollectorManager</code> provides a list of mapping
+   * between application and collector's address in
+   * {@link ReportNewCollectorInfoRequest} to a <code>NodeManager</code> to
+   * <em>register</em> collector's info, include: applicationId and REST URI to
+   * access collector. NodeManager will add them into registered collectors
+   * and register them into <code>ResourceManager</code> afterwards.
+   * </p>
+   *
+   * @param request the request of registering a new collector or a list of
+   *                collectors
+   * @return the response for registering the new collector
+   * @throws YarnException if the request is invalid
+   * @throws IOException if there are I/O errors
+   */
+  ReportNewCollectorInfoResponse reportNewCollectorInfo(
+      ReportNewCollectorInfoRequest request)
+      throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The collector needs to get the context information including user, flow
+   * and flow run ID to associate with every incoming put-entity requests.
+   * </p>
+   * @param request the request of getting the aggregator context information of
+   *                the given application
+   * @return the response for registering the new collector
+   * @throws YarnException if the request is invalid
+   * @throws IOException if there are I/O errors
+   */
+  GetTimelineCollectorContextResponse getTimelineCollectorContext(
+      GetTimelineCollectorContextRequest request)
+      throws YarnException, IOException;
+}

+ 34 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocolPB.java

@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.api;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.yarn.proto.CollectorNodemanagerProtocol.CollectorNodemanagerProtocolService;
+
+@Private
+@Unstable
+@ProtocolInfo(
+    protocolName =
+        "org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB",
+    protocolVersion = 1)
+public interface CollectorNodemanagerProtocolPB extends
+    CollectorNodemanagerProtocolService.BlockingInterface {
+
+}

Nem az összes módosított fájl került megjelenítésre, mert túl sok fájl változott