tez-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From hit...@apache.org
Subject tez git commit: TEZ-1942. Number of tasks show in Tez UI with auto-reduce parallelism is misleading. (Prakash Ramachandran via hitesh)
Date Mon, 19 Jan 2015 18:57:58 GMT
Repository: tez
Updated Branches:
  refs/heads/branch-0.5 cb427d9c3 -> 4bbd257db


TEZ-1942. Number of tasks show in Tez UI with auto-reduce parallelism is misleading. (Prakash
Ramachandran via hitesh)


Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/4bbd257d
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/4bbd257d
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/4bbd257d

Branch: refs/heads/branch-0.5
Commit: 4bbd257db9a8085df7f4688ffea3eb40cbd2c174
Parents: cb427d9
Author: Hitesh Shah <hitesh@apache.org>
Authored: Mon Jan 19 10:57:36 2015 -0800
Committer: Hitesh Shah <hitesh@apache.org>
Committed: Mon Jan 19 10:57:36 2015 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../ats/HistoryEventTimelineConversion.java     |  21 ++
 .../ats/TestHistoryEventTimelineConversion.java | 205 ++++++++++++++++++-
 3 files changed, 224 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/4bbd257d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e2dced1..7244db8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,7 @@ Apache Tez Change Log
 Release 0.5.4: Unreleased
 
 ALL CHANGES:
+  TEZ-1942. Number of tasks show in Tez UI with auto-reduce parallelism is misleading.
   TEZ-1962. Fix a thread leak in LocalMode.
   TEZ-1924. Tez AM does not register with AM with full FQDN causing jobs
   to fail in some environments.

http://git-wip-us.apache.org/repos/asf/tez/blob/4bbd257d/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
index 4b6d648..19e2741 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
@@ -160,6 +160,8 @@ public class HistoryEventTimelineConversion {
     atsEntity.addRelatedEntity(ATSConstants.USER, event.getUser());
 
     atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser());
+    atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID,
+        event.getApplicationAttemptId().getApplicationId().toString());
 
     atsEntity.setStartTime(event.getLaunchTime());
 
@@ -230,6 +232,8 @@ public class HistoryEventTimelineConversion {
     stoppedEvt.setTimestamp(event.getStoppedTime());
     atsEntity.addEvent(stoppedEvt);
 
+    atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID,
+        event.getApplicationAttemptId().getApplicationId().toString());
     atsEntity.addPrimaryFilter(ATSConstants.EXIT_STATUS, event.getExitStatus());
 
     atsEntity.addOtherInfo(ATSConstants.EXIT_STATUS, event.getExitStatus());
@@ -249,6 +253,8 @@ public class HistoryEventTimelineConversion {
     atsEntity.addEvent(finishEvt);
 
     atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser());
+    atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID,
+        event.getDagID().getApplicationId().toString());
     atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName());
     atsEntity.addPrimaryFilter(ATSConstants.STATUS, event.getState().name());
 
@@ -281,6 +287,8 @@ public class HistoryEventTimelineConversion {
     atsEntity.addEvent(initEvt);
 
     atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser());
+    atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID,
+        event.getDagID().getApplicationId().toString());
     atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName());
 
     atsEntity.addOtherInfo(ATSConstants.INIT_TIME, event.getInitTime());
@@ -307,6 +315,8 @@ public class HistoryEventTimelineConversion {
     atsEntity.addEvent(startEvt);
 
     atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser());
+    atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID,
+        event.getDagID().getApplicationId().toString());
     atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName());
 
     atsEntity.addOtherInfo(ATSConstants.START_TIME, event.getStartTime());
@@ -359,6 +369,8 @@ public class HistoryEventTimelineConversion {
     atsEntity.setEntityId(event.getTaskAttemptID().toString());
     atsEntity.setEntityType(EntityTypes.TEZ_TASK_ATTEMPT_ID.name());
 
+    atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID,
+        event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getApplicationId().toString());
     atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(),
         event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().toString());
     atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(),
@@ -424,6 +436,8 @@ public class HistoryEventTimelineConversion {
     atsEntity.setEntityId(event.getTaskID().toString());
     atsEntity.setEntityType(EntityTypes.TEZ_TASK_ID.name());
 
+    atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID,
+        event.getTaskID().getVertexID().getDAGId().getApplicationId().toString());
     atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(),
         event.getTaskID().getVertexID().getDAGId().toString());
     atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(),
