|
@@ -24,6 +24,7 @@ import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.StringTokenizer;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import junit.framework.Assert;
|
|
|
|
|
@@ -37,14 +38,18 @@ import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.TaskID;
|
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.EventReader;
|
|
|
+import org.apache.hadoop.mapreduce.jobhistory.HistoryEvent;
|
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
|
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
|
|
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.mapreduce.jobhistory.TaskFinishedEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.MRApp;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
|
@@ -61,6 +66,9 @@ import org.apache.hadoop.yarn.service.Service;
|
|
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.RackResolver;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
+import org.mockito.invocation.InvocationOnMock;
|
|
|
+import org.mockito.stubbing.Answer;
|
|
|
|
|
|
public class TestJobHistoryParsing {
|
|
|
private static final Log LOG = LogFactory.getLog(TestJobHistoryParsing.class);
|
|
@@ -76,6 +84,17 @@ public class TestJobHistoryParsing {
|
|
|
|
|
|
@Test
|
|
|
public void testHistoryParsing() throws Exception {
|
|
|
+ checkHistoryParsing(2, 1, 2);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testHistoryParsingWithParseErrors() throws Exception {
|
|
|
+ checkHistoryParsing(3, 0, 2);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void checkHistoryParsing(final int numMaps, final int numReduces,
|
|
|
+ final int numSuccessfulMaps)
|
|
|
+ throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
|
|
|
long amStartTimeEst = System.currentTimeMillis();
|
|
@@ -83,8 +102,9 @@ public class TestJobHistoryParsing {
|
|
|
CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
MyResolver.class, DNSToSwitchMapping.class);
|
|
|
RackResolver.init(conf);
|
|
|
- MRApp app = new MRAppWithHistory(2, 1, true, this.getClass().getName(),
|
|
|
- true);
|
|
|
+ MRApp app =
|
|
|
+ new MRAppWithHistory(numMaps, numReduces, true,
|
|
|
+ this.getClass().getName(), true);
|
|
|
app.submit(conf);
|
|
|
Job job = app.getContext().getAllJobs().values().iterator().next();
|
|
|
JobId jobId = job.getID();
|
|
@@ -117,8 +137,42 @@ public class TestJobHistoryParsing {
|
|
|
}
|
|
|
|
|
|
JobHistoryParser parser = new JobHistoryParser(in);
|
|
|
- JobInfo jobInfo = parser.parse();
|
|
|
-
|
|
|
+ final EventReader realReader = new EventReader(in);
|
|
|
+ EventReader reader = Mockito.mock(EventReader.class);
|
|
|
+ if (numMaps == numSuccessfulMaps) {
|
|
|
+ reader = realReader;
|
|
|
+ } else {
|
|
|
+ final AtomicInteger numFinishedEvents = new AtomicInteger(0); // Hack!
|
|
|
+ Mockito.when(reader.getNextEvent()).thenAnswer(
|
|
|
+ new Answer<HistoryEvent>() {
|
|
|
+ public HistoryEvent answer(InvocationOnMock invocation)
|
|
|
+ throws IOException {
|
|
|
+ HistoryEvent event = realReader.getNextEvent();
|
|
|
+ if (event instanceof TaskFinishedEvent) {
|
|
|
+ numFinishedEvents.incrementAndGet();
|
|
|
+ }
|
|
|
+
|
|
|
+ if (numFinishedEvents.get() <= numSuccessfulMaps) {
|
|
|
+ return event;
|
|
|
+ } else {
|
|
|
+ throw new IOException("test");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
+ JobInfo jobInfo = parser.parse(reader);
|
|
|
+
|
|
|
+ long numFinishedMaps =
|
|
|
+ computeFinishedMaps(jobInfo, numMaps, numSuccessfulMaps);
|
|
|
+
|
|
|
+ if (numFinishedMaps != numMaps) {
|
|
|
+ Exception parseException = parser.getParseException();
|
|
|
+ Assert.assertNotNull("Didn't get expected parse exception",
|
|
|
+ parseException);
|
|
|
+ }
|
|
|
+
|
|
|
Assert.assertEquals("Incorrect username ", System.getProperty("user.name"),
|
|
|
jobInfo.getUsername());
|
|
|
Assert.assertEquals("Incorrect jobName ", "test", jobInfo.getJobname());
|
|
@@ -126,14 +180,16 @@ public class TestJobHistoryParsing {
|
|
|
jobInfo.getJobQueueName());
|
|
|
Assert
|
|
|
.assertEquals("incorrect conf path", "test", jobInfo.getJobConfPath());
|
|
|
- Assert.assertEquals("incorrect finishedMap ", 2, jobInfo.getFinishedMaps());
|
|
|
- Assert.assertEquals("incorrect finishedReduces ", 1,
|
|
|
+ Assert.assertEquals("incorrect finishedMap ", numSuccessfulMaps,
|
|
|
+ numFinishedMaps);
|
|
|
+ Assert.assertEquals("incorrect finishedReduces ", numReduces,
|
|
|
jobInfo.getFinishedReduces());
|
|
|
Assert.assertEquals("incorrect uberized ", job.isUber(),
|
|
|
jobInfo.getUberized());
|
|
|
Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
|
|
|
int totalTasks = allTasks.size();
|
|
|
- Assert.assertEquals("total number of tasks is incorrect ", 3, totalTasks);
|
|
|
+ Assert.assertEquals("total number of tasks is incorrect ",
|
|
|
+ (numMaps+numReduces), totalTasks);
|
|
|
|
|
|
// Verify aminfo
|
|
|
Assert.assertEquals(1, jobInfo.getAMInfos().size());
|
|
@@ -172,55 +228,78 @@ public class TestJobHistoryParsing {
|
|
|
Assert.assertNotNull("TaskAttemptInfo not found", taskAttemptInfo);
|
|
|
Assert.assertEquals("Incorrect shuffle port for task attempt",
|
|
|
taskAttempt.getShufflePort(), taskAttemptInfo.getShufflePort());
|
|
|
- Assert.assertEquals(MRApp.NM_HOST, taskAttemptInfo.getHostname());
|
|
|
- Assert.assertEquals(MRApp.NM_PORT, taskAttemptInfo.getPort());
|
|
|
-
|
|
|
- // Verify rack-name
|
|
|
- Assert.assertEquals("rack-name is incorrect", taskAttemptInfo
|
|
|
- .getRackname(), RACK_NAME);
|
|
|
+ if (numMaps == numSuccessfulMaps) {
|
|
|
+ Assert.assertEquals(MRApp.NM_HOST, taskAttemptInfo.getHostname());
|
|
|
+ Assert.assertEquals(MRApp.NM_PORT, taskAttemptInfo.getPort());
|
|
|
+
|
|
|
+ // Verify rack-name
|
|
|
+ Assert.assertEquals("rack-name is incorrect", taskAttemptInfo
|
|
|
+ .getRackname(), RACK_NAME);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- String summaryFileName = JobHistoryUtils
|
|
|
- .getIntermediateSummaryFileName(jobId);
|
|
|
- Path summaryFile = new Path(jobhistoryDir, summaryFileName);
|
|
|
- String jobSummaryString = jobHistory.getJobSummary(fc, summaryFile);
|
|
|
- Assert.assertTrue(jobSummaryString.contains("resourcesPerMap=100"));
|
|
|
- Assert.assertTrue(jobSummaryString.contains("resourcesPerReduce=100"));
|
|
|
- Assert.assertNotNull(jobSummaryString);
|
|
|
+ if (numMaps == numSuccessfulMaps) {
|
|
|
|
|
|
- Map<String, String> jobSummaryElements = new HashMap<String, String>();
|
|
|
- StringTokenizer strToken = new StringTokenizer(jobSummaryString, ",");
|
|
|
- while (strToken.hasMoreTokens()) {
|
|
|
- String keypair = strToken.nextToken();
|
|
|
- jobSummaryElements.put(keypair.split("=")[0], keypair.split("=")[1]);
|
|
|
+ String summaryFileName = JobHistoryUtils
|
|
|
+ .getIntermediateSummaryFileName(jobId);
|
|
|
+ Path summaryFile = new Path(jobhistoryDir, summaryFileName);
|
|
|
+ String jobSummaryString = jobHistory.getJobSummary(fc, summaryFile);
|
|
|
+ Assert.assertTrue(jobSummaryString.contains("resourcesPerMap=100"));
|
|
|
+ Assert.assertTrue(jobSummaryString.contains("resourcesPerReduce=100"));
|
|
|
+ Assert.assertNotNull(jobSummaryString);
|
|
|
|
|
|
- }
|
|
|
+ Map<String, String> jobSummaryElements = new HashMap<String, String>();
|
|
|
+ StringTokenizer strToken = new StringTokenizer(jobSummaryString, ",");
|
|
|
+ while (strToken.hasMoreTokens()) {
|
|
|
+ String keypair = strToken.nextToken();
|
|
|
+ jobSummaryElements.put(keypair.split("=")[0], keypair.split("=")[1]);
|
|
|
|
|
|
- Assert.assertEquals("JobId does not match", jobId.toString(),
|
|
|
- jobSummaryElements.get("jobId"));
|
|
|
- Assert.assertTrue("submitTime should not be 0",
|
|
|
- Long.parseLong(jobSummaryElements.get("submitTime")) != 0);
|
|
|
- Assert.assertTrue("launchTime should not be 0",
|
|
|
- Long.parseLong(jobSummaryElements.get("launchTime")) != 0);
|
|
|
- Assert.assertTrue("firstMapTaskLaunchTime should not be 0",
|
|
|
- Long.parseLong(jobSummaryElements.get("firstMapTaskLaunchTime")) != 0);
|
|
|
- Assert
|
|
|
- .assertTrue(
|
|
|
- "firstReduceTaskLaunchTime should not be 0",
|
|
|
- Long.parseLong(jobSummaryElements.get("firstReduceTaskLaunchTime")) != 0);
|
|
|
- Assert.assertTrue("finishTime should not be 0",
|
|
|
- Long.parseLong(jobSummaryElements.get("finishTime")) != 0);
|
|
|
- Assert.assertEquals("Mismatch in num map slots", 2,
|
|
|
- Integer.parseInt(jobSummaryElements.get("numMaps")));
|
|
|
- Assert.assertEquals("Mismatch in num reduce slots", 1,
|
|
|
- Integer.parseInt(jobSummaryElements.get("numReduces")));
|
|
|
- Assert.assertEquals("User does not match", System.getProperty("user.name"),
|
|
|
- jobSummaryElements.get("user"));
|
|
|
- Assert.assertEquals("Queue does not match", "default",
|
|
|
- jobSummaryElements.get("queue"));
|
|
|
- Assert.assertEquals("Status does not match", "SUCCEEDED",
|
|
|
- jobSummaryElements.get("status"));
|
|
|
+ }
|
|
|
+
|
|
|
+ Assert.assertEquals("JobId does not match", jobId.toString(),
|
|
|
+ jobSummaryElements.get("jobId"));
|
|
|
+ Assert.assertTrue("submitTime should not be 0",
|
|
|
+ Long.parseLong(jobSummaryElements.get("submitTime")) != 0);
|
|
|
+ Assert.assertTrue("launchTime should not be 0",
|
|
|
+ Long.parseLong(jobSummaryElements.get("launchTime")) != 0);
|
|
|
+ Assert.assertTrue("firstMapTaskLaunchTime should not be 0",
|
|
|
+ Long.parseLong(jobSummaryElements.get("firstMapTaskLaunchTime")) != 0);
|
|
|
+ Assert
|
|
|
+ .assertTrue(
|
|
|
+ "firstReduceTaskLaunchTime should not be 0",
|
|
|
+ Long.parseLong(jobSummaryElements.get("firstReduceTaskLaunchTime")) != 0);
|
|
|
+ Assert.assertTrue("finishTime should not be 0",
|
|
|
+ Long.parseLong(jobSummaryElements.get("finishTime")) != 0);
|
|
|
+ Assert.assertEquals("Mismatch in num map slots", numSuccessfulMaps,
|
|
|
+ Integer.parseInt(jobSummaryElements.get("numMaps")));
|
|
|
+ Assert.assertEquals("Mismatch in num reduce slots", numReduces,
|
|
|
+ Integer.parseInt(jobSummaryElements.get("numReduces")));
|
|
|
+ Assert.assertEquals("User does not match", System.getProperty("user.name"),
|
|
|
+ jobSummaryElements.get("user"));
|
|
|
+ Assert.assertEquals("Queue does not match", "default",
|
|
|
+ jobSummaryElements.get("queue"));
|
|
|
+ Assert.assertEquals("Status does not match", "SUCCEEDED",
|
|
|
+ jobSummaryElements.get("status"));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Computes finished maps similar to RecoveryService...
|
|
|
+ private long computeFinishedMaps(JobInfo jobInfo,
|
|
|
+ int numMaps, int numSuccessfulMaps) {
|
|
|
+ if (numMaps == numSuccessfulMaps) {
|
|
|
+ return jobInfo.getFinishedMaps();
|
|
|
+ }
|
|
|
+
|
|
|
+ long numFinishedMaps = 0;
|
|
|
+ Map<org.apache.hadoop.mapreduce.TaskID, TaskInfo> taskInfos =
|
|
|
+ jobInfo.getAllTasks();
|
|
|
+ for (TaskInfo taskInfo : taskInfos.values()) {
|
|
|
+ if (TaskState.SUCCEEDED.toString().equals(taskInfo.getTaskStatus())) {
|
|
|
+ ++numFinishedMaps;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return numFinishedMaps;
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -264,6 +343,9 @@ public class TestJobHistoryParsing {
|
|
|
|
|
|
JobHistoryParser parser = new JobHistoryParser(in);
|
|
|
JobInfo jobInfo = parser.parse();
|
|
|
+ Exception parseException = parser.getParseException();
|
|
|
+ Assert.assertNull("Caught an expected exception " + parseException,
|
|
|
+ parseException);
|
|
|
int noOffailedAttempts = 0;
|
|
|
Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
|
|
|
for (Task task : job.getTasks().values()) {
|