|
@@ -27,6 +27,7 @@ import static org.junit.Assert.fail;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
import java.io.StringReader;
|
|
|
import java.util.Map;
|
|
|
|
|
@@ -40,8 +41,11 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.MockJobs;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
|
+import org.apache.hadoop.mapreduce.v2.hs.MockHistoryJobs;
|
|
|
+import org.apache.hadoop.mapreduce.v2.hs.MockHistoryJobs.JobsPair;
|
|
|
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
|
|
import org.apache.hadoop.yarn.Clock;
|
|
|
+import org.apache.hadoop.yarn.YarnException;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
@@ -89,13 +93,21 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
final ApplicationAttemptId appAttemptID;
|
|
|
final ApplicationId appID;
|
|
|
final String user = MockJobs.newUserName();
|
|
|
- final Map<JobId, Job> jobs;
|
|
|
+ final Map<JobId, Job> partialJobs;
|
|
|
+ final Map<JobId, Job> fullJobs;
|
|
|
final long startTime = System.currentTimeMillis();
|
|
|
|
|
|
TestAppContext(int appid, int numJobs, int numTasks, int numAttempts) {
|
|
|
appID = MockJobs.newAppID(appid);
|
|
|
appAttemptID = MockJobs.newAppAttemptID(appID, 0);
|
|
|
- jobs = MockJobs.newJobs(appID, numJobs, numTasks, numAttempts);
|
|
|
+ JobsPair jobs;
|
|
|
+ try {
|
|
|
+ jobs = MockHistoryJobs.newHistoryJobs(appID, numJobs, numTasks, numAttempts);
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new YarnException(e);
|
|
|
+ }
|
|
|
+ partialJobs = jobs.partial;
|
|
|
+ fullJobs = jobs.full;
|
|
|
}
|
|
|
|
|
|
TestAppContext() {
|
|
@@ -119,12 +131,16 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
|
|
|
@Override
|
|
|
public Job getJob(JobId jobID) {
|
|
|
- return jobs.get(jobID);
|
|
|
+ return fullJobs.get(jobID);
|
|
|
}
|
|
|
|
|
|
+ public Job getPartialJob(JobId jobID) {
|
|
|
+ return partialJobs.get(jobID);
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public Map<JobId, Job> getAllJobs() {
|
|
|
- return jobs; // OK
|
|
|
+ return partialJobs; // OK
|
|
|
}
|
|
|
|
|
|
@SuppressWarnings("rawtypes")
|
|
@@ -204,8 +220,8 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
JSONArray arr = jobs.getJSONArray("job");
|
|
|
assertEquals("incorrect number of elements", 1, arr.length());
|
|
|
JSONObject info = arr.getJSONObject(0);
|
|
|
- Job job = appContext.getJob(MRApps.toJobID(info.getString("id")));
|
|
|
- VerifyJobsUtils.verifyHsJob(info, job);
|
|
|
+ Job job = appContext.getPartialJob(MRApps.toJobID(info.getString("id")));
|
|
|
+ VerifyJobsUtils.verifyHsJobPartial(info, job);
|
|
|
|
|
|
}
|
|
|
|
|
@@ -222,8 +238,8 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
JSONArray arr = jobs.getJSONArray("job");
|
|
|
assertEquals("incorrect number of elements", 1, arr.length());
|
|
|
JSONObject info = arr.getJSONObject(0);
|
|
|
- Job job = appContext.getJob(MRApps.toJobID(info.getString("id")));
|
|
|
- VerifyJobsUtils.verifyHsJob(info, job);
|
|
|
+ Job job = appContext.getPartialJob(MRApps.toJobID(info.getString("id")));
|
|
|
+ VerifyJobsUtils.verifyHsJobPartial(info, job);
|
|
|
|
|
|
}
|
|
|
|
|
@@ -239,8 +255,8 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
JSONArray arr = jobs.getJSONArray("job");
|
|
|
assertEquals("incorrect number of elements", 1, arr.length());
|
|
|
JSONObject info = arr.getJSONObject(0);
|
|
|
- Job job = appContext.getJob(MRApps.toJobID(info.getString("id")));
|
|
|
- VerifyJobsUtils.verifyHsJob(info, job);
|
|
|
+ Job job = appContext.getPartialJob(MRApps.toJobID(info.getString("id")));
|
|
|
+ VerifyJobsUtils.verifyHsJobPartial(info, job);
|
|
|
|
|
|
}
|
|
|
|
|
@@ -261,10 +277,35 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
assertEquals("incorrect number of elements", 1, jobs.getLength());
|
|
|
NodeList job = dom.getElementsByTagName("job");
|
|
|
assertEquals("incorrect number of elements", 1, job.getLength());
|
|
|
- verifyHsJobXML(job, appContext);
|
|
|
-
|
|
|
+ verifyHsJobPartialXML(job, appContext);
|
|
|
}
|
|
|
|
|
|
+ public void verifyHsJobPartialXML(NodeList nodes, TestAppContext appContext) {
|
|
|
+
|
|
|
+ assertEquals("incorrect number of elements", 1, nodes.getLength());
|
|
|
+
|
|
|
+ for (int i = 0; i < nodes.getLength(); i++) {
|
|
|
+ Element element = (Element) nodes.item(i);
|
|
|
+
|
|
|
+ Job job = appContext.getPartialJob(MRApps.toJobID(WebServicesTestUtils
|
|
|
+ .getXmlString(element, "id")));
|
|
|
+ assertNotNull("Job not found - output incorrect", job);
|
|
|
+
|
|
|
+ VerifyJobsUtils.verifyHsJobGeneric(job,
|
|
|
+ WebServicesTestUtils.getXmlString(element, "id"),
|
|
|
+ WebServicesTestUtils.getXmlString(element, "user"),
|
|
|
+ WebServicesTestUtils.getXmlString(element, "name"),
|
|
|
+ WebServicesTestUtils.getXmlString(element, "state"),
|
|
|
+ WebServicesTestUtils.getXmlString(element, "queue"),
|
|
|
+ WebServicesTestUtils.getXmlLong(element, "startTime"),
|
|
|
+ WebServicesTestUtils.getXmlLong(element, "finishTime"),
|
|
|
+ WebServicesTestUtils.getXmlInt(element, "mapsTotal"),
|
|
|
+ WebServicesTestUtils.getXmlInt(element, "mapsCompleted"),
|
|
|
+ WebServicesTestUtils.getXmlInt(element, "reducesTotal"),
|
|
|
+ WebServicesTestUtils.getXmlInt(element, "reducesCompleted"));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
public void verifyHsJobXML(NodeList nodes, TestAppContext appContext) {
|
|
|
|
|
|
assertEquals("incorrect number of elements", 1, nodes.getLength());
|
|
@@ -320,7 +361,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
JSONObject json = response.getEntity(JSONObject.class);
|
|
|
assertEquals("incorrect number of elements", 1, json.length());
|
|
|
JSONObject info = json.getJSONObject("job");
|
|
|
- VerifyJobsUtils.verifyHsJob(info, jobsMap.get(id));
|
|
|
+ VerifyJobsUtils.verifyHsJob(info, appContext.getJob(id));
|
|
|
}
|
|
|
|
|
|
}
|
|
@@ -356,7 +397,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
JSONObject json = response.getEntity(JSONObject.class);
|
|
|
assertEquals("incorrect number of elements", 1, json.length());
|
|
|
JSONObject info = json.getJSONObject("job");
|
|
|
- VerifyJobsUtils.verifyHsJob(info, jobsMap.get(id));
|
|
|
+ VerifyJobsUtils.verifyHsJob(info, appContext.getJob(id));
|
|
|
}
|
|
|
|
|
|
}
|
|
@@ -694,7 +735,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
JSONObject json = response.getEntity(JSONObject.class);
|
|
|
assertEquals("incorrect number of elements", 1, json.length());
|
|
|
JSONObject info = json.getJSONObject("jobAttempts");
|
|
|
- verifyHsJobAttempts(info, jobsMap.get(id));
|
|
|
+ verifyHsJobAttempts(info, appContext.getJob(id));
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -712,7 +753,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
JSONObject json = response.getEntity(JSONObject.class);
|
|
|
assertEquals("incorrect number of elements", 1, json.length());
|
|
|
JSONObject info = json.getJSONObject("jobAttempts");
|
|
|
- verifyHsJobAttempts(info, jobsMap.get(id));
|
|
|
+ verifyHsJobAttempts(info, appContext.getJob(id));
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -730,7 +771,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
JSONObject json = response.getEntity(JSONObject.class);
|
|
|
assertEquals("incorrect number of elements", 1, json.length());
|
|
|
JSONObject info = json.getJSONObject("jobAttempts");
|
|
|
- verifyHsJobAttempts(info, jobsMap.get(id));
|
|
|
+ verifyHsJobAttempts(info, appContext.getJob(id));
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -754,7 +795,7 @@ public class TestHsWebServicesJobs extends JerseyTest {
|
|
|
NodeList attempts = dom.getElementsByTagName("jobAttempts");
|
|
|
assertEquals("incorrect number of elements", 1, attempts.getLength());
|
|
|
NodeList info = dom.getElementsByTagName("jobAttempt");
|
|
|
- verifyHsJobAttemptsXML(info, jobsMap.get(id));
|
|
|
+ verifyHsJobAttemptsXML(info, appContext.getJob(id));
|
|
|
}
|
|
|
}
|
|
|
|