|
@@ -29,6 +29,7 @@ import static org.mockito.Mockito.when;
|
|
|
import java.io.File;
|
|
|
import java.io.FileOutputStream;
|
|
|
import java.io.IOException;
|
|
|
+import java.util.HashMap;
|
|
|
|
|
|
import org.junit.Assert;
|
|
|
|
|
@@ -42,9 +43,12 @@ import org.apache.hadoop.fs.LocalFileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.mapreduce.CounterGroup;
|
|
|
import org.apache.hadoop.mapreduce.Counters;
|
|
|
+import org.apache.hadoop.mapreduce.JobACL;
|
|
|
import org.apache.hadoop.mapreduce.JobID;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
+import org.apache.hadoop.mapreduce.TaskAttemptID;
|
|
|
import org.apache.hadoop.mapreduce.TaskID;
|
|
|
import org.apache.hadoop.mapreduce.TaskType;
|
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
@@ -55,14 +59,22 @@ import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
|
|
|
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
|
+import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
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.api.records.timeline.TimelineEntities;
|
|
|
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
|
|
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.junit.After;
|
|
|
import org.junit.AfterClass;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import org.junit.BeforeClass;
|
|
|
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
|
|
+import org.apache.hadoop.yarn.server.timeline.TimelineStore;
|
|
|
+import org.codehaus.jackson.JsonNode;
|
|
|
+import org.codehaus.jackson.map.ObjectMapper;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.Mockito;
|
|
|
|
|
@@ -126,7 +138,7 @@ public class TestJobHistoryEventHandler {
|
|
|
|
|
|
// First completion event, but min-queue-size for batching flushes is 10
|
|
|
handleEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
|
|
|
- t.taskID, null, 0, TaskType.MAP, "", null)));
|
|
|
+ t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null)));
|
|
|
verify(mockWriter).flush();
|
|
|
|
|
|
} finally {
|
|
@@ -162,7 +174,7 @@ public class TestJobHistoryEventHandler {
|
|
|
|
|
|
for (int i = 0 ; i < 100 ; i++) {
|
|
|
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
|
|
|
- t.taskID, null, 0, TaskType.MAP, "", null)));
|
|
|
+ t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null)));
|
|
|
}
|
|
|
|
|
|
handleNextNEvents(jheh, 9);
|
|
@@ -207,7 +219,7 @@ public class TestJobHistoryEventHandler {
|
|
|
|
|
|
for (int i = 0 ; i < 100 ; i++) {
|
|
|
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
|
|
|
- t.taskID, null, 0, TaskType.MAP, "", null)));
|
|
|
+ t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null)));
|
|
|
}
|
|
|
|
|
|
handleNextNEvents(jheh, 9);
|
|
@@ -248,7 +260,7 @@ public class TestJobHistoryEventHandler {
|
|
|
|
|
|
for (int i = 0 ; i < 100 ; i++) {
|
|
|
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
|
|
|
- t.taskID, null, 0, TaskType.MAP, "", null)));
|
|
|
+ t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null)));
|
|
|
}
|
|
|
queueEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
|
|
|
TypeConverter.fromYarn(t.jobId), 0, 10, 10, 0, 0, null, null, new Counters())));
|
|
@@ -427,6 +439,231 @@ public class TestJobHistoryEventHandler {
|
|
|
pathStr);
|
|
|
}
|
|
|
|
|
|
+ // Have JobHistoryEventHandler handle some events and make sure they get
|
|
|
+ // stored to the Timeline store
|
|
|
+ @Test (timeout=50000)
|
|
|
+ public void testTimelineEventHandling() throws Exception {
|
|
|
+ TestParams t = new TestParams(false);
|
|
|
+ Configuration conf = new YarnConfiguration();
|
|
|
+ conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
|
|
+ JHEvenHandlerForTest jheh = new JHEvenHandlerForTest(t.mockAppContext, 0);
|
|
|
+ jheh.init(conf);
|
|
|
+ MiniYARNCluster yarnCluster = null;
|
|
|
+ long currentTime = System.currentTimeMillis();
|
|
|
+ try {
|
|
|
+ yarnCluster = new MiniYARNCluster(
|
|
|
+ TestJobHistoryEventHandler.class.getSimpleName(), 1, 1, 1, 1, true);
|
|
|
+ yarnCluster.init(conf);
|
|
|
+ yarnCluster.start();
|
|
|
+ jheh.start();
|
|
|
+ TimelineStore ts = yarnCluster.getApplicationHistoryServer()
|
|
|
+ .getTimelineStore();
|
|
|
+
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
|
|
|
+ t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000),
|
|
|
+ currentTime - 10));
|
|
|
+ TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
|
|
|
+ null, null, null, null, null, null);
|
|
|
+ Assert.assertEquals(1, entities.getEntities().size());
|
|
|
+ TimelineEntity tEntity = entities.getEntities().get(0);
|
|
|
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
|
|
|
+ Assert.assertEquals(1, tEntity.getEvents().size());
|
|
|
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
|
|
|
+ tEntity.getEvents().get(0).getEventType());
|
|
|
+ Assert.assertEquals(currentTime - 10,
|
|
|
+ tEntity.getEvents().get(0).getTimestamp());
|
|
|
+
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId,
|
|
|
+ new JobSubmittedEvent(TypeConverter.fromYarn(t.jobId), "name",
|
|
|
+ "user", 200, "/foo/job.xml",
|
|
|
+ new HashMap<JobACL, AccessControlList>(), "default"),
|
|
|
+ currentTime + 10));
|
|
|
+ entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
|
|
|
+ null, null, null, null, null);
|
|
|
+ Assert.assertEquals(1, entities.getEntities().size());
|
|
|
+ tEntity = entities.getEntities().get(0);
|
|
|
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
|
|
|
+ Assert.assertEquals(2, tEntity.getEvents().size());
|
|
|
+ Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
|
|
|
+ tEntity.getEvents().get(0).getEventType());
|
|
|
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
|
|
|
+ tEntity.getEvents().get(1).getEventType());
|
|
|
+ Assert.assertEquals(currentTime + 10,
|
|
|
+ tEntity.getEvents().get(0).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime - 10,
|
|
|
+ tEntity.getEvents().get(1).getTimestamp());
|
|
|
+
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId,
|
|
|
+ new JobQueueChangeEvent(TypeConverter.fromYarn(t.jobId), "q2"),
|
|
|
+ currentTime - 20));
|
|
|
+ entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
|
|
|
+ null, null, null, null, null);
|
|
|
+ Assert.assertEquals(1, entities.getEntities().size());
|
|
|
+ tEntity = entities.getEntities().get(0);
|
|
|
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
|
|
|
+ Assert.assertEquals(3, tEntity.getEvents().size());
|
|
|
+ Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
|
|
|
+ tEntity.getEvents().get(0).getEventType());
|
|
|
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
|
|
|
+ tEntity.getEvents().get(1).getEventType());
|
|
|
+ Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
|
|
|
+ tEntity.getEvents().get(2).getEventType());
|
|
|
+ Assert.assertEquals(currentTime + 10,
|
|
|
+ tEntity.getEvents().get(0).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime - 10,
|
|
|
+ tEntity.getEvents().get(1).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime - 20,
|
|
|
+ tEntity.getEvents().get(2).getTimestamp());
|
|
|
+
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId,
|
|
|
+ new JobFinishedEvent(TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0,
|
|
|
+ 0, new Counters(), new Counters(), new Counters()), currentTime));
|
|
|
+ entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
|
|
|
+ null, null, null, null, null);
|
|
|
+ Assert.assertEquals(1, entities.getEntities().size());
|
|
|
+ tEntity = entities.getEntities().get(0);
|
|
|
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
|
|
|
+ Assert.assertEquals(4, tEntity.getEvents().size());
|
|
|
+ Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
|
|
|
+ tEntity.getEvents().get(0).getEventType());
|
|
|
+ Assert.assertEquals(EventType.JOB_FINISHED.toString(),
|
|
|
+ tEntity.getEvents().get(1).getEventType());
|
|
|
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
|
|
|
+ tEntity.getEvents().get(2).getEventType());
|
|
|
+ Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
|
|
|
+ tEntity.getEvents().get(3).getEventType());
|
|
|
+ Assert.assertEquals(currentTime + 10,
|
|
|
+ tEntity.getEvents().get(0).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime,
|
|
|
+ tEntity.getEvents().get(1).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime - 10,
|
|
|
+ tEntity.getEvents().get(2).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime - 20,
|
|
|
+ tEntity.getEvents().get(3).getTimestamp());
|
|
|
+
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId,
|
|
|
+ new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId),
|
|
|
+ 0, 0, 0, JobStateInternal.KILLED.toString()), currentTime + 20));
|
|
|
+ entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
|
|
|
+ null, null, null, null, null);
|
|
|
+ Assert.assertEquals(1, entities.getEntities().size());
|
|
|
+ tEntity = entities.getEntities().get(0);
|
|
|
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
|
|
|
+ Assert.assertEquals(5, tEntity.getEvents().size());
|
|
|
+ Assert.assertEquals(EventType.JOB_KILLED.toString(),
|
|
|
+ tEntity.getEvents().get(0).getEventType());
|
|
|
+ Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
|
|
|
+ tEntity.getEvents().get(1).getEventType());
|
|
|
+ Assert.assertEquals(EventType.JOB_FINISHED.toString(),
|
|
|
+ tEntity.getEvents().get(2).getEventType());
|
|
|
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
|
|
|
+ tEntity.getEvents().get(3).getEventType());
|
|
|
+ Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
|
|
|
+ tEntity.getEvents().get(4).getEventType());
|
|
|
+ Assert.assertEquals(currentTime + 20,
|
|
|
+ tEntity.getEvents().get(0).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime + 10,
|
|
|
+ tEntity.getEvents().get(1).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime,
|
|
|
+ tEntity.getEvents().get(2).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime - 10,
|
|
|
+ tEntity.getEvents().get(3).getTimestamp());
|
|
|
+ Assert.assertEquals(currentTime - 20,
|
|
|
+ tEntity.getEvents().get(4).getTimestamp());
|
|
|
+
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId,
|
|
|
+ new TaskStartedEvent(t.taskID, 0, TaskType.MAP, "")));
|
|
|
+ entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
|
|
|
+ null, null, null, null, null);
|
|
|
+ Assert.assertEquals(1, entities.getEntities().size());
|
|
|
+ tEntity = entities.getEntities().get(0);
|
|
|
+ Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());
|
|
|
+ Assert.assertEquals(1, tEntity.getEvents().size());
|
|
|
+ Assert.assertEquals(EventType.TASK_STARTED.toString(),
|
|
|
+ tEntity.getEvents().get(0).getEventType());
|
|
|
+ Assert.assertEquals(TaskType.MAP.toString(),
|
|
|
+ tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE"));
|
|
|
+
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId,
|
|
|
+ new TaskStartedEvent(t.taskID, 0, TaskType.REDUCE, "")));
|
|
|
+ entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
|
|
|
+ null, null, null, null, null);
|
|
|
+ Assert.assertEquals(1, entities.getEntities().size());
|
|
|
+ tEntity = entities.getEntities().get(0);
|
|
|
+ Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());
|
|
|
+ Assert.assertEquals(2, tEntity.getEvents().size());
|
|
|
+ Assert.assertEquals(EventType.TASK_STARTED.toString(),
|
|
|
+ tEntity.getEvents().get(1).getEventType());
|
|
|
+ Assert.assertEquals(TaskType.REDUCE.toString(),
|
|
|
+ tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE"));
|
|
|
+ Assert.assertEquals(TaskType.MAP.toString(),
|
|
|
+ tEntity.getEvents().get(1).getEventInfo().get("TASK_TYPE"));
|
|
|
+ } finally {
|
|
|
+ if (yarnCluster != null) {
|
|
|
+ yarnCluster.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test (timeout=50000)
|
|
|
+ public void testCountersToJSON() throws Exception {
|
|
|
+ JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0);
|
|
|
+ Counters counters = new Counters();
|
|
|
+ CounterGroup group1 = counters.addGroup("DOCTORS",
|
|
|
+ "Incarnations of the Doctor");
|
|
|
+ group1.addCounter("PETER_CAPALDI", "Peter Capaldi", 12);
|
|
|
+ group1.addCounter("MATT_SMITH", "Matt Smith", 11);
|
|
|
+ group1.addCounter("DAVID_TENNANT", "David Tennant", 10);
|
|
|
+ CounterGroup group2 = counters.addGroup("COMPANIONS",
|
|
|
+ "Companions of the Doctor");
|
|
|
+ group2.addCounter("CLARA_OSWALD", "Clara Oswald", 6);
|
|
|
+ group2.addCounter("RORY_WILLIAMS", "Rory Williams", 5);
|
|
|
+ group2.addCounter("AMY_POND", "Amy Pond", 4);
|
|
|
+ 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);
|
|
|
+ String jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
|
|
|
+ String expected = "[{\"NAME\":\"COMPANIONS\",\"DISPLAY_NAME\":\"Companions "
|
|
|
+ + "of the Doctor\",\"COUNTERS\":[{\"NAME\":\"AMY_POND\",\"DISPLAY_NAME\""
|
|
|
+ + ":\"Amy Pond\",\"VALUE\":4},{\"NAME\":\"CLARA_OSWALD\","
|
|
|
+ + "\"DISPLAY_NAME\":\"Clara Oswald\",\"VALUE\":6},{\"NAME\":"
|
|
|
+ + "\"DONNA_NOBLE\",\"DISPLAY_NAME\":\"Donna Noble\",\"VALUE\":2},"
|
|
|
+ + "{\"NAME\":\"MARTHA_JONES\",\"DISPLAY_NAME\":\"Martha Jones\","
|
|
|
+ + "\"VALUE\":3},{\"NAME\":\"RORY_WILLIAMS\",\"DISPLAY_NAME\":\"Rory "
|
|
|
+ + "Williams\",\"VALUE\":5},{\"NAME\":\"ROSE_TYLER\",\"DISPLAY_NAME\":"
|
|
|
+ + "\"Rose Tyler\",\"VALUE\":1}]},{\"NAME\":\"DOCTORS\",\"DISPLAY_NAME\""
|
|
|
+ + ":\"Incarnations of the Doctor\",\"COUNTERS\":[{\"NAME\":"
|
|
|
+ + "\"DAVID_TENNANT\",\"DISPLAY_NAME\":\"David Tennant\",\"VALUE\":10},"
|
|
|
+ + "{\"NAME\":\"MATT_SMITH\",\"DISPLAY_NAME\":\"Matt Smith\",\"VALUE\":"
|
|
|
+ + "11},{\"NAME\":\"PETER_CAPALDI\",\"DISPLAY_NAME\":\"Peter Capaldi\","
|
|
|
+ + "\"VALUE\":12}]}]";
|
|
|
+ Assert.assertEquals(expected, jsonStr);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test (timeout=50000)
|
|
|
+ public void testCountersToJSONEmpty() throws Exception {
|
|
|
+ JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0);
|
|
|
+ Counters counters = null;
|
|
|
+ JsonNode jsonNode = jheh.countersToJSON(counters);
|
|
|
+ String jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
|
|
|
+ String expected = "[]";
|
|
|
+ Assert.assertEquals(expected, jsonStr);
|
|
|
+
|
|
|
+ counters = new Counters();
|
|
|
+ jsonNode = jheh.countersToJSON(counters);
|
|
|
+ jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
|
|
|
+ expected = "[]";
|
|
|
+ Assert.assertEquals(expected, jsonStr);
|
|
|
+
|
|
|
+ counters.addGroup("DOCTORS", "Incarnations of the Doctor");
|
|
|
+ jsonNode = jheh.countersToJSON(counters);
|
|
|
+ jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
|
|
|
+ expected = "[{\"NAME\":\"DOCTORS\",\"DISPLAY_NAME\":\"Incarnations of the "
|
|
|
+ + "Doctor\",\"COUNTERS\":[]}]";
|
|
|
+ Assert.assertEquals(expected, jsonStr);
|
|
|
+ }
|
|
|
+
|
|
|
private void queueEvent(JHEvenHandlerForTest jheh, JobHistoryEvent event) {
|
|
|
jheh.handle(event);
|
|
|
}
|
|
@@ -480,6 +717,7 @@ public class TestJobHistoryEventHandler {
|
|
|
ApplicationAttemptId.newInstance(appId, 1);
|
|
|
ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
|
|
|
TaskID taskID = TaskID.forName("task_200707121733_0003_m_000005");
|
|
|
+ TaskAttemptID taskAttemptID = new TaskAttemptID(taskID, 0);
|
|
|
JobId jobId = MRBuilderUtils.newJobId(appId, 1);
|
|
|
AppContext mockAppContext;
|
|
|
|
|
@@ -557,11 +795,13 @@ class JHEvenHandlerForTest extends JobHistoryEventHandler {
|
|
|
private boolean mockHistoryProcessing = true;
|
|
|
public JHEvenHandlerForTest(AppContext context, int startCount) {
|
|
|
super(context, startCount);
|
|
|
+ JobHistoryEventHandler.fileMap.clear();
|
|
|
}
|
|
|
|
|
|
public JHEvenHandlerForTest(AppContext context, int startCount, boolean mockHistoryProcessing) {
|
|
|
super(context, startCount);
|
|
|
this.mockHistoryProcessing = mockHistoryProcessing;
|
|
|
+ JobHistoryEventHandler.fileMap.clear();
|
|
|
}
|
|
|
|
|
|
@Override
|