@@ -484,6 +498,8 @@ public class HistoryEventTimelineConversion {
     atsEntity.setEntityId(event.getVertexID().toString());
     atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name());
 
+    atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID,
+        event.getVertexID().getDAGId().getApplicationId().toString());
     atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(),
         event.getVertexID().getDAGId().toString());
 
@@ -571,6 +587,11 @@ public class HistoryEventTimelineConversion {
     atsEntity.setEntityId(event.getVertexID().toString());
     atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name());
 
+    atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID,
+        event.getVertexID().getDAGId().getApplicationId().toString());
+    atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(),
+        event.getVertexID().getDAGId().toString());
+
     TimelineEvent updateEvt = new TimelineEvent();
     updateEvt.setEventType(HistoryEventType.VERTEX_PARALLELISM_UPDATED.name());
     updateEvt.setTimestamp(event.getUpdateTime());

http://git-wip-us.apache.org/repos/asf/tez/blob/4bbd257d/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
index 1347d28..fe3d780 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
@@ -21,6 +21,7 @@ package org.apache.tez.dag.history.logging.ats;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -30,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.tez.common.ATSConstants;
+import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.EdgeManagerPluginDescriptor;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.api.oldrecords.TaskState;
@@ -228,6 +230,43 @@ public class TestHistoryEventTimelineConversion {
     Assert.assertEquals(conf.get("applicationId"), config.get("applicationId"));
   }
 
+  @Test(timeout = 5000)
+  public void testConvertAMLaunchedEvent() {
+    long launchTime = random.nextLong();
+    long submitTime = random.nextLong();
+    AMLaunchedEvent event = new AMLaunchedEvent(applicationAttemptId, launchTime, submitTime,
user);
+
+    TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
+
+    Assert.assertEquals("tez_" + applicationAttemptId.toString(), timelineEntity.getEntityId());
+    Assert.assertEquals(EntityTypes.TEZ_APPLICATION_ATTEMPT.name(), timelineEntity.getEntityType());
+
+    final Map<String, Set<String>> relatedEntities = timelineEntity.getRelatedEntities();
+    Assert.assertEquals(3, relatedEntities.size());
+    Assert.assertTrue(relatedEntities.get(ATSConstants.APPLICATION_ID)
+        .contains(applicationId.toString()));
+    Assert.assertTrue(relatedEntities.get(ATSConstants.APPLICATION_ATTEMPT_ID)
+        .contains(applicationAttemptId.toString()));
+    Assert.assertTrue(relatedEntities.get(ATSConstants.USER).contains(user));
+
+    final Map<String, Set<Object>> primaryFilters = timelineEntity.getPrimaryFilters();
+    Assert.assertEquals(2, primaryFilters.size());
+    Assert.assertTrue(primaryFilters.get(ATSConstants.USER).contains(user));
+    Assert.assertTrue(primaryFilters.get(ATSConstants.APPLICATION_ID)
+        .contains(applicationId.toString()));
+
+    Assert.assertEquals(launchTime, timelineEntity.getStartTime().longValue());
+
+    Assert.assertEquals(1, timelineEntity.getEvents().size());
+    TimelineEvent evt = timelineEntity.getEvents().get(0);
+    Assert.assertEquals(HistoryEventType.AM_LAUNCHED.name(), evt.getEventType());
+    Assert.assertEquals(launchTime, evt.getTimestamp());
+
+    final Map<String, Object> otherInfo = timelineEntity.getOtherInfo();
+    Assert.assertEquals(1, otherInfo.size());
+    Assert.assertEquals(submitTime, otherInfo.get(ATSConstants.APP_SUBMIT_TIME));
+  }
+
   @Test
   public void testConvertContainerLaunchedEvent() {
     long launchTime = random.nextLong();
@@ -258,6 +297,67 @@ public class TestHistoryEventTimelineConversion {
         timelineEntity.getEvents().get(0).getTimestamp());
   }
 
