|
@@ -18,13 +18,15 @@
|
|
|
|
|
|
package org.apache.hadoop.mapreduce.v2.hs;
|
|
|
|
|
|
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic
|
|
|
- .NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY;
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.
|
|
|
+ NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY;
|
|
|
import static org.assertj.core.api.Assertions.assertThat;
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertNotNull;
|
|
|
-import static org.junit.Assert.assertNull;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertFalse;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertNotNull;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertNull;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertTrue;
|
|
|
|
|
|
import java.io.ByteArrayOutputStream;
|
|
|
import java.io.IOException;
|
|
@@ -38,7 +40,6 @@ import java.util.StringTokenizer;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
|
|
|
-import org.junit.Assert;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
@@ -92,7 +93,8 @@ import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.service.Service;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.util.RackResolver;
|
|
|
-import org.junit.Test;
|
|
|
+import org.junit.jupiter.api.Test;
|
|
|
+import org.junit.jupiter.api.Timeout;
|
|
|
import org.mockito.Mockito;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
@@ -122,14 +124,16 @@ public class TestJobHistoryParsing {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test(timeout = 50000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 50)
|
|
|
public void testJobInfo() throws Exception {
|
|
|
JobInfo info = new JobInfo();
|
|
|
- Assert.assertEquals("NORMAL", info.getPriority());
|
|
|
+ assertEquals("NORMAL", info.getPriority());
|
|
|
info.printAll();
|
|
|
}
|
|
|
|
|
|
- @Test(timeout = 300000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 300)
|
|
|
public void testHistoryParsing() throws Exception {
|
|
|
LOG.info("STARTING testHistoryParsing()");
|
|
|
try {
|
|
@@ -139,7 +143,8 @@ public class TestJobHistoryParsing {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test(timeout = 50000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 50)
|
|
|
public void testHistoryParsingWithParseErrors() throws Exception {
|
|
|
LOG.info("STARTING testHistoryParsingWithParseErrors()");
|
|
|
try {
|
|
@@ -194,9 +199,9 @@ public class TestJobHistoryParsing {
|
|
|
.getIntermediateSummaryFileName(jobId);
|
|
|
Path summaryFile = new Path(jobhistoryDir, summaryFileName);
|
|
|
String jobSummaryString = getJobSummary(fc, summaryFile);
|
|
|
- Assert.assertNotNull(jobSummaryString);
|
|
|
- Assert.assertTrue(jobSummaryString.contains("resourcesPerMap=100"));
|
|
|
- Assert.assertTrue(jobSummaryString.contains("resourcesPerReduce=100"));
|
|
|
+ assertNotNull(jobSummaryString);
|
|
|
+ assertTrue(jobSummaryString.contains("resourcesPerMap=100"));
|
|
|
+ assertTrue(jobSummaryString.contains("resourcesPerReduce=100"));
|
|
|
|
|
|
Map<String, String> jobSummaryElements = new HashMap<String, String>();
|
|
|
StringTokenizer strToken = new StringTokenizer(jobSummaryString, ",");
|
|
@@ -205,34 +210,27 @@ public class TestJobHistoryParsing {
|
|
|
jobSummaryElements.put(keypair.split("=")[0], keypair.split("=")[1]);
|
|
|
}
|
|
|
|
|
|
- Assert.assertEquals("JobId does not match", jobId.toString(),
|
|
|
- jobSummaryElements.get("jobId"));
|
|
|
- Assert.assertEquals("JobName does not match", "test",
|
|
|
- jobSummaryElements.get("jobName"));
|
|
|
- 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"));
|
|
|
+ assertEquals(jobId.toString(), jobSummaryElements.get("jobId"),
|
|
|
+ "JobId does not match");
|
|
|
+ assertEquals("test", jobSummaryElements.get("jobName"), "JobName does not match");
|
|
|
+ assertTrue(Long.parseLong(jobSummaryElements.get("submitTime")) != 0,
|
|
|
+ "submitTime should not be 0");
|
|
|
+ assertTrue(Long.parseLong(jobSummaryElements.get("launchTime")) != 0,
|
|
|
+ "launchTime should not be 0");
|
|
|
+ assertTrue(Long.parseLong(jobSummaryElements.get("firstMapTaskLaunchTime")) != 0,
|
|
|
+ "firstMapTaskLaunchTime should not be 0");
|
|
|
+ assertTrue(Long.parseLong(jobSummaryElements.get("firstReduceTaskLaunchTime")) != 0,
|
|
|
+ "firstReduceTaskLaunchTime should not be 0");
|
|
|
+ assertTrue(Long.parseLong(jobSummaryElements.get("finishTime")) != 0,
|
|
|
+ "finishTime should not be 0");
|
|
|
+ assertEquals(numSuccessfulMaps, Integer.parseInt(jobSummaryElements.get("numMaps")),
|
|
|
+ "Mismatch in num map slots");
|
|
|
+ assertEquals(numReduces, Integer.parseInt(jobSummaryElements.get("numReduces")),
|
|
|
+ "Mismatch in num reduce slots");
|
|
|
+ assertEquals(System.getProperty("user.name"), jobSummaryElements.get("user"),
|
|
|
+ "User does not match");
|
|
|
+ assertEquals("default", jobSummaryElements.get("queue"), "Queue does not match");
|
|
|
+ assertEquals("SUCCEEDED", jobSummaryElements.get("status"), "Status does not match");
|
|
|
}
|
|
|
|
|
|
JobHistory jobHistory = new JobHistory();
|
|
@@ -283,72 +281,69 @@ public class TestJobHistoryParsing {
|
|
|
|
|
|
if (numFinishedMaps != numMaps) {
|
|
|
Exception parseException = parser.getParseException();
|
|
|
- Assert.assertNotNull("Didn't get expected parse exception",
|
|
|
- parseException);
|
|
|
+ assertNotNull(parseException, "Didn't get expected parse exception");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- Assert.assertEquals("Incorrect username ", System.getProperty("user.name"),
|
|
|
- jobInfo.getUsername());
|
|
|
- Assert.assertEquals("Incorrect jobName ", "test", jobInfo.getJobname());
|
|
|
- Assert.assertEquals("Incorrect queuename ", "default",
|
|
|
- jobInfo.getJobQueueName());
|
|
|
- Assert
|
|
|
- .assertEquals("incorrect conf path", "test", jobInfo.getJobConfPath());
|
|
|
- Assert.assertEquals("incorrect finishedMap ", numSuccessfulMaps,
|
|
|
- numFinishedMaps);
|
|
|
- Assert.assertEquals("incorrect finishedReduces ", numReduces,
|
|
|
- jobInfo.getSucceededReduces());
|
|
|
- Assert.assertEquals("incorrect uberized ", job.isUber(),
|
|
|
- jobInfo.getUberized());
|
|
|
+ assertEquals(System.getProperty("user.name"),
|
|
|
+ jobInfo.getUsername(), "Incorrect username ");
|
|
|
+ assertEquals("test", jobInfo.getJobname(), "Incorrect jobName");
|
|
|
+ assertEquals("default", jobInfo.getJobQueueName(),
|
|
|
+ "Incorrect queuename");
|
|
|
+ assertEquals("test", jobInfo.getJobConfPath(), "incorrect conf path");
|
|
|
+ assertEquals(numSuccessfulMaps,
|
|
|
+ numFinishedMaps, "incorrect finishedMap ");
|
|
|
+ assertEquals(numReduces,
|
|
|
+ jobInfo.getSucceededReduces(), "incorrect finishedReduces ");
|
|
|
+ assertEquals(job.isUber(),
|
|
|
+ jobInfo.getUberized(), "incorrect uberized ");
|
|
|
Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
|
|
|
int totalTasks = allTasks.size();
|
|
|
- Assert.assertEquals("total number of tasks is incorrect ",
|
|
|
- (numMaps + numReduces), totalTasks);
|
|
|
+ assertEquals((numMaps + numReduces), totalTasks,
|
|
|
+ "total number of tasks is incorrect");
|
|
|
|
|
|
// Verify aminfo
|
|
|
- Assert.assertEquals(1, jobInfo.getAMInfos().size());
|
|
|
- Assert.assertEquals(MRApp.NM_HOST, jobInfo.getAMInfos().get(0)
|
|
|
+ assertEquals(1, jobInfo.getAMInfos().size());
|
|
|
+ assertEquals(MRApp.NM_HOST, jobInfo.getAMInfos().get(0)
|
|
|
.getNodeManagerHost());
|
|
|
AMInfo amInfo = jobInfo.getAMInfos().get(0);
|
|
|
- Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
|
|
|
- Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
|
|
|
- Assert.assertEquals(1, amInfo.getAppAttemptId().getAttemptId());
|
|
|
- Assert.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
|
|
|
+ assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
|
|
|
+ assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
|
|
|
+ assertEquals(1, amInfo.getAppAttemptId().getAttemptId());
|
|
|
+ assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
|
|
|
.getApplicationAttemptId());
|
|
|
- Assert.assertTrue(amInfo.getStartTime() <= System.currentTimeMillis()
|
|
|
+ assertTrue(amInfo.getStartTime() <= System.currentTimeMillis()
|
|
|
&& amInfo.getStartTime() >= amStartTimeEst);
|
|
|
|
|
|
ContainerId fakeCid = MRApp.newContainerId(-1, -1, -1, -1);
|
|
|
// Assert at taskAttempt level
|
|
|
for (TaskInfo taskInfo : allTasks.values()) {
|
|
|
int taskAttemptCount = taskInfo.getAllTaskAttempts().size();
|
|
|
- Assert
|
|
|
- .assertEquals("total number of task attempts ", 1, taskAttemptCount);
|
|
|
+ assertEquals(1, taskAttemptCount, "total number of task attempts");
|
|
|
TaskAttemptInfo taInfo = taskInfo.getAllTaskAttempts().values()
|
|
|
.iterator().next();
|
|
|
- Assert.assertNotNull(taInfo.getContainerId());
|
|
|
+ assertNotNull(taInfo.getContainerId());
|
|
|
// Verify the wrong ctor is not being used. Remove after mrv1 is removed.
|
|
|
- Assert.assertFalse(taInfo.getContainerId().equals(fakeCid));
|
|
|
+ assertNotEquals(taInfo.getContainerId(), fakeCid);
|
|
|
}
|
|
|
|
|
|
// Deep compare Job and JobInfo
|
|
|
for (Task task : job.getTasks().values()) {
|
|
|
TaskInfo taskInfo = allTasks.get(TypeConverter.fromYarn(task.getID()));
|
|
|
- Assert.assertNotNull("TaskInfo not found", taskInfo);
|
|
|
+ assertNotNull(taskInfo, "TaskInfo not found");
|
|
|
for (TaskAttempt taskAttempt : task.getAttempts().values()) {
|
|
|
TaskAttemptInfo taskAttemptInfo = taskInfo.getAllTaskAttempts().get(
|
|
|
TypeConverter.fromYarn((taskAttempt.getID())));
|
|
|
- Assert.assertNotNull("TaskAttemptInfo not found", taskAttemptInfo);
|
|
|
- Assert.assertEquals("Incorrect shuffle port for task attempt",
|
|
|
- taskAttempt.getShufflePort(), taskAttemptInfo.getShufflePort());
|
|
|
+ assertNotNull(taskAttemptInfo, "TaskAttemptInfo not found");
|
|
|
+ assertEquals(taskAttempt.getShufflePort(), taskAttemptInfo.getShufflePort(),
|
|
|
+ "Incorrect shuffle port for task attempt");
|
|
|
if (numMaps == numSuccessfulMaps) {
|
|
|
- Assert.assertEquals(MRApp.NM_HOST, taskAttemptInfo.getHostname());
|
|
|
- Assert.assertEquals(MRApp.NM_PORT, taskAttemptInfo.getPort());
|
|
|
+ assertEquals(MRApp.NM_HOST, taskAttemptInfo.getHostname());
|
|
|
+ assertEquals(MRApp.NM_PORT, taskAttemptInfo.getPort());
|
|
|
|
|
|
// Verify rack-name
|
|
|
- Assert.assertEquals("rack-name is incorrect",
|
|
|
- taskAttemptInfo.getRackname(), RACK_NAME);
|
|
|
+ assertEquals(taskAttemptInfo.getRackname(), RACK_NAME,
|
|
|
+ "rack-name is incorrect");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -371,8 +366,8 @@ public class TestJobHistoryParsing {
|
|
|
+ " "
|
|
|
+ taskInfo.getTaskType()
|
|
|
+ " task list for " + taskInfo.getTaskId().getJobID();
|
|
|
- Assert.assertTrue(outContent.toString().indexOf(test) > 0);
|
|
|
- Assert.assertTrue(outContent.toString().indexOf(
|
|
|
+ assertTrue(outContent.toString().indexOf(test) > 0);
|
|
|
+ assertTrue(outContent.toString().indexOf(
|
|
|
taskInfo.getTaskId().toString()) > 0);
|
|
|
}
|
|
|
} finally {
|
|
@@ -399,7 +394,8 @@ public class TestJobHistoryParsing {
|
|
|
return numFinishedMaps;
|
|
|
}
|
|
|
|
|
|
- @Test(timeout = 30000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 30)
|
|
|
public void testHistoryParsingForFailedAttempts() throws Exception {
|
|
|
LOG.info("STARTING testHistoryParsingForFailedAttempts");
|
|
|
try {
|
|
@@ -440,8 +436,8 @@ public class TestJobHistoryParsing {
|
|
|
jobInfo = parser.parse();
|
|
|
}
|
|
|
Exception parseException = parser.getParseException();
|
|
|
- Assert.assertNull("Caught an expected exception " + parseException,
|
|
|
- parseException);
|
|
|
+ assertNull(parseException,
|
|
|
+ "Caught an expected exception " + parseException);
|
|
|
int noOffailedAttempts = 0;
|
|
|
Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
|
|
|
for (Task task : job.getTasks().values()) {
|
|
@@ -457,14 +453,14 @@ public class TestJobHistoryParsing {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- Assert.assertEquals("No of Failed tasks doesn't match.", 2,
|
|
|
- noOffailedAttempts);
|
|
|
+ assertEquals(2, noOffailedAttempts, "No of Failed tasks doesn't match.");
|
|
|
} finally {
|
|
|
LOG.info("FINISHED testHistoryParsingForFailedAttempts");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test(timeout = 30000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 30)
|
|
|
public void testHistoryParsingForKilledAndFailedAttempts() throws Exception {
|
|
|
MRApp app = null;
|
|
|
JobHistory jobHistory = null;
|
|
@@ -516,13 +512,13 @@ public class TestJobHistoryParsing {
|
|
|
jobInfo = parser.parse();
|
|
|
}
|
|
|
Exception parseException = parser.getParseException();
|
|
|
- Assert.assertNull("Caught an expected exception " + parseException,
|
|
|
- parseException);
|
|
|
+ assertNull(parseException,
|
|
|
+ "Caught an expected exception " + parseException);
|
|
|
|
|
|
- assertEquals("FailedMaps", 1, jobInfo.getFailedMaps());
|
|
|
- assertEquals("KilledMaps", 1, jobInfo.getKilledMaps());
|
|
|
- assertEquals("FailedReduces", 1, jobInfo.getFailedReduces());
|
|
|
- assertEquals("KilledReduces", 1, jobInfo.getKilledReduces());
|
|
|
+ assertEquals(1, jobInfo.getFailedMaps(), "FailedMaps");
|
|
|
+ assertEquals(1, jobInfo.getKilledMaps(), "KilledMaps");
|
|
|
+ assertEquals(1, jobInfo.getFailedReduces(), "FailedReduces");
|
|
|
+ assertEquals(1, jobInfo.getKilledReduces(), "KilledReduces");
|
|
|
} finally {
|
|
|
LOG.info("FINISHED testHistoryParsingForKilledAndFailedAttempts");
|
|
|
if (app != null) {
|
|
@@ -534,7 +530,8 @@ public class TestJobHistoryParsing {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test(timeout = 60000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 60)
|
|
|
public void testCountersForFailedTask() throws Exception {
|
|
|
LOG.info("STARTING testCountersForFailedTask");
|
|
|
try {
|
|
@@ -576,19 +573,18 @@ public class TestJobHistoryParsing {
|
|
|
jobInfo = parser.parse();
|
|
|
}
|
|
|
Exception parseException = parser.getParseException();
|
|
|
- Assert.assertNull("Caught an expected exception " + parseException,
|
|
|
- parseException);
|
|
|
+ assertNull(parseException, "Caught an expected exception " + 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());
|
|
|
+ assertNotNull(ct.getReport().getCounters(),
|
|
|
+ "completed task report has null counters");
|
|
|
}
|
|
|
final List<String> originalDiagnostics = job.getDiagnostics();
|
|
|
final String historyError = jobInfo.getErrorInfo();
|
|
|
- assertTrue("No original diagnostics for a failed job",
|
|
|
- originalDiagnostics != null && !originalDiagnostics.isEmpty());
|
|
|
- assertNotNull("No history error info for a failed job ", historyError);
|
|
|
+ assertTrue(originalDiagnostics != null && !originalDiagnostics.isEmpty(),
|
|
|
+ "No original diagnostics for a failed job");
|
|
|
+ assertNotNull(historyError, "No history error info for a failed job");
|
|
|
for (String diagString : originalDiagnostics) {
|
|
|
assertTrue(historyError.contains(diagString));
|
|
|
}
|
|
@@ -597,7 +593,8 @@ public class TestJobHistoryParsing {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test(timeout = 60000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 60)
|
|
|
public void testDiagnosticsForKilledJob() throws Exception {
|
|
|
LOG.info("STARTING testDiagnosticsForKilledJob");
|
|
|
try {
|
|
@@ -639,24 +636,24 @@ public class TestJobHistoryParsing {
|
|
|
jobInfo = parser.parse();
|
|
|
}
|
|
|
Exception parseException = parser.getParseException();
|
|
|
- assertNull("Caught an expected exception " + parseException,
|
|
|
- parseException);
|
|
|
+ assertNull(parseException, "Caught an expected exception " + parseException);
|
|
|
final List<String> originalDiagnostics = job.getDiagnostics();
|
|
|
final String historyError = jobInfo.getErrorInfo();
|
|
|
- assertTrue("No original diagnostics for a failed job",
|
|
|
- originalDiagnostics != null && !originalDiagnostics.isEmpty());
|
|
|
- assertNotNull("No history error info for a failed job ", historyError);
|
|
|
+ assertTrue(originalDiagnostics != null && !originalDiagnostics.isEmpty(),
|
|
|
+ "No original diagnostics for a failed job");
|
|
|
+ assertNotNull(historyError, "No history error info for a failed job ");
|
|
|
for (String diagString : originalDiagnostics) {
|
|
|
assertTrue(historyError.contains(diagString));
|
|
|
}
|
|
|
- assertTrue("No killed message in diagnostics",
|
|
|
- historyError.contains(JobImpl.JOB_KILLED_DIAG));
|
|
|
+ assertTrue(historyError.contains(JobImpl.JOB_KILLED_DIAG),
|
|
|
+ "No killed message in diagnostics");
|
|
|
} finally {
|
|
|
LOG.info("FINISHED testDiagnosticsForKilledJob");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test(timeout = 50000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 50)
|
|
|
public void testScanningOldDirs() throws Exception {
|
|
|
LOG.info("STARTING testScanningOldDirs");
|
|
|
try {
|
|
@@ -679,24 +676,24 @@ public class TestJobHistoryParsing {
|
|
|
HistoryFileManagerForTest hfm = new HistoryFileManagerForTest();
|
|
|
hfm.init(conf);
|
|
|
HistoryFileInfo fileInfo = hfm.getFileInfo(jobId);
|
|
|
- Assert.assertNotNull("Unable to locate job history", fileInfo);
|
|
|
+ assertNotNull(fileInfo, "Unable to locate job history");
|
|
|
|
|
|
// 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());
|
|
|
+ assertFalse(fileInfo.didMoveFail());
|
|
|
msecToSleep -= msecPerSleep;
|
|
|
Thread.sleep(msecPerSleep);
|
|
|
}
|
|
|
- Assert.assertTrue("Timeout waiting for history move", msecToSleep > 0);
|
|
|
+ assertTrue(msecToSleep > 0, "Timeout waiting for history move");
|
|
|
|
|
|
fileInfo = hfm.getFileInfo(jobId);
|
|
|
hfm.stop();
|
|
|
- Assert.assertNotNull("Unable to locate old job history", fileInfo);
|
|
|
- Assert.assertTrue("HistoryFileManager not shutdown properly",
|
|
|
- hfm.moveToDoneExecutor.isTerminated());
|
|
|
+ assertNotNull(fileInfo, "Unable to locate old job history");
|
|
|
+ assertTrue(hfm.moveToDoneExecutor.isTerminated(),
|
|
|
+ "HistoryFileManager not shutdown properly");
|
|
|
} finally {
|
|
|
LOG.info("FINISHED testScanningOldDirs");
|
|
|
}
|
|
@@ -813,7 +810,8 @@ public class TestJobHistoryParsing {
|
|
|
* Test clean old history files. Files should be deleted after 1 week by
|
|
|
* default.
|
|
|
*/
|
|
|
- @Test(timeout = 15000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 15)
|
|
|
public void testDeleteFileInfo() throws Exception {
|
|
|
LOG.info("STARTING testDeleteFileInfo");
|
|
|
try {
|
|
@@ -845,20 +843,19 @@ public class TestJobHistoryParsing {
|
|
|
Thread.sleep(300);
|
|
|
}
|
|
|
|
|
|
- Assert.assertNotNull(hfm.jobListCache.values());
|
|
|
+ assertNotNull(hfm.jobListCache.values());
|
|
|
|
|
|
// try to remove fileInfo
|
|
|
hfm.clean();
|
|
|
// check that fileInfo does not deleted
|
|
|
- Assert.assertFalse(fileInfo.isDeleted());
|
|
|
+ assertFalse(fileInfo.isDeleted());
|
|
|
// correct live time
|
|
|
hfm.setMaxHistoryAge(-1);
|
|
|
hfm.clean();
|
|
|
hfm.stop();
|
|
|
- Assert.assertTrue("Thread pool shutdown",
|
|
|
- hfm.moveToDoneExecutor.isTerminated());
|
|
|
+ assertTrue(hfm.moveToDoneExecutor.isTerminated(), "Thread pool shutdown");
|
|
|
// should be deleted !
|
|
|
- Assert.assertTrue("file should be deleted ", fileInfo.isDeleted());
|
|
|
+ assertTrue(fileInfo.isDeleted(), "file should be deleted ");
|
|
|
|
|
|
} finally {
|
|
|
LOG.info("FINISHED testDeleteFileInfo");
|
|
@@ -868,7 +865,8 @@ public class TestJobHistoryParsing {
|
|
|
/**
|
|
|
* Simple test some methods of JobHistory
|
|
|
*/
|
|
|
- @Test(timeout = 20000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 20)
|
|
|
public void testJobHistoryMethods() throws Exception {
|
|
|
LOG.info("STARTING testJobHistoryMethods");
|
|
|
try {
|
|
@@ -892,27 +890,25 @@ public class TestJobHistoryParsing {
|
|
|
JobHistory jobHistory = new JobHistory();
|
|
|
jobHistory.init(configuration);
|
|
|
// Method getAllJobs
|
|
|
- Assert.assertEquals(1, jobHistory.getAllJobs().size());
|
|
|
+ assertEquals(1, jobHistory.getAllJobs().size());
|
|
|
// and with ApplicationId
|
|
|
- Assert.assertEquals(1, jobHistory.getAllJobs(app.getAppID()).size());
|
|
|
+ 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());
|
|
|
+ assertEquals(1, jobsinfo.getJobs().size());
|
|
|
+ assertNotNull(jobHistory.getApplicationAttemptId());
|
|
|
// test Application Id
|
|
|
- Assert.assertEquals("application_0_0000", jobHistory.getApplicationID()
|
|
|
- .toString());
|
|
|
- Assert
|
|
|
- .assertEquals("Job History Server", jobHistory.getApplicationName());
|
|
|
+ assertEquals("application_0_0000", jobHistory.getApplicationID().toString());
|
|
|
+ assertEquals("Job History Server", jobHistory.getApplicationName());
|
|
|
// method does not work
|
|
|
- Assert.assertNull(jobHistory.getEventHandler());
|
|
|
+ assertNull(jobHistory.getEventHandler());
|
|
|
// method does not work
|
|
|
- Assert.assertNull(jobHistory.getClock());
|
|
|
+ assertNull(jobHistory.getClock());
|
|
|
// method does not work
|
|
|
- Assert.assertNull(jobHistory.getClusterInfo());
|
|
|
+ assertNull(jobHistory.getClusterInfo());
|
|
|
|
|
|
} finally {
|
|
|
LOG.info("FINISHED testJobHistoryMethods");
|
|
@@ -922,7 +918,8 @@ public class TestJobHistoryParsing {
|
|
|
/**
|
|
|
* Simple test PartialJob
|
|
|
*/
|
|
|
- @Test(timeout = 3000)
|
|
|
+ @Test
|
|
|
+ @Timeout(value = 3)
|
|
|
public void testPartialJob() throws Exception {
|
|
|
JobId jobId = new JobIdPBImpl();
|
|
|
jobId.setId(0);
|
|
@@ -930,7 +927,7 @@ public class TestJobHistoryParsing {
|
|
|
"jobName", jobId, 3, 2, "JobStatus");
|
|
|
PartialJob test = new PartialJob(jii, jobId);
|
|
|
|
|
|
- Assert.assertEquals(1.0f, test.getProgress(), 0.001f);
|
|
|
+ assertEquals(1.0f, test.getProgress(), 0.001f);
|
|
|
assertNull(test.getAllCounters());
|
|
|
assertNull(test.getTasks());
|
|
|
assertNull(test.getTasks(TaskType.MAP));
|
|
@@ -983,8 +980,8 @@ public class TestJobHistoryParsing {
|
|
|
}
|
|
|
});
|
|
|
JobInfo info = parser.parse(reader);
|
|
|
- assertTrue("Task 0 not implicated",
|
|
|
- info.getErrorInfo().contains(tids[0].toString()));
|
|
|
+ assertTrue(info.getErrorInfo().contains(tids[0].toString()),
|
|
|
+ "Task 0 not implicated");
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -997,9 +994,9 @@ public class TestJobHistoryParsing {
|
|
|
try {
|
|
|
JobHistoryParser parser = new JobHistoryParser(fsdis);
|
|
|
JobInfo info = parser.parse();
|
|
|
- assertEquals("History parsed jobId incorrectly",
|
|
|
- info.getJobId(), JobID.forName("job_1393307629410_0001") );
|
|
|
- assertEquals("Default diagnostics incorrect ", "", info.getErrorInfo());
|
|
|
+ assertEquals(info.getJobId(), JobID.forName("job_1393307629410_0001"),
|
|
|
+ "History parsed jobId incorrectly");
|
|
|
+ assertEquals("", info.getErrorInfo(), "Default diagnostics incorrect");
|
|
|
} finally {
|
|
|
fsdis.close();
|
|
|
}
|