|
@@ -19,6 +19,9 @@
|
|
|
package org.apache.hadoop.mapreduce.v2.hs;
|
|
|
|
|
|
import java.io.ByteArrayOutputStream;
|
|
|
+import static org.junit.Assert.assertNull;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+
|
|
|
import java.io.IOException;
|
|
|
import java.io.PrintStream;
|
|
|
import java.util.Arrays;
|
|
@@ -54,6 +57,9 @@ 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.TaskId;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.JobIdPBImpl;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.TaskIdPBImpl;
|
|
|
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;
|
|
@@ -65,7 +71,9 @@ import org.apache.hadoop.mapreduce.v2.hs.TestJobHistoryEvents.MRAppWithHistory;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
|
|
|
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
|
|
|
import org.apache.hadoop.net.DNSToSwitchMapping;
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.service.Service;
|
|
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
@@ -80,12 +88,12 @@ public class TestJobHistoryParsing {
|
|
|
|
|
|
private static final String RACK_NAME = "/MyRackName";
|
|
|
|
|
|
- private ByteArrayOutputStream outContent = new ByteArrayOutputStream();
|
|
|
+ private ByteArrayOutputStream outContent = new ByteArrayOutputStream();
|
|
|
|
|
|
public static class MyResolver implements DNSToSwitchMapping {
|
|
|
@Override
|
|
|
public List<String> resolve(List<String> names) {
|
|
|
- return Arrays.asList(new String[]{RACK_NAME});
|
|
|
+ return Arrays.asList(new String[] { RACK_NAME });
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -93,14 +101,14 @@ public class TestJobHistoryParsing {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test (timeout=50000)
|
|
|
+ @Test(timeout = 50000)
|
|
|
public void testJobInfo() throws Exception {
|
|
|
JobInfo info = new JobInfo();
|
|
|
Assert.assertEquals("NORMAL", info.getPriority());
|
|
|
info.printAll();
|
|
|
}
|
|
|
|
|
|
- @Test (timeout=50000)
|
|
|
+ @Test(timeout = 300000)
|
|
|
public void testHistoryParsing() throws Exception {
|
|
|
LOG.info("STARTING testHistoryParsing()");
|
|
|
try {
|
|
@@ -109,8 +117,8 @@ public class TestJobHistoryParsing {
|
|
|
LOG.info("FINISHED testHistoryParsing()");
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- @Test (timeout=50000)
|
|
|
+
|
|
|
+ @Test(timeout = 50000)
|
|
|
public void testHistoryParsingWithParseErrors() throws Exception {
|
|
|
LOG.info("STARTING testHistoryParsingWithParseErrors()");
|
|
|
try {
|
|
@@ -119,18 +127,18 @@ public class TestJobHistoryParsing {
|
|
|
LOG.info("FINISHED testHistoryParsingWithParseErrors()");
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- private static String getJobSummary(FileContext fc, Path path) throws IOException {
|
|
|
+
|
|
|
+ private static String getJobSummary(FileContext fc, Path path)
|
|
|
+ throws IOException {
|
|
|
Path qPath = fc.makeQualified(path);
|
|
|
FSDataInputStream in = fc.open(qPath);
|
|
|
String jobSummaryString = in.readUTF();
|
|
|
in.close();
|
|
|
return jobSummaryString;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private void checkHistoryParsing(final int numMaps, final int numReduces,
|
|
|
- final int numSuccessfulMaps)
|
|
|
- throws Exception {
|
|
|
+ final int numSuccessfulMaps) throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
|
|
|
long amStartTimeEst = System.currentTimeMillis();
|
|
@@ -138,9 +146,8 @@ public class TestJobHistoryParsing {
|
|
|
CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
MyResolver.class, DNSToSwitchMapping.class);
|
|
|
RackResolver.init(conf);
|
|
|
- MRApp app =
|
|
|
- new MRAppWithHistory(numMaps, numReduces, 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();
|
|
@@ -152,7 +159,7 @@ public class TestJobHistoryParsing {
|
|
|
|
|
|
String jobhistoryDir = JobHistoryUtils
|
|
|
.getHistoryIntermediateDoneDirForUser(conf);
|
|
|
-
|
|
|
+
|
|
|
FileContext fc = null;
|
|
|
try {
|
|
|
fc = FileContext.getFileContext(conf);
|
|
@@ -160,7 +167,7 @@ public class TestJobHistoryParsing {
|
|
|
LOG.info("Can not get FileContext", ioe);
|
|
|
throw (new Exception("Can not get File Context"));
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
if (numMaps == numSuccessfulMaps) {
|
|
|
String summaryFileName = JobHistoryUtils
|
|
|
.getIntermediateSummaryFileName(jobId);
|
|
@@ -185,20 +192,22 @@ public class TestJobHistoryParsing {
|
|
|
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);
|
|
|
+ .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("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",
|
|
@@ -210,8 +219,8 @@ public class TestJobHistoryParsing {
|
|
|
HistoryFileInfo fileInfo = jobHistory.getJobFileInfo(jobId);
|
|
|
JobInfo jobInfo;
|
|
|
long numFinishedMaps;
|
|
|
-
|
|
|
- synchronized(fileInfo) {
|
|
|
+
|
|
|
+ synchronized (fileInfo) {
|
|
|
Path historyFilePath = fileInfo.getHistoryFile();
|
|
|
FSDataInputStream in = null;
|
|
|
LOG.info("JobHistoryFile is: " + historyFilePath);
|
|
@@ -228,11 +237,11 @@ public class TestJobHistoryParsing {
|
|
|
if (numMaps == numSuccessfulMaps) {
|
|
|
reader = realReader;
|
|
|
} else {
|
|
|
- final AtomicInteger numFinishedEvents = new AtomicInteger(0); // Hack!
|
|
|
+ final AtomicInteger numFinishedEvents = new AtomicInteger(0); // Hack!
|
|
|
Mockito.when(reader.getNextEvent()).thenAnswer(
|
|
|
new Answer<HistoryEvent>() {
|
|
|
- public HistoryEvent answer(InvocationOnMock invocation)
|
|
|
- throws IOException {
|
|
|
+ public HistoryEvent answer(InvocationOnMock invocation)
|
|
|
+ throws IOException {
|
|
|
HistoryEvent event = realReader.getNextEvent();
|
|
|
if (event instanceof TaskFinishedEvent) {
|
|
|
numFinishedEvents.incrementAndGet();
|
|
@@ -244,22 +253,20 @@ public class TestJobHistoryParsing {
|
|
|
throw new IOException("test");
|
|
|
}
|
|
|
}
|
|
|
- }
|
|
|
- );
|
|
|
+ });
|
|
|
}
|
|
|
|
|
|
jobInfo = parser.parse(reader);
|
|
|
|
|
|
- numFinishedMaps =
|
|
|
- computeFinishedMaps(jobInfo, numMaps, numSuccessfulMaps);
|
|
|
+ numFinishedMaps = computeFinishedMaps(jobInfo, numMaps, numSuccessfulMaps);
|
|
|
|
|
|
if (numFinishedMaps != numMaps) {
|
|
|
Exception parseException = parser.getParseException();
|
|
|
- Assert.assertNotNull("Didn't get expected parse exception",
|
|
|
+ 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());
|
|
@@ -267,7 +274,7 @@ public class TestJobHistoryParsing {
|
|
|
jobInfo.getJobQueueName());
|
|
|
Assert
|
|
|
.assertEquals("incorrect conf path", "test", jobInfo.getJobConfPath());
|
|
|
- Assert.assertEquals("incorrect finishedMap ", numSuccessfulMaps,
|
|
|
+ Assert.assertEquals("incorrect finishedMap ", numSuccessfulMaps,
|
|
|
numFinishedMaps);
|
|
|
Assert.assertEquals("incorrect finishedReduces ", numReduces,
|
|
|
jobInfo.getFinishedReduces());
|
|
@@ -275,8 +282,8 @@ public class TestJobHistoryParsing {
|
|
|
jobInfo.getUberized());
|
|
|
Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
|
|
|
int totalTasks = allTasks.size();
|
|
|
- Assert.assertEquals("total number of tasks is incorrect ",
|
|
|
- (numMaps+numReduces), totalTasks);
|
|
|
+ Assert.assertEquals("total number of tasks is incorrect ",
|
|
|
+ (numMaps + numReduces), totalTasks);
|
|
|
|
|
|
// Verify aminfo
|
|
|
Assert.assertEquals(1, jobInfo.getAMInfos().size());
|
|
@@ -306,8 +313,7 @@ public class TestJobHistoryParsing {
|
|
|
|
|
|
// Deep compare Job and JobInfo
|
|
|
for (Task task : job.getTasks().values()) {
|
|
|
- TaskInfo taskInfo = allTasks.get(
|
|
|
- TypeConverter.fromYarn(task.getID()));
|
|
|
+ TaskInfo taskInfo = allTasks.get(TypeConverter.fromYarn(task.getID()));
|
|
|
Assert.assertNotNull("TaskInfo not found", taskInfo);
|
|
|
for (TaskAttempt taskAttempt : task.getAttempts().values()) {
|
|
|
TaskAttemptInfo taskAttemptInfo = taskInfo.getAllTaskAttempts().get(
|
|
@@ -318,27 +324,32 @@ public class TestJobHistoryParsing {
|
|
|
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);
|
|
|
+ Assert.assertEquals("rack-name is incorrect",
|
|
|
+ taskAttemptInfo.getRackname(), RACK_NAME);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
// test output for HistoryViewer
|
|
|
- PrintStream stdps=System.out;
|
|
|
+ PrintStream stdps = System.out;
|
|
|
try {
|
|
|
System.setOut(new PrintStream(outContent));
|
|
|
HistoryViewer viewer = new HistoryViewer(fc.makeQualified(
|
|
|
fileInfo.getHistoryFile()).toString(), conf, true);
|
|
|
viewer.print();
|
|
|
-
|
|
|
- for (TaskInfo taskInfo : allTasks.values()) {
|
|
|
-
|
|
|
- String test= (taskInfo.getTaskStatus()==null?"":taskInfo.getTaskStatus())+" "+taskInfo.getTaskType()+" task list for "+taskInfo.getTaskId().getJobID();
|
|
|
- Assert.assertTrue(outContent.toString().indexOf(test)>0);
|
|
|
- Assert.assertTrue(outContent.toString().indexOf(taskInfo.getTaskId().toString())>0);
|
|
|
+
|
|
|
+ for (TaskInfo taskInfo : allTasks.values()) {
|
|
|
+
|
|
|
+ String test = (taskInfo.getTaskStatus() == null ? "" : taskInfo
|
|
|
+ .getTaskStatus())
|
|
|
+ + " "
|
|
|
+ + taskInfo.getTaskType()
|
|
|
+ + " task list for " + taskInfo.getTaskId().getJobID();
|
|
|
+ Assert.assertTrue(outContent.toString().indexOf(test) > 0);
|
|
|
+ Assert.assertTrue(outContent.toString().indexOf(
|
|
|
+ taskInfo.getTaskId().toString()) > 0);
|
|
|
}
|
|
|
} finally {
|
|
|
System.setOut(stdps);
|
|
@@ -363,186 +374,180 @@ public class TestJobHistoryParsing {
|
|
|
}
|
|
|
return numFinishedMaps;
|
|
|
}
|
|
|
-
|
|
|
- @Test (timeout=50000)
|
|
|
+
|
|
|
+ @Test(timeout = 30000)
|
|
|
public void testHistoryParsingForFailedAttempts() throws Exception {
|
|
|
LOG.info("STARTING testHistoryParsingForFailedAttempts");
|
|
|
try {
|
|
|
- Configuration conf = new Configuration();
|
|
|
- conf
|
|
|
- .setClass(
|
|
|
- CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
- MyResolver.class, DNSToSwitchMapping.class);
|
|
|
- RackResolver.init(conf);
|
|
|
- MRApp app = new MRAppWithHistoryWithFailedAttempt(2, 1, true, this.getClass().getName(),
|
|
|
- true);
|
|
|
- app.submit(conf);
|
|
|
- Job job = app.getContext().getAllJobs().values().iterator().next();
|
|
|
- JobId jobId = job.getID();
|
|
|
- app.waitForState(job, JobState.SUCCEEDED);
|
|
|
-
|
|
|
- // make sure all events are flushed
|
|
|
- app.waitForState(Service.STATE.STOPPED);
|
|
|
-
|
|
|
- String jobhistoryDir = JobHistoryUtils
|
|
|
- .getHistoryIntermediateDoneDirForUser(conf);
|
|
|
- JobHistory jobHistory = new JobHistory();
|
|
|
- jobHistory.init(conf);
|
|
|
-
|
|
|
- JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
|
|
|
- .getJobIndexInfo();
|
|
|
- String jobhistoryFileName = FileNameIndexUtils
|
|
|
- .getDoneFileName(jobIndexInfo);
|
|
|
-
|
|
|
- Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
|
|
|
- FSDataInputStream in = null;
|
|
|
- FileContext fc = null;
|
|
|
- try {
|
|
|
- fc = FileContext.getFileContext(conf);
|
|
|
- in = fc.open(fc.makeQualified(historyFilePath));
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.info("Can not open history file: " + historyFilePath, ioe);
|
|
|
- throw (new Exception("Can not open History File"));
|
|
|
- }
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setClass(
|
|
|
+ CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
+ MyResolver.class, DNSToSwitchMapping.class);
|
|
|
+ RackResolver.init(conf);
|
|
|
+ MRApp app = new MRAppWithHistoryWithFailedAttempt(2, 1, true, this
|
|
|
+ .getClass().getName(), true);
|
|
|
+ app.submit(conf);
|
|
|
+ Job job = app.getContext().getAllJobs().values().iterator().next();
|
|
|
+ JobId jobId = job.getID();
|
|
|
+ app.waitForState(job, JobState.SUCCEEDED);
|
|
|
+
|
|
|
+ // make sure all events are flushed
|
|
|
+ app.waitForState(Service.STATE.STOPPED);
|
|
|
+
|
|
|
+ String jobhistoryDir = JobHistoryUtils
|
|
|
+ .getHistoryIntermediateDoneDirForUser(conf);
|
|
|
+ JobHistory jobHistory = new JobHistory();
|
|
|
+ jobHistory.init(conf);
|
|
|
+
|
|
|
+ JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
|
|
|
+ .getJobIndexInfo();
|
|
|
+ String jobhistoryFileName = FileNameIndexUtils
|
|
|
+ .getDoneFileName(jobIndexInfo);
|
|
|
+
|
|
|
+ Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
|
|
|
+ FSDataInputStream in = null;
|
|
|
+ FileContext fc = null;
|
|
|
+ try {
|
|
|
+ fc = FileContext.getFileContext(conf);
|
|
|
+ in = fc.open(fc.makeQualified(historyFilePath));
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.info("Can not open history file: " + historyFilePath, ioe);
|
|
|
+ throw (new Exception("Can not open History File"));
|
|
|
+ }
|
|
|
|
|
|
- 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()) {
|
|
|
- TaskInfo taskInfo = allTasks.get(TypeConverter.fromYarn(task.getID()));
|
|
|
- for (TaskAttempt taskAttempt : task.getAttempts().values()) {
|
|
|
- TaskAttemptInfo taskAttemptInfo = taskInfo.getAllTaskAttempts().get(
|
|
|
- TypeConverter.fromYarn((taskAttempt.getID())));
|
|
|
- // Verify rack-name for all task attempts
|
|
|
- Assert.assertEquals("rack-name is incorrect", taskAttemptInfo
|
|
|
- .getRackname(), RACK_NAME);
|
|
|
- if (taskAttemptInfo.getTaskStatus().equals("FAILED")) {
|
|
|
- noOffailedAttempts++;
|
|
|
+ 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()) {
|
|
|
+ TaskInfo taskInfo = allTasks.get(TypeConverter.fromYarn(task.getID()));
|
|
|
+ for (TaskAttempt taskAttempt : task.getAttempts().values()) {
|
|
|
+ TaskAttemptInfo taskAttemptInfo = taskInfo.getAllTaskAttempts().get(
|
|
|
+ TypeConverter.fromYarn((taskAttempt.getID())));
|
|
|
+ // Verify rack-name for all task attempts
|
|
|
+ Assert.assertEquals("rack-name is incorrect",
|
|
|
+ taskAttemptInfo.getRackname(), RACK_NAME);
|
|
|
+ if (taskAttemptInfo.getTaskStatus().equals("FAILED")) {
|
|
|
+ noOffailedAttempts++;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
- }
|
|
|
- Assert.assertEquals("No of Failed tasks doesn't match.", 2, noOffailedAttempts);
|
|
|
+ Assert.assertEquals("No of Failed tasks doesn't match.", 2,
|
|
|
+ noOffailedAttempts);
|
|
|
} finally {
|
|
|
LOG.info("FINISHED testHistoryParsingForFailedAttempts");
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- @Test (timeout=5000)
|
|
|
+
|
|
|
+ @Test(timeout = 60000)
|
|
|
public void testCountersForFailedTask() throws Exception {
|
|
|
LOG.info("STARTING testCountersForFailedTask");
|
|
|
try {
|
|
|
- Configuration conf = new Configuration();
|
|
|
- conf
|
|
|
- .setClass(
|
|
|
- CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
- MyResolver.class, DNSToSwitchMapping.class);
|
|
|
- RackResolver.init(conf);
|
|
|
- MRApp app = new MRAppWithHistoryWithFailedTask(2, 1, true,
|
|
|
- this.getClass().getName(), true);
|
|
|
- app.submit(conf);
|
|
|
- Job job = app.getContext().getAllJobs().values().iterator().next();
|
|
|
- JobId jobId = job.getID();
|
|
|
- app.waitForState(job, JobState.FAILED);
|
|
|
-
|
|
|
- // make sure all events are flushed
|
|
|
- app.waitForState(Service.STATE.STOPPED);
|
|
|
-
|
|
|
- String jobhistoryDir = JobHistoryUtils
|
|
|
- .getHistoryIntermediateDoneDirForUser(conf);
|
|
|
- JobHistory jobHistory = new JobHistory();
|
|
|
- jobHistory.init(conf);
|
|
|
-
|
|
|
- JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
|
|
|
- .getJobIndexInfo();
|
|
|
- String jobhistoryFileName = FileNameIndexUtils
|
|
|
- .getDoneFileName(jobIndexInfo);
|
|
|
-
|
|
|
- Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
|
|
|
- FSDataInputStream in = null;
|
|
|
- FileContext fc = null;
|
|
|
- try {
|
|
|
- fc = FileContext.getFileContext(conf);
|
|
|
- in = fc.open(fc.makeQualified(historyFilePath));
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.info("Can not open history file: " + historyFilePath, ioe);
|
|
|
- throw (new Exception("Can not open History File"));
|
|
|
- }
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setClass(
|
|
|
+ CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
+ MyResolver.class, DNSToSwitchMapping.class);
|
|
|
+ RackResolver.init(conf);
|
|
|
+ MRApp app = new MRAppWithHistoryWithFailedTask(2, 1, true, this
|
|
|
+ .getClass().getName(), true);
|
|
|
+ app.submit(conf);
|
|
|
+ Job job = app.getContext().getAllJobs().values().iterator().next();
|
|
|
+ JobId jobId = job.getID();
|
|
|
+ app.waitForState(job, JobState.FAILED);
|
|
|
+
|
|
|
+ // make sure all events are flushed
|
|
|
+ app.waitForState(Service.STATE.STOPPED);
|
|
|
+
|
|
|
+ String jobhistoryDir = JobHistoryUtils
|
|
|
+ .getHistoryIntermediateDoneDirForUser(conf);
|
|
|
+ JobHistory jobHistory = new JobHistory();
|
|
|
+ jobHistory.init(conf);
|
|
|
+
|
|
|
+ JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
|
|
|
+ .getJobIndexInfo();
|
|
|
+ String jobhistoryFileName = FileNameIndexUtils
|
|
|
+ .getDoneFileName(jobIndexInfo);
|
|
|
+
|
|
|
+ Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
|
|
|
+ FSDataInputStream in = null;
|
|
|
+ FileContext fc = null;
|
|
|
+ try {
|
|
|
+ fc = FileContext.getFileContext(conf);
|
|
|
+ in = fc.open(fc.makeQualified(historyFilePath));
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.info("Can not open history file: " + historyFilePath, ioe);
|
|
|
+ throw (new Exception("Can not open History File"));
|
|
|
+ }
|
|
|
|
|
|
- JobHistoryParser parser = new JobHistoryParser(in);
|
|
|
- JobInfo jobInfo = parser.parse();
|
|
|
- Exception parseException = parser.getParseException();
|
|
|
- Assert.assertNull("Caught an expected exception " + parseException,
|
|
|
- parseException);
|
|
|
- for (Map.Entry<TaskID,TaskInfo> entry : jobInfo.getAllTasks().entrySet()) {
|
|
|
- TaskId yarnTaskID = TypeConverter.toYarn(entry.getKey());
|
|
|
- CompletedTask ct = new CompletedTask(yarnTaskID, entry.getValue());
|
|
|
- Assert.assertNotNull("completed task report has null counters",
|
|
|
- ct.getReport().getCounters());
|
|
|
- //Make sure all the completedTask has counters, and the counters are not empty
|
|
|
- Assert.assertTrue(ct.getReport().getCounters()
|
|
|
- .getAllCounterGroups().size() > 0);
|
|
|
- }
|
|
|
+ JobHistoryParser parser = new JobHistoryParser(in);
|
|
|
+ JobInfo jobInfo = parser.parse();
|
|
|
+ Exception parseException = parser.getParseException();
|
|
|
+ Assert.assertNull("Caught an expected exception " + parseException,
|
|
|
+ parseException);
|
|
|
+ for (Map.Entry<TaskID, TaskInfo> entry : jobInfo.getAllTasks().entrySet()) {
|
|
|
+ TaskId yarnTaskID = TypeConverter.toYarn(entry.getKey());
|
|
|
+ CompletedTask ct = new CompletedTask(yarnTaskID, entry.getValue());
|
|
|
+ Assert.assertNotNull("completed task report has null counters", ct
|
|
|
+ .getReport().getCounters());
|
|
|
+ }
|
|
|
} finally {
|
|
|
LOG.info("FINISHED testCountersForFailedTask");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test (timeout=50000)
|
|
|
+ @Test(timeout = 50000)
|
|
|
public void testScanningOldDirs() throws Exception {
|
|
|
LOG.info("STARTING testScanningOldDirs");
|
|
|
try {
|
|
|
- Configuration conf = new Configuration();
|
|
|
- conf
|
|
|
- .setClass(
|
|
|
- CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
- MyResolver.class, DNSToSwitchMapping.class);
|
|
|
- RackResolver.init(conf);
|
|
|
- MRApp app =
|
|
|
- new MRAppWithHistory(1, 1, true,
|
|
|
- this.getClass().getName(), true);
|
|
|
- app.submit(conf);
|
|
|
- Job job = app.getContext().getAllJobs().values().iterator().next();
|
|
|
- JobId jobId = job.getID();
|
|
|
- LOG.info("JOBID is " + TypeConverter.fromYarn(jobId).toString());
|
|
|
- app.waitForState(job, JobState.SUCCEEDED);
|
|
|
-
|
|
|
- // make sure all events are flushed
|
|
|
- app.waitForState(Service.STATE.STOPPED);
|
|
|
-
|
|
|
- HistoryFileManagerForTest hfm = new HistoryFileManagerForTest();
|
|
|
- hfm.init(conf);
|
|
|
- HistoryFileInfo fileInfo = hfm.getFileInfo(jobId);
|
|
|
- Assert.assertNotNull("Unable to locate job history", fileInfo);
|
|
|
-
|
|
|
- // force the manager to "forget" the job
|
|
|
- hfm.deleteJobFromJobListCache(fileInfo);
|
|
|
- final int msecPerSleep = 10;
|
|
|
- int msecToSleep = 10 * 1000;
|
|
|
- while (fileInfo.isMovePending() && msecToSleep > 0) {
|
|
|
- Assert.assertTrue(!fileInfo.didMoveFail());
|
|
|
- msecToSleep -= msecPerSleep;
|
|
|
- Thread.sleep(msecPerSleep);
|
|
|
- }
|
|
|
- Assert.assertTrue("Timeout waiting for history move", msecToSleep > 0);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setClass(
|
|
|
+ CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
+ MyResolver.class, DNSToSwitchMapping.class);
|
|
|
+ RackResolver.init(conf);
|
|
|
+ MRApp app = new MRAppWithHistory(1, 1, true, this.getClass().getName(),
|
|
|
+ true);
|
|
|
+ app.submit(conf);
|
|
|
+ Job job = app.getContext().getAllJobs().values().iterator().next();
|
|
|
+ JobId jobId = job.getID();
|
|
|
+ LOG.info("JOBID is " + TypeConverter.fromYarn(jobId).toString());
|
|
|
+ app.waitForState(job, JobState.SUCCEEDED);
|
|
|
+
|
|
|
+ // make sure all events are flushed
|
|
|
+ app.waitForState(Service.STATE.STOPPED);
|
|
|
+
|
|
|
+ HistoryFileManagerForTest hfm = new HistoryFileManagerForTest();
|
|
|
+ hfm.init(conf);
|
|
|
+ HistoryFileInfo fileInfo = hfm.getFileInfo(jobId);
|
|
|
+ Assert.assertNotNull("Unable to locate job history", fileInfo);
|
|
|
+
|
|
|
+ // force the manager to "forget" the job
|
|
|
+ hfm.deleteJobFromJobListCache(fileInfo);
|
|
|
+ final int msecPerSleep = 10;
|
|
|
+ int msecToSleep = 10 * 1000;
|
|
|
+ while (fileInfo.isMovePending() && msecToSleep > 0) {
|
|
|
+ Assert.assertTrue(!fileInfo.didMoveFail());
|
|
|
+ msecToSleep -= msecPerSleep;
|
|
|
+ Thread.sleep(msecPerSleep);
|
|
|
+ }
|
|
|
+ Assert.assertTrue("Timeout waiting for history move", msecToSleep > 0);
|
|
|
|
|
|
- fileInfo = hfm.getFileInfo(jobId);
|
|
|
- Assert.assertNotNull("Unable to locate old job history", fileInfo);
|
|
|
- } finally {
|
|
|
+ fileInfo = hfm.getFileInfo(jobId);
|
|
|
+ Assert.assertNotNull("Unable to locate old job history", fileInfo);
|
|
|
+ } finally {
|
|
|
LOG.info("FINISHED testScanningOldDirs");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
static class MRAppWithHistoryWithFailedAttempt extends MRAppWithHistory {
|
|
|
|
|
|
- public MRAppWithHistoryWithFailedAttempt(int maps, int reduces, boolean autoComplete,
|
|
|
- String testName, boolean cleanOnStart) {
|
|
|
+ public MRAppWithHistoryWithFailedAttempt(int maps, int reduces,
|
|
|
+ boolean autoComplete, String testName, boolean cleanOnStart) {
|
|
|
super(maps, reduces, autoComplete, testName, cleanOnStart);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
protected void attemptLaunched(TaskAttemptId attemptID) {
|
|
@@ -558,8 +563,8 @@ public class TestJobHistoryParsing {
|
|
|
|
|
|
static class MRAppWithHistoryWithFailedTask extends MRAppWithHistory {
|
|
|
|
|
|
- public MRAppWithHistoryWithFailedTask(int maps, int reduces, boolean autoComplete,
|
|
|
- String testName, boolean cleanOnStart) {
|
|
|
+ public MRAppWithHistoryWithFailedTask(int maps, int reduces,
|
|
|
+ boolean autoComplete, String testName, boolean cleanOnStart) {
|
|
|
super(maps, reduces, autoComplete, testName, cleanOnStart);
|
|
|
}
|
|
|
|
|
@@ -587,4 +592,133 @@ public class TestJobHistoryParsing {
|
|
|
t.testHistoryParsing();
|
|
|
t.testHistoryParsingForFailedAttempts();
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test clean old history files. Files should be deleted after 1 week by
|
|
|
+ * default.
|
|
|
+ */
|
|
|
+ @Test(timeout = 15000)
|
|
|
+ public void testDeleteFileInfo() throws Exception {
|
|
|
+ LOG.info("STARTING testDeleteFileInfo");
|
|
|
+ try {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+
|
|
|
+ conf.setClass(
|
|
|
+ CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
+ MyResolver.class, DNSToSwitchMapping.class);
|
|
|
+
|
|
|
+ RackResolver.init(conf);
|
|
|
+ MRApp app = new MRAppWithHistory(1, 1, true, this.getClass().getName(),
|
|
|
+ true);
|
|
|
+ app.submit(conf);
|
|
|
+ Job job = app.getContext().getAllJobs().values().iterator().next();
|
|
|
+ JobId jobId = job.getID();
|
|
|
+
|
|
|
+ app.waitForState(job, JobState.SUCCEEDED);
|
|
|
+
|
|
|
+ // make sure all events are flushed
|
|
|
+ app.waitForState(Service.STATE.STOPPED);
|
|
|
+
|
|
|
+ HistoryFileManager hfm = new HistoryFileManager();
|
|
|
+ hfm.init(conf);
|
|
|
+ HistoryFileInfo fileInfo = hfm.getFileInfo(jobId);
|
|
|
+ hfm.initExisting();
|
|
|
+ // wait for move files form the done_intermediate directory to the gone
|
|
|
+ // directory
|
|
|
+ while (fileInfo.isMovePending()) {
|
|
|
+ Thread.sleep(300);
|
|
|
+ }
|
|
|
+
|
|
|
+ Assert.assertNotNull(hfm.jobListCache.values());
|
|
|
+
|
|
|
+ // try to remove fileInfo
|
|
|
+ hfm.clean();
|
|
|
+ // check that fileInfo does not deleted
|
|
|
+ Assert.assertFalse(fileInfo.isDeleted());
|
|
|
+ // correct live time
|
|
|
+ hfm.setMaxHistoryAge(-1);
|
|
|
+ hfm.clean();
|
|
|
+ // should be deleted !
|
|
|
+ Assert.assertTrue("file should be deleted ", fileInfo.isDeleted());
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ LOG.info("FINISHED testDeleteFileInfo");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Simple test some methods of JobHistory
|
|
|
+ */
|
|
|
+ @Test(timeout = 20000)
|
|
|
+ public void testJobHistoryMethods() throws Exception {
|
|
|
+ LOG.info("STARTING testJobHistoryMethods");
|
|
|
+ try {
|
|
|
+ Configuration configuration = new Configuration();
|
|
|
+ configuration
|
|
|
+ .setClass(
|
|
|
+ CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
+ MyResolver.class, DNSToSwitchMapping.class);
|
|
|
+
|
|
|
+ RackResolver.init(configuration);
|
|
|
+ MRApp app = new MRAppWithHistory(1, 1, true, this.getClass().getName(),
|
|
|
+ true);
|
|
|
+ app.submit(configuration);
|
|
|
+ Job job = app.getContext().getAllJobs().values().iterator().next();
|
|
|
+ JobId jobId = job.getID();
|
|
|
+ LOG.info("JOBID is " + TypeConverter.fromYarn(jobId).toString());
|
|
|
+ app.waitForState(job, JobState.SUCCEEDED);
|
|
|
+
|
|
|
+ JobHistory jobHistory = new JobHistory();
|
|
|
+ jobHistory.init(configuration);
|
|
|
+ // Method getAllJobs
|
|
|
+ Assert.assertEquals(1, jobHistory.getAllJobs().size());
|
|
|
+ // and with ApplicationId
|
|
|
+ Assert.assertEquals(1, jobHistory.getAllJobs(app.getAppID()).size());
|
|
|
+
|
|
|
+ JobsInfo jobsinfo = jobHistory.getPartialJobs(0L, 10L, null, "default",
|
|
|
+ 0L, System.currentTimeMillis() + 1, 0L,
|
|
|
+ System.currentTimeMillis() + 1, JobState.SUCCEEDED);
|
|
|
+
|
|
|
+ Assert.assertEquals(1, jobsinfo.getJobs().size());
|
|
|
+ Assert.assertNotNull(jobHistory.getApplicationAttemptId());
|
|
|
+ // test Application Id
|
|
|
+ Assert.assertEquals("application_0_0000", jobHistory.getApplicationID()
|
|
|
+ .toString());
|
|
|
+ Assert
|
|
|
+ .assertEquals("Job History Server", jobHistory.getApplicationName());
|
|
|
+ // method does not work
|
|
|
+ Assert.assertNull(jobHistory.getEventHandler());
|
|
|
+ // method does not work
|
|
|
+ Assert.assertNull(jobHistory.getClock());
|
|
|
+ // method does not work
|
|
|
+ Assert.assertNull(jobHistory.getClusterInfo());
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ LOG.info("FINISHED testJobHistoryMethods");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Simple test PartialJob
|
|
|
+ */
|
|
|
+ @Test(timeout = 1000)
|
|
|
+ public void testPartialJob() throws Exception {
|
|
|
+ JobId jobId = new JobIdPBImpl();
|
|
|
+ jobId.setId(0);
|
|
|
+ JobIndexInfo jii = new JobIndexInfo(0L, System.currentTimeMillis(), "user",
|
|
|
+ "jobName", jobId, 3, 2, "JobStatus");
|
|
|
+ PartialJob test = new PartialJob(jii, jobId);
|
|
|
+
|
|
|
+ Assert.assertEquals(1.0f, test.getProgress(), 0.001f);
|
|
|
+ assertNull(test.getAllCounters());
|
|
|
+ assertNull(test.getTasks());
|
|
|
+ assertNull(test.getTasks(TaskType.MAP));
|
|
|
+ assertNull(test.getTask(new TaskIdPBImpl()));
|
|
|
+
|
|
|
+ assertNull(test.getTaskAttemptCompletionEvents(0, 100));
|
|
|
+ assertNull(test.getMapAttemptCompletionEvents(0, 100));
|
|
|
+ assertTrue(test.checkAccess(UserGroupInformation.getCurrentUser(), null));
|
|
|
+ assertNull(test.getAMInfos());
|
|
|
+
|
|
|
+ }
|
|
|
}
|