+  @Test(timeout = 5000)
+  public void testConvertContainerStoppedEvent() {
+    long stopTime = random.nextLong();
+    int exitStatus = random.nextInt();
+    ContainerStoppedEvent event = new ContainerStoppedEvent(containerId, stopTime, exitStatus,
+        applicationAttemptId);
+    TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
+
+    Assert.assertEquals("tez_" + containerId.toString(), timelineEntity.getEntityId());
+    Assert.assertEquals(EntityTypes.TEZ_CONTAINER_ID.name(), timelineEntity.getEntityType());
+
+    final Map<String, Set<String>> relatedEntities = timelineEntity.getRelatedEntities();
+    Assert.assertEquals(1, relatedEntities.size());
+    Assert.assertTrue(relatedEntities.get(EntityTypes.TEZ_APPLICATION_ATTEMPT.name())
+        .contains("tez_" + applicationAttemptId.toString()));
+
+    final Map<String, Set<Object>> primaryFilters = timelineEntity.getPrimaryFilters();
+    Assert.assertEquals(2, primaryFilters.size());
+    Assert.assertTrue(primaryFilters.get(ATSConstants.APPLICATION_ID)
+        .contains(applicationId.toString()));
+    Assert.assertTrue(primaryFilters.get(ATSConstants.EXIT_STATUS).contains(exitStatus));
+
+    Assert.assertEquals(1, timelineEntity.getEvents().size());
+    final TimelineEvent evt = timelineEntity.getEvents().get(0);
+    Assert.assertEquals(HistoryEventType.CONTAINER_STOPPED.name(), evt.getEventType());
+    Assert.assertEquals(stopTime, evt.getTimestamp());
+
+    final Map<String, Object> otherInfo = timelineEntity.getOtherInfo();
+    Assert.assertEquals(2, otherInfo.size());
+    Assert.assertEquals(exitStatus, otherInfo.get(ATSConstants.EXIT_STATUS));
+    Assert.assertEquals(stopTime, otherInfo.get(ATSConstants.FINISH_TIME));
+  }
+
+  @Test(timeout = 5000)
+  public void testConvertDAGStartedEvent() {
+    long startTime = random.nextLong();
+    String dagName = "testDagName";
+    DAGStartedEvent event = new DAGStartedEvent(tezDAGID, startTime, user, dagName);
+    TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
+
+    Assert.assertEquals(tezDAGID.toString(), timelineEntity.getEntityId());
+    Assert.assertEquals(EntityTypes.TEZ_DAG_ID.name(), timelineEntity.getEntityType());
+
+    Assert.assertEquals(1, timelineEntity.getEvents().size());
+    TimelineEvent evt = timelineEntity.getEvents().get(0);
+    Assert.assertEquals(HistoryEventType.DAG_STARTED.name(), evt.getEventType());
+    Assert.assertEquals(startTime, evt.getTimestamp());
+
+    final Map<String, Set<Object>> primaryFilters = timelineEntity.getPrimaryFilters();
+    Assert.assertEquals(3, primaryFilters.size());
+    Assert.assertTrue(primaryFilters.get(ATSConstants.USER).contains(user));
+    Assert.assertTrue(primaryFilters.get(ATSConstants.APPLICATION_ID)
+        .contains(applicationId.toString()));
+    Assert.assertTrue(primaryFilters.get(ATSConstants.DAG_NAME).contains(dagName));
+
+    final Map<String, Object> otherInfo = timelineEntity.getOtherInfo();
+    Assert.assertEquals(2, otherInfo.size());
+    Assert.assertEquals(startTime, otherInfo.get(ATSConstants.START_TIME));
+    Assert.assertEquals(DAGState.RUNNING.name(), otherInfo.get(ATSConstants.STATUS));
+  }
+
   @Test
   public void testConvertDAGSubmittedEvent() {
     long submitTime = random.nextLong();
@@ -309,6 +409,48 @@ public class TestHistoryEventTimelineConversion {
 
   }
 
+  @Test(timeout = 5000)
+  public void testConvertTaskAttemptFinishedEvent(){
+    String vertexName = "testVertex";
+    long startTime = random.nextLong();
+    long finishTime = startTime + 1234;
+    TaskAttemptState state = TaskAttemptState
+        .values()[random.nextInt(TaskAttemptState.values().length)];
+    String diagnostics = "random diagnostics message";
+    TezCounters counters = new TezCounters();
+
+    TaskAttemptFinishedEvent event = new TaskAttemptFinishedEvent(tezTaskAttemptID, vertexName,
+        startTime, finishTime, state, diagnostics, counters);
+    TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
+    Assert.assertEquals(tezTaskAttemptID.toString(), timelineEntity.getEntityId());
+    Assert.assertEquals(EntityTypes.TEZ_TASK_ATTEMPT_ID.name(), timelineEntity.getEntityType());
+
+    final Map<String, Set<Object>> primaryFilters = timelineEntity.getPrimaryFilters();
+    Assert.assertEquals(5, primaryFilters.size());
+    Assert.assertTrue(primaryFilters.get(ATSConstants.APPLICATION_ID)
+        .contains(applicationId.toString()));
+    Assert.assertTrue(primaryFilters.get(EntityTypes.TEZ_DAG_ID.name())
+        .contains(tezDAGID.toString()));
+    Assert.assertTrue(primaryFilters.get(EntityTypes.TEZ_VERTEX_ID.name())
+        .contains(tezVertexID.toString()));
+    Assert.assertTrue(primaryFilters.get(EntityTypes.TEZ_TASK_ID.name())
+        .contains(tezTaskID.toString()));
+    Assert.assertTrue(primaryFilters.get(ATSConstants.STATUS).contains(state.toString()));
+
+    Assert.assertEquals(1, timelineEntity.getEvents().size());
+    TimelineEvent evt = timelineEntity.getEvents().get(0);
+    Assert.assertEquals(HistoryEventType.TASK_ATTEMPT_FINISHED.name(), evt.getEventType());
+    Assert.assertEquals(finishTime, evt.getTimestamp());
+
+    final Map<String, Object> otherInfo = timelineEntity.getOtherInfo();
+    Assert.assertEquals(5, otherInfo.size());
+    Assert.assertEquals(finishTime, otherInfo.get(ATSConstants.FINISH_TIME));
+    Assert.assertEquals(finishTime - startTime, otherInfo.get(ATSConstants.TIME_TAKEN));
+    Assert.assertEquals(state.name(), otherInfo.get(ATSConstants.STATUS));
+    Assert.assertEquals(diagnostics, otherInfo.get(ATSConstants.DIAGNOSTICS));
+    Assert.assertTrue(otherInfo.containsKey(ATSConstants.COUNTERS));
+  }
+
   @Test
   public void testConvertDAGInitializedEvent() {
     long initTime = random.nextLong();
@@ -330,7 +472,10 @@ public class TestHistoryEventTimelineConversion {
     Assert.assertEquals(HistoryEventType.DAG_INITIALIZED.name(), timelineEvent.getEventType());
     Assert.assertEquals(initTime, timelineEvent.getTimestamp());
 
-    Assert.assertEquals(2, timelineEntity.getPrimaryFilters().size());
+    Assert.assertEquals(3, timelineEntity.getPrimaryFilters().size());
+    Assert.assertTrue(
+        timelineEntity.getPrimaryFilters().get(ATSConstants.APPLICATION_ID).contains(
+            applicationId.toString()));
     Assert.assertTrue(
         timelineEntity.getPrimaryFilters().get(ATSConstants.DAG_NAME).contains("dagName"));
     Assert.assertTrue(
@@ -368,7 +513,10 @@ public class TestHistoryEventTimelineConversion {
     Assert.assertEquals(HistoryEventType.DAG_FINISHED.name(), timelineEvent.getEventType());
     Assert.assertEquals(finishTime, timelineEvent.getTimestamp());
 
-    Assert.assertEquals(3, timelineEntity.getPrimaryFilters().size());
+    Assert.assertEquals(4, timelineEntity.getPrimaryFilters().size());
+    Assert.assertTrue(
+        timelineEntity.getPrimaryFilters().get(ATSConstants.APPLICATION_ID).contains(
+            applicationId.toString()));
     Assert.assertTrue(
         timelineEntity.getPrimaryFilters().get(ATSConstants.DAG_NAME).contains(dagPlan.getName()));
     Assert.assertTrue(
@@ -462,7 +610,9 @@ public class TestHistoryEventTimelineConversion {
 
     Assert.assertEquals(0, timelineEntity.getRelatedEntities().size());
 
-    Assert.assertEquals(2, timelineEntity.getPrimaryFilters().size());
+    Assert.assertEquals(3, timelineEntity.getPrimaryFilters().size());
+    Assert.assertTrue(timelineEntity.getPrimaryFilters().get(ATSConstants.APPLICATION_ID)
+        .contains(applicationId.toString()));
     Assert.assertTrue(
         timelineEntity.getPrimaryFilters().get(EntityTypes.TEZ_DAG_ID.name()).contains(
             tezDAGID.toString()));
@@ -588,6 +738,48 @@ public class TestHistoryEventTimelineConversion {
         timelineEntity.getOtherInfo().get(ATSConstants.NODE_HTTP_ADDRESS));
   }
 
+  @Test(timeout = 5000)
+  public void testConvertTaskFinishedEvent() {
+    String vertexName = "testVertexName";
+    long startTime = random.nextLong();
+    long finishTime = random.nextLong();
+    TaskState state = TaskState.values()[random.nextInt(TaskState.values().length)];
+    String diagnostics = "diagnostics message";
+    TezCounters counters = new TezCounters();
+
+    TaskFinishedEvent event = new TaskFinishedEvent(tezTaskID, vertexName, startTime, finishTime,
+        tezTaskAttemptID, state, diagnostics, counters);
+    TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
+
+    Assert.assertEquals(tezTaskID.toString(), timelineEntity.getEntityId());
+    Assert.assertEquals(EntityTypes.TEZ_TASK_ID.name(), timelineEntity.getEntityType());
+
+    final Map<String, Set<Object>> primaryFilters = timelineEntity.getPrimaryFilters();
+    Assert.assertEquals(4, primaryFilters.size());
+    Assert.assertTrue(primaryFilters.get(ATSConstants.APPLICATION_ID)
+        .contains(applicationId.toString()));
+    Assert.assertTrue(primaryFilters.get(EntityTypes.TEZ_DAG_ID.name())
+        .contains(tezDAGID.toString()));
+    Assert.assertTrue(primaryFilters.get(EntityTypes.TEZ_VERTEX_ID.name())
+        .contains(tezVertexID.toString()));
+    Assert.assertTrue(primaryFilters.get(ATSConstants.STATUS).contains(state.name()));
+
+    Assert.assertEquals(1, timelineEntity.getEvents().size());
+    TimelineEvent evt = timelineEntity.getEvents().get(0);
+    Assert.assertEquals(HistoryEventType.TASK_FINISHED.name(), evt.getEventType());
+    Assert.assertEquals(finishTime, evt.getTimestamp());
+
+    final Map<String, Object> otherInfo = timelineEntity.getOtherInfo();
+    Assert.assertEquals(6, otherInfo.size());
+    Assert.assertEquals(finishTime, otherInfo.get(ATSConstants.FINISH_TIME));
+    Assert.assertEquals(finishTime - startTime, otherInfo.get(ATSConstants.TIME_TAKEN));
+    Assert.assertEquals(state.name(), otherInfo.get(ATSConstants.STATUS));
+    Assert.assertEquals(tezTaskAttemptID.toString(),
+        otherInfo.get(ATSConstants.SUCCESSFUL_ATTEMPT_ID));
+    Assert.assertEquals(diagnostics, otherInfo.get(ATSConstants.DIAGNOSTICS));
+    Assert.assertTrue(otherInfo.containsKey(ATSConstants.COUNTERS));
+  }
+
   @Test
   public void testConvertVertexParallelismUpdatedEvent() {
     TezVertexID vId = tezVertexID;
@@ -602,6 +794,13 @@ public class TestHistoryEventTimelineConversion {
     Assert.assertEquals(vId.toString(), timelineEntity.getEntityId());
     Assert.assertEquals(1, timelineEntity.getEvents().size());
 
+    final Map<String, Set<Object>> primaryFilters = timelineEntity.getPrimaryFilters();
+    Assert.assertEquals(2, primaryFilters.size());
+    Assert.assertTrue(primaryFilters.get(ATSConstants.APPLICATION_ID)
+        .contains(applicationId.toString()));
+    Assert.assertTrue(primaryFilters.get(EntityTypes.TEZ_DAG_ID.name())
+        .contains(tezDAGID.toString()));
+
     TimelineEvent evt = timelineEntity.getEvents().get(0);
     Assert.assertEquals(HistoryEventType.VERTEX_PARALLELISM_UPDATED.name(), evt.getEventType());
     Assert.assertEquals(1, evt.getEventInfo().get(ATSConstants.NUM_TASKS));


Mime
View raw message