Merge -c 1227801 from trunk to branch-0.23 to fix MAPREDUCE-3548. MAPREDUCE-3548. Added more unit tests for MR AM & JHS web-services.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1227802 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1828be291a
commit
2281b945d8
|
@ -363,6 +363,9 @@ Release 0.23.1 - Unreleased
|
|||
MAPREDUCE-3326. Added detailed information about queue's to the
|
||||
CapacityScheduler web-ui. (Jason Lowe via acmurthy)
|
||||
|
||||
MAPREDUCE-3548. Added more unit tests for MR AM & JHS web-services.
|
||||
(Thomas Graves via acmurthy)
|
||||
|
||||
Release 0.23.0 - 2011-11-01
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -76,14 +76,90 @@ public class AMWebServices {
|
|||
}
|
||||
|
||||
Boolean hasAccess(Job job, HttpServletRequest request) {
|
||||
UserGroupInformation callerUgi = UserGroupInformation
|
||||
.createRemoteUser(request.getRemoteUser());
|
||||
if (!job.checkAccess(callerUgi, JobACL.VIEW_JOB)) {
|
||||
String remoteUser = request.getRemoteUser();
|
||||
UserGroupInformation callerUGI = null;
|
||||
if (remoteUser != null) {
|
||||
callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
|
||||
}
|
||||
if (callerUGI != null && !job.checkAccess(callerUGI, JobACL.VIEW_JOB)) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* convert a job id string to an actual job and handle all the error checking.
|
||||
*/
|
||||
public static Job getJobFromJobIdString(String jid, AppContext appCtx) throws NotFoundException {
|
||||
JobId jobId;
|
||||
Job job;
|
||||
try {
|
||||
jobId = MRApps.toJobID(jid);
|
||||
} catch (YarnException e) {
|
||||
throw new NotFoundException(e.getMessage());
|
||||
}
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
job = appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
return job;
|
||||
}
|
||||
|
||||
/**
|
||||
* convert a task id string to an actual task and handle all the error
|
||||
* checking.
|
||||
*/
|
||||
public static Task getTaskFromTaskIdString(String tid, Job job) throws NotFoundException {
|
||||
TaskId taskID;
|
||||
Task task;
|
||||
try {
|
||||
taskID = MRApps.toTaskID(tid);
|
||||
} catch (YarnException e) {
|
||||
throw new NotFoundException(e.getMessage());
|
||||
} catch (NumberFormatException ne) {
|
||||
throw new NotFoundException(ne.getMessage());
|
||||
}
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
return task;
|
||||
}
|
||||
|
||||
/**
|
||||
* convert a task attempt id string to an actual task attempt and handle all
|
||||
* the error checking.
|
||||
*/
|
||||
public static TaskAttempt getTaskAttemptFromTaskAttemptString(String attId, Task task)
|
||||
throws NotFoundException {
|
||||
TaskAttemptId attemptId;
|
||||
TaskAttempt ta;
|
||||
try {
|
||||
attemptId = MRApps.toTaskAttemptID(attId);
|
||||
} catch (YarnException e) {
|
||||
throw new NotFoundException(e.getMessage());
|
||||
} catch (NumberFormatException ne) {
|
||||
throw new NotFoundException(ne.getMessage());
|
||||
}
|
||||
if (attemptId == null) {
|
||||
throw new NotFoundException("task attempt id " + attId
|
||||
+ " not found or invalid");
|
||||
}
|
||||
ta = task.getAttempt(attemptId);
|
||||
if (ta == null) {
|
||||
throw new NotFoundException("Error getting info on task attempt id "
|
||||
+ attId);
|
||||
}
|
||||
return ta;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* check for job access.
|
||||
*
|
||||
|
@ -130,16 +206,8 @@ public class AMWebServices {
|
|||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public JobInfo getJob(@Context HttpServletRequest hsr,
|
||||
@PathParam("jobid") String jid) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = getJobFromJobIdString(jid, appCtx);
|
||||
return new JobInfo(job, hasAccess(job, hsr));
|
||||
|
||||
}
|
||||
|
||||
@GET
|
||||
|
@ -147,63 +215,25 @@ public class AMWebServices {
|
|||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public JobCounterInfo getJobCounters(@Context HttpServletRequest hsr,
|
||||
@PathParam("jobid") String jid) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = getJobFromJobIdString(jid, appCtx);
|
||||
checkAccess(job, hsr);
|
||||
return new JobCounterInfo(this.appCtx, job);
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/jobs/{jobid}/tasks/{taskid}/counters")
|
||||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public JobTaskCounterInfo getSingleTaskCounters(
|
||||
@Context HttpServletRequest hsr, @PathParam("jobid") String jid,
|
||||
@PathParam("taskid") String tid) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = this.appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
checkAccess(job, hsr);
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
Task task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
return new JobTaskCounterInfo(task);
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/jobs/{jobid}/conf")
|
||||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public ConfInfo getJobConf(@Context HttpServletRequest hsr,
|
||||
@PathParam("jobid") String jid) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
|
||||
Job job = getJobFromJobIdString(jid, appCtx);
|
||||
checkAccess(job, hsr);
|
||||
ConfInfo info;
|
||||
try {
|
||||
info = new ConfInfo(job, this.conf);
|
||||
} catch (IOException e) {
|
||||
throw new NotFoundException("unable to load configuration for job: " + jid);
|
||||
throw new NotFoundException("unable to load configuration for job: "
|
||||
+ jid);
|
||||
}
|
||||
return info;
|
||||
}
|
||||
|
@ -213,10 +243,8 @@ public class AMWebServices {
|
|||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public TasksInfo getJobTasks(@Context HttpServletRequest hsr,
|
||||
@PathParam("jobid") String jid, @QueryParam("type") String type) {
|
||||
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
|
||||
Job job = getJobFromJobIdString(jid, appCtx);
|
||||
checkAccess(job, hsr);
|
||||
TasksInfo allTasks = new TasksInfo();
|
||||
for (Task task : job.getTasks().values()) {
|
||||
|
@ -225,7 +253,8 @@ public class AMWebServices {
|
|||
try {
|
||||
ttype = MRApps.taskType(type);
|
||||
} catch (YarnException e) {
|
||||
throw new BadRequestException("tasktype must be either m or r"); }
|
||||
throw new BadRequestException("tasktype must be either m or r");
|
||||
}
|
||||
}
|
||||
if (ttype != null && task.getType() != ttype) {
|
||||
continue;
|
||||
|
@ -240,21 +269,24 @@ public class AMWebServices {
|
|||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public TaskInfo getJobTask(@Context HttpServletRequest hsr,
|
||||
@PathParam("jobid") String jid, @PathParam("taskid") String tid) {
|
||||
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
checkAccess(job, hsr);
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
Task task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
return new TaskInfo(task);
|
||||
|
||||
Job job = getJobFromJobIdString(jid, appCtx);
|
||||
checkAccess(job, hsr);
|
||||
Task task = getTaskFromTaskIdString(tid, job);
|
||||
return new TaskInfo(task);
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/jobs/{jobid}/tasks/{taskid}/counters")
|
||||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public JobTaskCounterInfo getSingleTaskCounters(
|
||||
@Context HttpServletRequest hsr, @PathParam("jobid") String jid,
|
||||
@PathParam("taskid") String tid) {
|
||||
|
||||
Job job = getJobFromJobIdString(jid, appCtx);
|
||||
checkAccess(job, hsr);
|
||||
Task task = getTaskFromTaskIdString(tid, job);
|
||||
return new JobTaskCounterInfo(task);
|
||||
}
|
||||
|
||||
@GET
|
||||
|
@ -263,19 +295,11 @@ public class AMWebServices {
|
|||
public TaskAttemptsInfo getJobTaskAttempts(@Context HttpServletRequest hsr,
|
||||
@PathParam("jobid") String jid, @PathParam("taskid") String tid) {
|
||||
TaskAttemptsInfo attempts = new TaskAttemptsInfo();
|
||||
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
|
||||
Job job = getJobFromJobIdString(jid, appCtx);
|
||||
checkAccess(job, hsr);
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
Task task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
Task task = getTaskFromTaskIdString(tid, job);
|
||||
|
||||
for (TaskAttempt ta : task.getAttempts().values()) {
|
||||
if (ta != null) {
|
||||
if (task.getType() == TaskType.REDUCE) {
|
||||
|
@ -294,29 +318,11 @@ public class AMWebServices {
|
|||
public TaskAttemptInfo getJobTaskAttemptId(@Context HttpServletRequest hsr,
|
||||
@PathParam("jobid") String jid, @PathParam("taskid") String tid,
|
||||
@PathParam("attemptid") String attId) {
|
||||
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
|
||||
Job job = getJobFromJobIdString(jid, appCtx);
|
||||
checkAccess(job, hsr);
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
Task task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
|
||||
if (attemptId == null) {
|
||||
throw new NotFoundException("task attempt id " + attId
|
||||
+ " not found or invalid");
|
||||
}
|
||||
TaskAttempt ta = task.getAttempt(attemptId);
|
||||
if (ta == null) {
|
||||
throw new NotFoundException("Error getting info on task attempt id "
|
||||
+ attId);
|
||||
}
|
||||
Task task = getTaskFromTaskIdString(tid, job);
|
||||
TaskAttempt ta = getTaskAttemptFromTaskAttemptString(attId, task);
|
||||
if (task.getType() == TaskType.REDUCE) {
|
||||
return new ReduceTaskAttemptInfo(ta, task.getType());
|
||||
} else {
|
||||
|
@ -330,33 +336,11 @@ public class AMWebServices {
|
|||
public JobTaskAttemptCounterInfo getJobTaskAttemptIdCounters(
|
||||
@Context HttpServletRequest hsr, @PathParam("jobid") String jid,
|
||||
@PathParam("taskid") String tid, @PathParam("attemptid") String attId) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = this.appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
|
||||
Job job = getJobFromJobIdString(jid, appCtx);
|
||||
checkAccess(job, hsr);
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
Task task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
|
||||
if (attemptId == null) {
|
||||
throw new NotFoundException("task attempt id " + attId
|
||||
+ " not found or invalid");
|
||||
}
|
||||
TaskAttempt ta = task.getAttempt(attemptId);
|
||||
if (ta == null) {
|
||||
throw new NotFoundException("Error getting info on task attempt id "
|
||||
+ attId);
|
||||
}
|
||||
Task task = getTaskFromTaskIdString(tid, job);
|
||||
TaskAttempt ta = getTaskAttemptFromTaskAttemptString(attId, task);
|
||||
return new JobTaskAttemptCounterInfo(ta);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,6 @@ public class AppInfo {
|
|||
protected String appId;
|
||||
protected String name;
|
||||
protected String user;
|
||||
protected String hostname;
|
||||
protected long startedOn;
|
||||
protected long elapsedTime;
|
||||
|
||||
|
@ -44,7 +43,7 @@ public class AppInfo {
|
|||
this.name = context.getApplicationName().toString();
|
||||
this.user = context.getUser().toString();
|
||||
this.startedOn = context.getStartTime();
|
||||
this.elapsedTime = Times.elapsed(context.getStartTime(), 0);
|
||||
this.elapsedTime = Times.elapsed(this.startedOn, 0);
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.fs.FileContext;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
||||
|
||||
@XmlRootElement
|
||||
@XmlRootElement(name = "conf")
|
||||
@XmlAccessorType(XmlAccessType.FIELD)
|
||||
public class ConfInfo {
|
||||
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.Counter;
|
|||
@XmlAccessorType(XmlAccessType.FIELD)
|
||||
public class CounterInfo {
|
||||
|
||||
protected String counterName;
|
||||
protected String name;
|
||||
protected long totalCounterValue;
|
||||
protected long mapCounterValue;
|
||||
protected long reduceCounterValue;
|
||||
|
@ -36,7 +36,7 @@ public class CounterInfo {
|
|||
}
|
||||
|
||||
public CounterInfo(Counter counter, Counter mc, Counter rc) {
|
||||
this.counterName = counter.getName();
|
||||
this.name = counter.getName();
|
||||
this.totalCounterValue = counter.getValue();
|
||||
this.mapCounterValue = mc == null ? 0 : mc.getValue();
|
||||
this.reduceCounterValue = rc == null ? 0 : rc.getValue();
|
||||
|
|
|
@ -46,14 +46,14 @@ public class JobCounterInfo {
|
|||
protected Counters reduce = null;
|
||||
|
||||
protected String id;
|
||||
protected ArrayList<CounterGroupInfo> counterGroups;
|
||||
protected ArrayList<CounterGroupInfo> counterGroup;
|
||||
|
||||
public JobCounterInfo() {
|
||||
}
|
||||
|
||||
public JobCounterInfo(AppContext ctx, Job job) {
|
||||
getCounters(ctx, job);
|
||||
counterGroups = new ArrayList<CounterGroupInfo>();
|
||||
counterGroup = new ArrayList<CounterGroupInfo>();
|
||||
this.id = MRApps.toString(job.getID());
|
||||
|
||||
int numGroups = 0;
|
||||
|
@ -68,7 +68,7 @@ public class JobCounterInfo {
|
|||
++numGroups;
|
||||
|
||||
CounterGroupInfo cginfo = new CounterGroupInfo(g.getName(), g, mg, rg);
|
||||
counterGroups.add(cginfo);
|
||||
counterGroup.add(cginfo);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import javax.xml.bind.annotation.XmlTransient;
|
|||
|
||||
import org.apache.hadoop.mapreduce.JobACL;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||
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.app.job.Job;
|
||||
|
@ -51,12 +52,12 @@ public class JobInfo {
|
|||
protected String id;
|
||||
protected String name;
|
||||
protected String user;
|
||||
protected String state;
|
||||
protected JobState state;
|
||||
protected int mapsTotal;
|
||||
protected int mapsCompleted;
|
||||
protected float mapProgress;
|
||||
protected int reducesTotal;
|
||||
protected int reducesCompleted;
|
||||
protected float mapProgress;
|
||||
protected float reduceProgress;
|
||||
|
||||
@XmlTransient
|
||||
|
@ -83,18 +84,12 @@ public class JobInfo {
|
|||
protected int successfulMapAttempts = 0;
|
||||
protected ArrayList<ConfEntryInfo> acls;
|
||||
|
||||
@XmlTransient
|
||||
protected int numMaps;
|
||||
@XmlTransient
|
||||
protected int numReduces;
|
||||
|
||||
public JobInfo() {
|
||||
}
|
||||
|
||||
public JobInfo(Job job, Boolean hasAccess) {
|
||||
this.id = MRApps.toString(job.getID());
|
||||
JobReport report = job.getReport();
|
||||
countTasksAndAttempts(job);
|
||||
this.startTime = report.getStartTime();
|
||||
this.finishTime = report.getFinishTime();
|
||||
this.elapsedTime = Times.elapsed(this.startTime, this.finishTime);
|
||||
|
@ -103,7 +98,7 @@ public class JobInfo {
|
|||
}
|
||||
this.name = job.getName().toString();
|
||||
this.user = job.getUserName();
|
||||
this.state = job.getState().toString();
|
||||
this.state = job.getState();
|
||||
this.mapsTotal = job.getTotalMaps();
|
||||
this.mapsCompleted = job.getCompletedMaps();
|
||||
this.mapProgress = report.getMapProgress() * 100;
|
||||
|
@ -115,6 +110,9 @@ public class JobInfo {
|
|||
|
||||
this.acls = new ArrayList<ConfEntryInfo>();
|
||||
if (hasAccess) {
|
||||
this.diagnostics = "";
|
||||
countTasksAndAttempts(job);
|
||||
|
||||
this.uberized = job.isUber();
|
||||
|
||||
List<String> diagnostics = job.getDiagnostics();
|
||||
|
@ -213,7 +211,7 @@ public class JobInfo {
|
|||
}
|
||||
|
||||
public String getState() {
|
||||
return this.state;
|
||||
return this.state.toString();
|
||||
}
|
||||
|
||||
public String getUser() {
|
||||
|
@ -272,8 +270,6 @@ public class JobInfo {
|
|||
* the job to get counts for.
|
||||
*/
|
||||
private void countTasksAndAttempts(Job job) {
|
||||
numReduces = 0;
|
||||
numMaps = 0;
|
||||
final Map<TaskId, Task> tasks = job.getTasks();
|
||||
if (tasks == null) {
|
||||
return;
|
||||
|
|
|
@ -38,23 +38,22 @@ public class JobTaskAttemptCounterInfo {
|
|||
protected Counters total = null;
|
||||
|
||||
protected String id;
|
||||
protected ArrayList<TaskCounterGroupInfo> taskCounterGroups;
|
||||
protected ArrayList<TaskCounterGroupInfo> taskAttemptCounterGroup;
|
||||
|
||||
public JobTaskAttemptCounterInfo() {
|
||||
}
|
||||
|
||||
public JobTaskAttemptCounterInfo(TaskAttempt taskattempt) {
|
||||
|
||||
long value = 0;
|
||||
this.id = MRApps.toString(taskattempt.getID());
|
||||
total = taskattempt.getCounters();
|
||||
taskCounterGroups = new ArrayList<TaskCounterGroupInfo>();
|
||||
taskAttemptCounterGroup = new ArrayList<TaskCounterGroupInfo>();
|
||||
if (total != null) {
|
||||
for (CounterGroup g : total.getAllCounterGroups().values()) {
|
||||
if (g != null) {
|
||||
TaskCounterGroupInfo cginfo = new TaskCounterGroupInfo(g.getName(), g);
|
||||
if (cginfo != null) {
|
||||
taskCounterGroups.add(cginfo);
|
||||
taskAttemptCounterGroup.add(cginfo);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ public class JobTaskCounterInfo {
|
|||
protected Counters total = null;
|
||||
|
||||
protected String id;
|
||||
protected ArrayList<TaskCounterGroupInfo> taskCounterGroups;
|
||||
protected ArrayList<TaskCounterGroupInfo> taskCounterGroup;
|
||||
|
||||
public JobTaskCounterInfo() {
|
||||
}
|
||||
|
@ -46,12 +46,12 @@ public class JobTaskCounterInfo {
|
|||
public JobTaskCounterInfo(Task task) {
|
||||
total = task.getCounters();
|
||||
this.id = MRApps.toString(task.getID());
|
||||
taskCounterGroups = new ArrayList<TaskCounterGroupInfo>();
|
||||
taskCounterGroup = new ArrayList<TaskCounterGroupInfo>();
|
||||
if (total != null) {
|
||||
for (CounterGroup g : total.getAllCounterGroups().values()) {
|
||||
if (g != null) {
|
||||
TaskCounterGroupInfo cginfo = new TaskCounterGroupInfo(g.getName(), g);
|
||||
taskCounterGroups.add(cginfo);
|
||||
taskCounterGroup.add(cginfo);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import javax.xml.bind.annotation.XmlRootElement;
|
|||
import javax.xml.bind.annotation.XmlSeeAlso;
|
||||
import javax.xml.bind.annotation.XmlTransient;
|
||||
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
|
@ -43,7 +44,7 @@ public class TaskAttemptInfo {
|
|||
protected float progress;
|
||||
protected String id;
|
||||
protected String rack;
|
||||
protected String state;
|
||||
protected TaskAttemptState state;
|
||||
protected String nodeHttpAddress;
|
||||
protected String diagnostics;
|
||||
protected String type;
|
||||
|
@ -69,7 +70,7 @@ public class TaskAttemptInfo {
|
|||
.getAssignedContainerID());
|
||||
this.assignedContainer = ta.getAssignedContainerID();
|
||||
this.progress = ta.getProgress() * 100;
|
||||
this.state = ta.getState().toString();
|
||||
this.state = ta.getState();
|
||||
this.elapsedTime = Times
|
||||
.elapsed(this.startTime, this.finishTime, isRunning);
|
||||
if (this.elapsedTime == -1) {
|
||||
|
@ -95,7 +96,7 @@ public class TaskAttemptInfo {
|
|||
}
|
||||
|
||||
public String getState() {
|
||||
return this.state;
|
||||
return this.state.toString();
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
|
|
|
@ -23,21 +23,21 @@ import javax.xml.bind.annotation.XmlAccessType;
|
|||
import javax.xml.bind.annotation.XmlAccessorType;
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
@XmlRootElement(name = "taskattempts")
|
||||
@XmlRootElement(name = "taskAttempts")
|
||||
@XmlAccessorType(XmlAccessType.FIELD)
|
||||
public class TaskAttemptsInfo {
|
||||
|
||||
protected ArrayList<TaskAttemptInfo> taskattempt = new ArrayList<TaskAttemptInfo>();
|
||||
protected ArrayList<TaskAttemptInfo> taskAttempt = new ArrayList<TaskAttemptInfo>();
|
||||
|
||||
public TaskAttemptsInfo() {
|
||||
} // JAXB needs this
|
||||
|
||||
public void add(TaskAttemptInfo taskattemptInfo) {
|
||||
taskattempt.add(taskattemptInfo);
|
||||
taskAttempt.add(taskattemptInfo);
|
||||
}
|
||||
|
||||
public ArrayList<TaskAttemptInfo> getTaskAttempts() {
|
||||
return taskattempt;
|
||||
return taskAttempt;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import javax.xml.bind.annotation.XmlTransient;
|
|||
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
||||
|
@ -39,7 +40,7 @@ public class TaskInfo {
|
|||
protected long elapsedTime;
|
||||
protected float progress;
|
||||
protected String id;
|
||||
protected String state;
|
||||
protected TaskState state;
|
||||
protected String type;
|
||||
protected String successfulAttempt;
|
||||
|
||||
|
@ -62,7 +63,7 @@ public class TaskInfo {
|
|||
if (this.elapsedTime == -1) {
|
||||
this.elapsedTime = 0;
|
||||
}
|
||||
this.state = report.getTaskState().toString();
|
||||
this.state = report.getTaskState();
|
||||
this.progress = report.getProgress() * 100;
|
||||
this.id = MRApps.toString(task.getID());
|
||||
this.taskNum = task.getID().getId();
|
||||
|
@ -79,7 +80,7 @@ public class TaskInfo {
|
|||
}
|
||||
|
||||
public String getState() {
|
||||
return this.state;
|
||||
return this.state.toString();
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
|
|
|
@ -1,39 +1,41 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.app;
|
||||
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobACLsManager;
|
||||
import org.apache.hadoop.mapred.ShuffleHandler;
|
||||
import org.apache.hadoop.mapreduce.FileSystemCounter;
|
||||
import org.apache.hadoop.mapreduce.JobACL;
|
||||
import org.apache.hadoop.mapreduce.JobCounter;
|
||||
import org.apache.hadoop.mapreduce.MRConfig;
|
||||
import org.apache.hadoop.mapreduce.TaskCounter;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
|
@ -48,7 +50,6 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
|||
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
||||
|
@ -63,28 +64,33 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
public class MockJobs extends MockApps {
|
||||
static final Iterator<JobState> JOB_STATES = Iterators.cycle(
|
||||
JobState.values());
|
||||
static final Iterator<TaskState> TASK_STATES = Iterators.cycle(
|
||||
TaskState.values());
|
||||
static final Iterator<TaskAttemptState> TASK_ATTEMPT_STATES = Iterators.cycle(
|
||||
TaskAttemptState.values());
|
||||
static final Iterator<TaskType> TASK_TYPES = Iterators.cycle(
|
||||
TaskType.values());
|
||||
static final Iterator<JobCounter> JOB_COUNTERS = Iterators.cycle(
|
||||
JobCounter.values());
|
||||
static final Iterator<FileSystemCounter> FS_COUNTERS = Iterators.cycle(
|
||||
FileSystemCounter.values());
|
||||
static final Iterator<TaskCounter> TASK_COUNTERS = Iterators.cycle(
|
||||
TaskCounter.values());
|
||||
static final Iterator<JobState> JOB_STATES = Iterators.cycle(JobState
|
||||
.values());
|
||||
static final Iterator<TaskState> TASK_STATES = Iterators.cycle(TaskState
|
||||
.values());
|
||||
static final Iterator<TaskAttemptState> TASK_ATTEMPT_STATES = Iterators
|
||||
.cycle(TaskAttemptState.values());
|
||||
static final Iterator<TaskType> TASK_TYPES = Iterators.cycle(TaskType
|
||||
.values());
|
||||
static final Iterator<JobCounter> JOB_COUNTERS = Iterators.cycle(JobCounter
|
||||
.values());
|
||||
static final Iterator<FileSystemCounter> FS_COUNTERS = Iterators
|
||||
.cycle(FileSystemCounter.values());
|
||||
static final Iterator<TaskCounter> TASK_COUNTERS = Iterators
|
||||
.cycle(TaskCounter.values());
|
||||
static final Iterator<String> FS_SCHEMES = Iterators.cycle("FILE", "HDFS",
|
||||
"LAFS", "CEPH");
|
||||
static final Iterator<String> USER_COUNTER_GROUPS = Iterators.cycle(
|
||||
static final Iterator<String> USER_COUNTER_GROUPS = Iterators
|
||||
.cycle(
|
||||
"com.company.project.subproject.component.subcomponent.UserDefinedSpecificSpecialTask$Counters",
|
||||
"PigCounters");
|
||||
static final Iterator<String> USER_COUNTERS = Iterators.cycle(
|
||||
"counter1", "counter2", "counter3");
|
||||
static final Iterator<String> USER_COUNTERS = Iterators.cycle("counter1",
|
||||
"counter2", "counter3");
|
||||
static final Iterator<Phase> PHASES = Iterators.cycle(Phase.values());
|
||||
static final Iterator<String> DIAGS = Iterators.cycle(
|
||||
"Error: java.lang.OutOfMemoryError: Java heap space",
|
||||
|
@ -101,8 +107,7 @@ public class MockJobs extends MockApps {
|
|||
}
|
||||
|
||||
public static Map<JobId, Job> newJobs(ApplicationId appID, int numJobsPerApp,
|
||||
int numTasksPerJob,
|
||||
int numAttemptsPerTask) {
|
||||
int numTasksPerJob, int numAttemptsPerTask) {
|
||||
Map<JobId, Job> map = Maps.newHashMap();
|
||||
for (int j = 0; j < numJobsPerApp; ++j) {
|
||||
Job job = newJob(appID, j, numTasksPerJob, numAttemptsPerTask);
|
||||
|
@ -121,10 +126,12 @@ public class MockJobs extends MockApps {
|
|||
public static JobReport newJobReport(JobId id) {
|
||||
JobReport report = Records.newRecord(JobReport.class);
|
||||
report.setJobId(id);
|
||||
report.setStartTime(System.currentTimeMillis() - (int)(Math.random() * DT));
|
||||
report.setFinishTime(System.currentTimeMillis() + (int)(Math.random() * DT) + 1);
|
||||
report.setMapProgress((float)Math.random());
|
||||
report.setReduceProgress((float)Math.random());
|
||||
report
|
||||
.setStartTime(System.currentTimeMillis() - (int) (Math.random() * DT));
|
||||
report.setFinishTime(System.currentTimeMillis()
|
||||
+ (int) (Math.random() * DT) + 1);
|
||||
report.setMapProgress((float) Math.random());
|
||||
report.setReduceProgress((float) Math.random());
|
||||
report.setJobState(JOB_STATES.next());
|
||||
return report;
|
||||
}
|
||||
|
@ -132,9 +139,11 @@ public class MockJobs extends MockApps {
|
|||
public static TaskReport newTaskReport(TaskId id) {
|
||||
TaskReport report = Records.newRecord(TaskReport.class);
|
||||
report.setTaskId(id);
|
||||
report.setStartTime(System.currentTimeMillis() - (int)(Math.random() * DT));
|
||||
report.setFinishTime(System.currentTimeMillis() + (int)(Math.random() * DT) + 1);
|
||||
report.setProgress((float)Math.random());
|
||||
report
|
||||
.setStartTime(System.currentTimeMillis() - (int) (Math.random() * DT));
|
||||
report.setFinishTime(System.currentTimeMillis()
|
||||
+ (int) (Math.random() * DT) + 1);
|
||||
report.setProgress((float) Math.random());
|
||||
report.setCounters(newCounters());
|
||||
report.setTaskState(TASK_STATES.next());
|
||||
return report;
|
||||
|
@ -143,35 +152,36 @@ public class MockJobs extends MockApps {
|
|||
public static TaskAttemptReport newTaskAttemptReport(TaskAttemptId id) {
|
||||
TaskAttemptReport report = Records.newRecord(TaskAttemptReport.class);
|
||||
report.setTaskAttemptId(id);
|
||||
report.setStartTime(System.currentTimeMillis() - (int)(Math.random() * DT));
|
||||
report.setFinishTime(System.currentTimeMillis() + (int)(Math.random() * DT) + 1);
|
||||
report
|
||||
.setStartTime(System.currentTimeMillis() - (int) (Math.random() * DT));
|
||||
report.setFinishTime(System.currentTimeMillis()
|
||||
+ (int) (Math.random() * DT) + 1);
|
||||
report.setPhase(PHASES.next());
|
||||
report.setTaskAttemptState(TASK_ATTEMPT_STATES.next());
|
||||
report.setProgress((float)Math.random());
|
||||
report.setProgress((float) Math.random());
|
||||
report.setCounters(newCounters());
|
||||
return report;
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public static Counters newCounters() {
|
||||
org.apache.hadoop.mapred.Counters hc =
|
||||
new org.apache.hadoop.mapred.Counters();
|
||||
org.apache.hadoop.mapred.Counters hc = new org.apache.hadoop.mapred.Counters();
|
||||
for (JobCounter c : JobCounter.values()) {
|
||||
hc.findCounter(c).setValue((long)(Math.random() * 1000));
|
||||
hc.findCounter(c).setValue((long) (Math.random() * 1000));
|
||||
}
|
||||
for (TaskCounter c : TaskCounter.values()) {
|
||||
hc.findCounter(c).setValue((long)(Math.random() * 1000));
|
||||
hc.findCounter(c).setValue((long) (Math.random() * 1000));
|
||||
}
|
||||
int nc = FileSystemCounter.values().length * 4;
|
||||
for (int i = 0; i < nc; ++i) {
|
||||
for (FileSystemCounter c : FileSystemCounter.values()) {
|
||||
hc.findCounter(FS_SCHEMES.next(), c).
|
||||
setValue((long)(Math.random() * DT));
|
||||
hc.findCounter(FS_SCHEMES.next(), c).setValue(
|
||||
(long) (Math.random() * DT));
|
||||
}
|
||||
}
|
||||
for (int i = 0; i < 2 * 3; ++i) {
|
||||
hc.findCounter(USER_COUNTER_GROUPS.next(), USER_COUNTERS.next()).
|
||||
setValue((long)(Math.random() * 100000));
|
||||
hc.findCounter(USER_COUNTER_GROUPS.next(), USER_COUNTERS.next())
|
||||
.setValue((long) (Math.random() * 100000));
|
||||
}
|
||||
return TypeConverter.toYarn(hc);
|
||||
}
|
||||
|
@ -239,7 +249,8 @@ public class MockJobs extends MockApps {
|
|||
switch (report.getTaskAttemptState()) {
|
||||
case SUCCEEDED:
|
||||
case FAILED:
|
||||
case KILLED: return true;
|
||||
case KILLED:
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
@ -247,8 +258,8 @@ public class MockJobs extends MockApps {
|
|||
@Override
|
||||
public ContainerId getAssignedContainerID() {
|
||||
ContainerId id = Records.newRecord(ContainerId.class);
|
||||
ApplicationAttemptId appAttemptId =
|
||||
Records.newRecord(ApplicationAttemptId.class);
|
||||
ApplicationAttemptId appAttemptId = Records
|
||||
.newRecord(ApplicationAttemptId.class);
|
||||
appAttemptId.setApplicationId(taid.getTaskId().getJobId().getAppId());
|
||||
appAttemptId.setAttemptId(0);
|
||||
id.setApplicationAttemptId(appAttemptId);
|
||||
|
@ -344,7 +355,8 @@ public class MockJobs extends MockApps {
|
|||
switch (report.getTaskState()) {
|
||||
case SUCCEEDED:
|
||||
case KILLED:
|
||||
case FAILED: return true;
|
||||
case FAILED:
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
@ -398,12 +410,26 @@ public class MockJobs extends MockApps {
|
|||
}
|
||||
|
||||
public static Job newJob(ApplicationId appID, int i, int n, int m) {
|
||||
return newJob(appID, i, n, m, null);
|
||||
}
|
||||
|
||||
public static Job newJob(ApplicationId appID, int i, int n, int m, Path confFile) {
|
||||
final JobId id = newJobID(appID, i);
|
||||
final String name = newJobName();
|
||||
final JobReport report = newJobReport(id);
|
||||
final Map<TaskId, Task> tasks = newTasks(id, n, m);
|
||||
final TaskCount taskCount = getTaskCount(tasks.values());
|
||||
final Counters counters = getCounters(tasks.values());
|
||||
final Path configFile = confFile;
|
||||
|
||||
Map<JobACL, AccessControlList> tmpJobACLs = new HashMap<JobACL, AccessControlList>();
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(JobACL.VIEW_JOB.getAclName(), "testuser");
|
||||
conf.setBoolean(MRConfig.MR_ACLS_ENABLED, true);
|
||||
|
||||
JobACLsManager aclsManager = new JobACLsManager(conf);
|
||||
tmpJobACLs = aclsManager.constructJobACLs(conf);
|
||||
final Map<JobACL, AccessControlList> jobACLs = tmpJobACLs;
|
||||
return new Job() {
|
||||
@Override
|
||||
public JobId getID() {
|
||||
|
@ -483,7 +509,7 @@ public class MockJobs extends MockApps {
|
|||
|
||||
@Override
|
||||
public List<String> getDiagnostics() {
|
||||
return Collections.<String>emptyList();
|
||||
return Collections.<String> emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -504,12 +530,12 @@ public class MockJobs extends MockApps {
|
|||
|
||||
@Override
|
||||
public Path getConfFile() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
return configFile;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobACL, AccessControlList> getJobACLs() {
|
||||
return Collections.<JobACL, AccessControlList>emptyMap();
|
||||
return jobACLs;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -523,8 +549,7 @@ public class MockJobs extends MockApps {
|
|||
}
|
||||
|
||||
private static AMInfo createAMInfo(int attempt) {
|
||||
ApplicationAttemptId appAttemptId =
|
||||
BuilderUtils.newApplicationAttemptId(
|
||||
ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
|
||||
BuilderUtils.newApplicationId(100, 1), attempt);
|
||||
ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
|
||||
return MRBuilderUtils.newAMInfo(appAttemptId, System.currentTimeMillis(),
|
||||
|
|
|
@ -0,0 +1,359 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.app.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
import com.sun.jersey.api.client.UniformInterfaceException;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the MapReduce Application master info web services api's. Also test
|
||||
* non-existent urls.
|
||||
*
|
||||
* /ws/v1/mapreduce
|
||||
* /ws/v1/mapreduce/info
|
||||
*/
|
||||
public class TestAMWebServices extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
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);
|
||||
}
|
||||
|
||||
TestAppContext() {
|
||||
this(0, 1, 1, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
appContext = new TestAppContext();
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(AMWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
}
|
||||
|
||||
public TestAMWebServices() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.app.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAM() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyAMInfo(json.getJSONObject("info"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAMSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyAMInfo(json.getJSONObject("info"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAMDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce/")
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyAMInfo(json.getJSONObject("info"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAMXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
verifyAMInfoXML(xml, appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInfo() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("info").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyAMInfo(json.getJSONObject("info"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInfoSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("info/").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyAMInfo(json.getJSONObject("info"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInfoDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("info/").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyAMInfo(json.getJSONObject("info"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInfoXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("info/").accept(MediaType.APPLICATION_XML)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
verifyAMInfoXML(xml, appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidUri() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
String responseStr = "";
|
||||
try {
|
||||
responseStr = r.path("ws").path("v1").path("mapreduce").path("bogus")
|
||||
.accept(MediaType.APPLICATION_JSON).get(String.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
WebServicesTestUtils.checkStringMatch(
|
||||
"error string exists and shouldn't", "", responseStr);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidUri2() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
String responseStr = "";
|
||||
try {
|
||||
responseStr = r.path("ws").path("v1").path("invalid")
|
||||
.accept(MediaType.APPLICATION_JSON).get(String.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
WebServicesTestUtils.checkStringMatch(
|
||||
"error string exists and shouldn't", "", responseStr);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidAccept() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
String responseStr = "";
|
||||
try {
|
||||
responseStr = r.path("ws").path("v1").path("mapreduce")
|
||||
.accept(MediaType.TEXT_PLAIN).get(String.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.INTERNAL_SERVER_ERROR,
|
||||
response.getClientResponseStatus());
|
||||
WebServicesTestUtils.checkStringMatch(
|
||||
"error string exists and shouldn't", "", responseStr);
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMInfo(JSONObject info, TestAppContext ctx)
|
||||
throws JSONException {
|
||||
assertEquals("incorrect number of elements", 5, info.length());
|
||||
|
||||
verifyAMInfoGeneric(ctx, info.getString("appId"), info.getString("user"),
|
||||
info.getString("name"), info.getLong("startedOn"),
|
||||
info.getLong("elapsedTime"));
|
||||
}
|
||||
|
||||
public void verifyAMInfoXML(String xml, TestAppContext ctx)
|
||||
throws JSONException, Exception {
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList nodes = dom.getElementsByTagName("info");
|
||||
assertEquals("incorrect number of elements", 1, nodes.getLength());
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
verifyAMInfoGeneric(ctx,
|
||||
WebServicesTestUtils.getXmlString(element, "appId"),
|
||||
WebServicesTestUtils.getXmlString(element, "user"),
|
||||
WebServicesTestUtils.getXmlString(element, "name"),
|
||||
WebServicesTestUtils.getXmlLong(element, "startedOn"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedTime"));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMInfoGeneric(TestAppContext ctx, String id, String user,
|
||||
String name, long startedOn, long elapsedTime) {
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", ctx.getApplicationID()
|
||||
.toString(), id);
|
||||
WebServicesTestUtils.checkStringMatch("user", ctx.getUser().toString(),
|
||||
user);
|
||||
WebServicesTestUtils.checkStringMatch("name", ctx.getApplicationName(),
|
||||
name);
|
||||
|
||||
assertEquals("startedOn incorrect", ctx.getStartTime(), startedOn);
|
||||
assertTrue("elapsedTime not greater then 0", (elapsedTime > 0));
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,732 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.app.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
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.app.job.Task;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONArray;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.UniformInterfaceException;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the app master web service Rest API for getting task attempts, a
|
||||
* specific task attempt, and task attempt counters
|
||||
*
|
||||
* /ws/v1/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts
|
||||
* /ws/v1/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}
|
||||
* /ws/v1/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/counters
|
||||
*/
|
||||
public class TestAMWebServicesAttempts extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
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);
|
||||
}
|
||||
|
||||
TestAppContext() {
|
||||
this(0, 1, 2, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
appContext = new TestAppContext();
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(AMWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
}
|
||||
|
||||
public TestAMWebServicesAttempts() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.app.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttempts() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid).path("attempts")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
verifyAMTaskAttempts(json, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptsSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid).path("attempts/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
verifyAMTaskAttempts(json, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptsDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid).path("attempts")
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
verifyAMTaskAttempts(json, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptsXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid).path("attempts")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList attempts = dom.getElementsByTagName("taskAttempts");
|
||||
assertEquals("incorrect number of elements", 1, attempts.getLength());
|
||||
|
||||
NodeList nodes = dom.getElementsByTagName("taskAttempt");
|
||||
verifyAMTaskAttemptsXML(nodes, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptId() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").path(attid).accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("taskAttempt");
|
||||
verifyAMTaskAttempt(info, att, task.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").path(attid + "/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("taskAttempt");
|
||||
verifyAMTaskAttempt(info, att, task.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").path(attid).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("taskAttempt");
|
||||
verifyAMTaskAttempt(info, att, task.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").path(attid).accept(MediaType.APPLICATION_XML)
|
||||
.get(ClientResponse.class);
|
||||
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList nodes = dom.getElementsByTagName("taskAttempt");
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
verifyAMTaskAttemptXML(element, att, task.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdBogus() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric("bogusid",
|
||||
"java.lang.Exception: Error parsing attempt ID: bogusid");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdNonExist() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric(
|
||||
"attempt_12345_0_0_r_1_0",
|
||||
"java.lang.Exception: Error getting info on task attempt id attempt_12345_0_0_r_1_0");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdInvalid() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric("attempt_12345_0_0_d_1_0",
|
||||
"java.lang.Exception: Unknown task symbol: d");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdInvalid2() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric("attempt_12345_0_r_1_0",
|
||||
"java.lang.Exception: For input string: \"r\"");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdInvalid3() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric("attempt_12345_0_0_r_1",
|
||||
"java.lang.Exception: Error parsing attempt ID: attempt_12345_0_0_r_1");
|
||||
}
|
||||
|
||||
private void testTaskAttemptIdErrorGeneric(String attid, String error)
|
||||
throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs").path(jobId)
|
||||
.path("tasks").path(tid).path("attempts").path(attid)
|
||||
.accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message", error,
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMTaskAttemptXML(Element element, TaskAttempt att,
|
||||
TaskType ttype) {
|
||||
verifyTaskAttemptGeneric(att, ttype,
|
||||
WebServicesTestUtils.getXmlString(element, "id"),
|
||||
WebServicesTestUtils.getXmlString(element, "state"),
|
||||
WebServicesTestUtils.getXmlString(element, "type"),
|
||||
WebServicesTestUtils.getXmlString(element, "rack"),
|
||||
WebServicesTestUtils.getXmlString(element, "nodeHttpAddress"),
|
||||
WebServicesTestUtils.getXmlString(element, "diagnostics"),
|
||||
WebServicesTestUtils.getXmlString(element, "assignedContainerId"),
|
||||
WebServicesTestUtils.getXmlLong(element, "startTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "finishTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedTime"),
|
||||
WebServicesTestUtils.getXmlFloat(element, "progress"));
|
||||
|
||||
if (ttype == TaskType.REDUCE) {
|
||||
verifyReduceTaskAttemptGeneric(att,
|
||||
WebServicesTestUtils.getXmlLong(element, "shuffleFinishTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "mergeFinishTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedShuffleTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedMergeTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedReduceTime"));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMTaskAttempt(JSONObject info, TaskAttempt att,
|
||||
TaskType ttype) throws JSONException {
|
||||
if (ttype == TaskType.REDUCE) {
|
||||
assertEquals("incorrect number of elements", 16, info.length());
|
||||
} else {
|
||||
assertEquals("incorrect number of elements", 11, info.length());
|
||||
}
|
||||
|
||||
verifyTaskAttemptGeneric(att, ttype, info.getString("id"),
|
||||
info.getString("state"), info.getString("type"),
|
||||
info.getString("rack"), info.getString("nodeHttpAddress"),
|
||||
info.getString("diagnostics"), info.getString("assignedContainerId"),
|
||||
info.getLong("startTime"), info.getLong("finishTime"),
|
||||
info.getLong("elapsedTime"), (float) info.getDouble("progress"));
|
||||
|
||||
if (ttype == TaskType.REDUCE) {
|
||||
verifyReduceTaskAttemptGeneric(att, info.getLong("shuffleFinishTime"),
|
||||
info.getLong("mergeFinishTime"), info.getLong("elapsedShuffleTime"),
|
||||
info.getLong("elapsedMergeTime"), info.getLong("elapsedReduceTime"));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMTaskAttempts(JSONObject json, Task task)
|
||||
throws JSONException {
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject attempts = json.getJSONObject("taskAttempts");
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONArray arr = attempts.getJSONArray("taskAttempt");
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId id = att.getID();
|
||||
String attid = MRApps.toString(id);
|
||||
Boolean found = false;
|
||||
|
||||
for (int i = 0; i < arr.length(); i++) {
|
||||
JSONObject info = arr.getJSONObject(i);
|
||||
if (attid.matches(info.getString("id"))) {
|
||||
found = true;
|
||||
verifyAMTaskAttempt(info, att, task.getType());
|
||||
}
|
||||
}
|
||||
assertTrue("task attempt with id: " + attid
|
||||
+ " not in web service output", found);
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMTaskAttemptsXML(NodeList nodes, Task task) {
|
||||
assertEquals("incorrect number of elements", 1, nodes.getLength());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId id = att.getID();
|
||||
String attid = MRApps.toString(id);
|
||||
Boolean found = false;
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
|
||||
if (attid.matches(WebServicesTestUtils.getXmlString(element, "id"))) {
|
||||
found = true;
|
||||
verifyAMTaskAttemptXML(element, att, task.getType());
|
||||
}
|
||||
}
|
||||
assertTrue("task with id: " + attid + " not in web service output", found);
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyTaskAttemptGeneric(TaskAttempt ta, TaskType ttype,
|
||||
String id, String state, String type, String rack,
|
||||
String nodeHttpAddress, String diagnostics, String assignedContainerId,
|
||||
long startTime, long finishTime, long elapsedTime, float progress) {
|
||||
|
||||
TaskAttemptId attid = ta.getID();
|
||||
String attemptId = MRApps.toString(attid);
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", attemptId, id);
|
||||
WebServicesTestUtils.checkStringMatch("type", ttype.toString(), type);
|
||||
WebServicesTestUtils.checkStringMatch("state", ta.getState().toString(),
|
||||
state);
|
||||
WebServicesTestUtils.checkStringMatch("rack", ta.getNodeRackName(), rack);
|
||||
WebServicesTestUtils.checkStringMatch("nodeHttpAddress",
|
||||
ta.getNodeHttpAddress(), nodeHttpAddress);
|
||||
|
||||
String expectDiag = "";
|
||||
List<String> diagnosticsList = ta.getDiagnostics();
|
||||
if (diagnosticsList != null && !diagnostics.isEmpty()) {
|
||||
StringBuffer b = new StringBuffer();
|
||||
for (String diag : diagnosticsList) {
|
||||
b.append(diag);
|
||||
}
|
||||
expectDiag = b.toString();
|
||||
}
|
||||
WebServicesTestUtils.checkStringMatch("diagnostics", expectDiag,
|
||||
diagnostics);
|
||||
WebServicesTestUtils.checkStringMatch("assignedContainerId",
|
||||
ConverterUtils.toString(ta.getAssignedContainerID()),
|
||||
assignedContainerId);
|
||||
|
||||
assertEquals("startTime wrong", ta.getLaunchTime(), startTime);
|
||||
assertEquals("finishTime wrong", ta.getFinishTime(), finishTime);
|
||||
assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime);
|
||||
assertEquals("progress wrong", ta.getProgress() * 100, progress, 1e-3f);
|
||||
}
|
||||
|
||||
public void verifyReduceTaskAttemptGeneric(TaskAttempt ta,
|
||||
long shuffleFinishTime, long mergeFinishTime, long elapsedShuffleTime,
|
||||
long elapsedMergeTime, long elapsedReduceTime) {
|
||||
|
||||
assertEquals("shuffleFinishTime wrong", ta.getShuffleFinishTime(),
|
||||
shuffleFinishTime);
|
||||
assertEquals("mergeFinishTime wrong", ta.getSortFinishTime(),
|
||||
mergeFinishTime);
|
||||
assertEquals("elapsedShuffleTime wrong",
|
||||
ta.getLaunchTime() - ta.getShuffleFinishTime(), elapsedShuffleTime);
|
||||
assertEquals("elapsedMergeTime wrong",
|
||||
ta.getShuffleFinishTime() - ta.getSortFinishTime(), elapsedMergeTime);
|
||||
assertEquals("elapsedReduceTime wrong",
|
||||
ta.getSortFinishTime() - ta.getFinishTime(), elapsedReduceTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdCounters() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").path(attid).path("counters")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("JobTaskAttemptCounters");
|
||||
verifyAMJobTaskAttemptCounters(info, att);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdXMLCounters() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").path(attid).path("counters")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList nodes = dom.getElementsByTagName("JobTaskAttemptCounters");
|
||||
|
||||
verifyAMTaskCountersXML(nodes, att);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMJobTaskAttemptCounters(JSONObject info, TaskAttempt att)
|
||||
throws JSONException {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, info.length());
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(att.getID()),
|
||||
info.getString("id"));
|
||||
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
JSONArray counterGroups = info.getJSONArray("taskAttemptCounterGroup");
|
||||
for (int i = 0; i < counterGroups.length(); i++) {
|
||||
JSONObject counterGroup = counterGroups.getJSONObject(i);
|
||||
String name = counterGroup.getString("counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
JSONArray counters = counterGroup.getJSONArray("counter");
|
||||
for (int j = 0; j < counters.length(); j++) {
|
||||
JSONObject counter = counters.getJSONObject(i);
|
||||
String counterName = counter.getString("name");
|
||||
assertTrue("name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
long value = counter.getLong("value");
|
||||
assertTrue("value >= 0", value >= 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMTaskCountersXML(NodeList nodes, TaskAttempt att) {
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
|
||||
Element element = (Element) nodes.item(i);
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(att.getID()),
|
||||
WebServicesTestUtils.getXmlString(element, "id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
NodeList groups = element.getElementsByTagName("taskAttemptCounterGroup");
|
||||
|
||||
for (int j = 0; j < groups.getLength(); j++) {
|
||||
Element counters = (Element) groups.item(j);
|
||||
assertNotNull("should have counters in the web service info", counters);
|
||||
String name = WebServicesTestUtils.getXmlString(counters,
|
||||
"counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
NodeList counterArr = counters.getElementsByTagName("counter");
|
||||
for (int z = 0; z < counterArr.getLength(); z++) {
|
||||
Element counter = (Element) counterArr.item(z);
|
||||
String counterName = WebServicesTestUtils.getXmlString(counter,
|
||||
"name");
|
||||
assertTrue("counter name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
|
||||
long value = WebServicesTestUtils.getXmlLong(counter, "value");
|
||||
assertTrue("value not >= 0", value >= 0);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,336 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.app.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.io.StringReader;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
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.util.MRApps;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONArray;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the app master web service Rest API for getting the job conf. This
|
||||
* requires created a temporary configuration file.
|
||||
*
|
||||
* /ws/v1/mapreduce/job/{jobid}/conf
|
||||
*/
|
||||
public class TestAMWebServicesJobConf extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
|
||||
private static File testConfDir = new File("target",
|
||||
TestAMWebServicesJobConf.class.getSimpleName() + "confDir");
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
final long startTime = System.currentTimeMillis();
|
||||
|
||||
TestAppContext(int appid, int numTasks, int numAttempts, Path confPath) {
|
||||
appID = MockJobs.newAppID(appid);
|
||||
appAttemptID = MockJobs.newAppAttemptID(appID, 0);
|
||||
Map<JobId, Job> map = Maps.newHashMap();
|
||||
Job job = MockJobs.newJob(appID, 0, numTasks, numAttempts, confPath);
|
||||
map.put(job.getID(), job);
|
||||
jobs = map;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
Path confPath = new Path(testConfDir.toString(),
|
||||
MRJobConfig.JOB_CONF_FILE);
|
||||
Configuration config = new Configuration();
|
||||
|
||||
FileSystem localFs;
|
||||
try {
|
||||
localFs = FileSystem.getLocal(config);
|
||||
confPath = localFs.makeQualified(confPath);
|
||||
|
||||
OutputStream out = localFs.create(confPath);
|
||||
try {
|
||||
conf.writeXml(out);
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
if (!localFs.exists(confPath)) {
|
||||
fail("error creating config file: " + confPath);
|
||||
}
|
||||
|
||||
} catch (IOException e) {
|
||||
fail("error creating config file: " + e.getMessage());
|
||||
}
|
||||
|
||||
appContext = new TestAppContext(0, 2, 1, confPath);
|
||||
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(AMWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
testConfDir.mkdir();
|
||||
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
static public void stop() {
|
||||
FileUtil.fullyDelete(testConfDir);
|
||||
}
|
||||
|
||||
public TestAMWebServicesJobConf() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.app.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobConf() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("conf")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("conf");
|
||||
verifyAMJobConf(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobConfSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("conf/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("conf");
|
||||
verifyAMJobConf(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobConfDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("conf").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("conf");
|
||||
verifyAMJobConf(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobConfXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("conf")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList info = dom.getElementsByTagName("conf");
|
||||
verifyAMJobConfXML(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMJobConf(JSONObject info, Job job) throws JSONException {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, info.length());
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("path", job.getConfFile().toString(),
|
||||
info.getString("path"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
JSONArray properties = info.getJSONArray("property");
|
||||
for (int i = 0; i < properties.length(); i++) {
|
||||
JSONObject prop = properties.getJSONObject(i);
|
||||
String name = prop.getString("name");
|
||||
String value = prop.getString("value");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
assertTrue("value not set", (value != null && !value.isEmpty()));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMJobConfXML(NodeList nodes, Job job) {
|
||||
|
||||
assertEquals("incorrect number of elements", 1, nodes.getLength());
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
WebServicesTestUtils.checkStringMatch("path", job.getConfFile()
|
||||
.toString(), WebServicesTestUtils.getXmlString(element, "path"));
|
||||
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
NodeList properties = element.getElementsByTagName("property");
|
||||
|
||||
for (int j = 0; j < properties.getLength(); j++) {
|
||||
Element property = (Element) properties.item(j);
|
||||
assertNotNull("should have counters in the web service info", property);
|
||||
String name = WebServicesTestUtils.getXmlString(property, "name");
|
||||
String value = WebServicesTestUtils.getXmlString(property, "value");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
assertTrue("name not set", (value != null && !value.isEmpty()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,780 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.app.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapreduce.JobACL;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||
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.util.MRApps;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.util.Times;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONArray;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.UniformInterfaceException;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the app master web service Rest API for getting jobs, a specific job,
|
||||
* and job counters.
|
||||
*
|
||||
* /ws/v1/mapreduce/jobs
|
||||
* /ws/v1/mapreduce/jobs/{jobid}
|
||||
* /ws/v1/mapreduce/jobs/{jobid}/counters
|
||||
*/
|
||||
public class TestAMWebServicesJobs extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
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);
|
||||
}
|
||||
|
||||
TestAppContext() {
|
||||
this(0, 1, 2, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
appContext = new TestAppContext();
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(AMWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
}
|
||||
|
||||
public TestAMWebServicesJobs() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.app.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobs() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
JSONObject info = arr.getJSONObject(0);
|
||||
Job job = appContext.getJob(MRApps.toJobID(info.getString("id")));
|
||||
verifyAMJob(info, job);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs/").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
JSONObject info = arr.getJSONObject(0);
|
||||
Job job = appContext.getJob(MRApps.toJobID(info.getString("id")));
|
||||
verifyAMJob(info, job);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
JSONObject info = arr.getJSONObject(0);
|
||||
Job job = appContext.getJob(MRApps.toJobID(info.getString("id")));
|
||||
verifyAMJob(info, job);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsXML() throws Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").accept(MediaType.APPLICATION_XML)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList jobs = dom.getElementsByTagName("jobs");
|
||||
assertEquals("incorrect number of elements", 1, jobs.getLength());
|
||||
NodeList job = dom.getElementsByTagName("job");
|
||||
assertEquals("incorrect number of elements", 1, job.getLength());
|
||||
verifyAMJobXML(job, appContext);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobId() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("job");
|
||||
verifyAMJob(info, jobsMap.get(id));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId + "/").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("job");
|
||||
verifyAMJob(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("job");
|
||||
verifyAMJob(info, jobsMap.get(id));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdNonExist() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs")
|
||||
.path("job_1234_1_2").get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: job, job_1234_1_2, is not found", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdInvalid() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs").path("job_foo")
|
||||
.get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"For input string: \"foo\"", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NumberFormatException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"java.lang.NumberFormatException", classname);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdInvalidBogus() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs").path("bogusfoo")
|
||||
.get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: Error parsing job ID: bogusfoo", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdXML() throws Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).accept(MediaType.APPLICATION_XML)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList job = dom.getElementsByTagName("job");
|
||||
verifyAMJobXML(job, appContext);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void verifyAMJob(JSONObject info, Job job) throws JSONException {
|
||||
|
||||
assertEquals("incorrect number of elements", 30, info.length());
|
||||
|
||||
// everyone access fields
|
||||
verifyAMJobGeneric(job, info.getString("id"), info.getString("user"),
|
||||
info.getString("name"), info.getString("state"),
|
||||
info.getLong("startTime"), info.getLong("finishTime"),
|
||||
info.getLong("elapsedTime"), info.getInt("mapsTotal"),
|
||||
info.getInt("mapsCompleted"), info.getInt("reducesTotal"),
|
||||
info.getInt("reducesCompleted"),
|
||||
(float) info.getDouble("reduceProgress"),
|
||||
(float) info.getDouble("mapProgress"));
|
||||
|
||||
String diagnostics = "";
|
||||
if (info.has("diagnostics")) {
|
||||
diagnostics = info.getString("diagnostics");
|
||||
}
|
||||
|
||||
// restricted access fields - if security and acls set
|
||||
verifyAMJobGenericSecure(job, info.getInt("mapsPending"),
|
||||
info.getInt("mapsRunning"), info.getInt("reducesPending"),
|
||||
info.getInt("reducesRunning"), info.getBoolean("uberized"),
|
||||
diagnostics, info.getInt("newReduceAttempts"),
|
||||
info.getInt("runningReduceAttempts"),
|
||||
info.getInt("failedReduceAttempts"),
|
||||
info.getInt("killedReduceAttempts"),
|
||||
info.getInt("successfulReduceAttempts"), info.getInt("newMapAttempts"),
|
||||
info.getInt("runningMapAttempts"), info.getInt("failedMapAttempts"),
|
||||
info.getInt("killedMapAttempts"), info.getInt("successfulMapAttempts"));
|
||||
|
||||
Map<JobACL, AccessControlList> allacls = job.getJobACLs();
|
||||
if (allacls != null) {
|
||||
|
||||
for (Map.Entry<JobACL, AccessControlList> entry : allacls.entrySet()) {
|
||||
String expectName = entry.getKey().getAclName();
|
||||
String expectValue = entry.getValue().getAclString();
|
||||
Boolean found = false;
|
||||
// make sure ws includes it
|
||||
if (info.has("acls")) {
|
||||
JSONArray arr = info.getJSONArray("acls");
|
||||
|
||||
for (int i = 0; i < arr.length(); i++) {
|
||||
JSONObject aclInfo = arr.getJSONObject(i);
|
||||
if (expectName.matches(aclInfo.getString("name"))) {
|
||||
found = true;
|
||||
WebServicesTestUtils.checkStringMatch("value", expectValue,
|
||||
aclInfo.getString("value"));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
fail("should have acls in the web service info");
|
||||
}
|
||||
assertTrue("acl: " + expectName + " not found in webservice output",
|
||||
found);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void verifyAMJobXML(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.getJob(MRApps.toJobID(WebServicesTestUtils
|
||||
.getXmlString(element, "id")));
|
||||
assertNotNull("Job not found - output incorrect", job);
|
||||
|
||||
verifyAMJobGeneric(job, WebServicesTestUtils.getXmlString(element, "id"),
|
||||
WebServicesTestUtils.getXmlString(element, "user"),
|
||||
WebServicesTestUtils.getXmlString(element, "name"),
|
||||
WebServicesTestUtils.getXmlString(element, "state"),
|
||||
WebServicesTestUtils.getXmlLong(element, "startTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "finishTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedTime"),
|
||||
WebServicesTestUtils.getXmlInt(element, "mapsTotal"),
|
||||
WebServicesTestUtils.getXmlInt(element, "mapsCompleted"),
|
||||
WebServicesTestUtils.getXmlInt(element, "reducesTotal"),
|
||||
WebServicesTestUtils.getXmlInt(element, "reducesCompleted"),
|
||||
WebServicesTestUtils.getXmlFloat(element, "reduceProgress"),
|
||||
WebServicesTestUtils.getXmlFloat(element, "mapProgress"));
|
||||
|
||||
// restricted access fields - if security and acls set
|
||||
verifyAMJobGenericSecure(job,
|
||||
WebServicesTestUtils.getXmlInt(element, "mapsPending"),
|
||||
WebServicesTestUtils.getXmlInt(element, "mapsRunning"),
|
||||
WebServicesTestUtils.getXmlInt(element, "reducesPending"),
|
||||
WebServicesTestUtils.getXmlInt(element, "reducesRunning"),
|
||||
WebServicesTestUtils.getXmlBoolean(element, "uberized"),
|
||||
WebServicesTestUtils.getXmlString(element, "diagnostics"),
|
||||
WebServicesTestUtils.getXmlInt(element, "newReduceAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "runningReduceAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "failedReduceAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "killedReduceAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "successfulReduceAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "newMapAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "runningMapAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "failedMapAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "killedMapAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "successfulMapAttempts"));
|
||||
|
||||
Map<JobACL, AccessControlList> allacls = job.getJobACLs();
|
||||
if (allacls != null) {
|
||||
for (Map.Entry<JobACL, AccessControlList> entry : allacls.entrySet()) {
|
||||
String expectName = entry.getKey().getAclName();
|
||||
String expectValue = entry.getValue().getAclString();
|
||||
Boolean found = false;
|
||||
// make sure ws includes it
|
||||
NodeList id = element.getElementsByTagName("acls");
|
||||
if (id != null) {
|
||||
for (int j = 0; j < id.getLength(); j++) {
|
||||
Element aclElem = (Element) id.item(j);
|
||||
if (aclElem == null) {
|
||||
fail("should have acls in the web service info");
|
||||
}
|
||||
if (expectName.matches(WebServicesTestUtils.getXmlString(aclElem,
|
||||
"name"))) {
|
||||
found = true;
|
||||
WebServicesTestUtils.checkStringMatch("value", expectValue,
|
||||
WebServicesTestUtils.getXmlString(aclElem, "value"));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
fail("should have acls in the web service info");
|
||||
}
|
||||
assertTrue("acl: " + expectName + " not found in webservice output",
|
||||
found);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMJobGeneric(Job job, String id, String user, String name,
|
||||
String state, long startTime, long finishTime, long elapsedTime,
|
||||
int mapsTotal, int mapsCompleted, int reducesTotal, int reducesCompleted,
|
||||
float reduceProgress, float mapProgress) {
|
||||
JobReport report = job.getReport();
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
|
||||
id);
|
||||
WebServicesTestUtils.checkStringMatch("user", job.getUserName().toString(),
|
||||
user);
|
||||
WebServicesTestUtils.checkStringMatch("name", job.getName(), name);
|
||||
WebServicesTestUtils.checkStringMatch("state", job.getState().toString(),
|
||||
state);
|
||||
|
||||
assertEquals("startTime incorrect", report.getStartTime(), startTime);
|
||||
assertEquals("finishTime incorrect", report.getFinishTime(), finishTime);
|
||||
assertEquals("elapsedTime incorrect",
|
||||
Times.elapsed(report.getStartTime(), report.getFinishTime()),
|
||||
elapsedTime);
|
||||
assertEquals("mapsTotal incorrect", job.getTotalMaps(), mapsTotal);
|
||||
assertEquals("mapsCompleted incorrect", job.getCompletedMaps(),
|
||||
mapsCompleted);
|
||||
assertEquals("reducesTotal incorrect", job.getTotalReduces(), reducesTotal);
|
||||
assertEquals("reducesCompleted incorrect", job.getCompletedReduces(),
|
||||
reducesCompleted);
|
||||
assertEquals("mapProgress incorrect", report.getMapProgress() * 100,
|
||||
mapProgress, 0);
|
||||
assertEquals("reduceProgress incorrect", report.getReduceProgress() * 100,
|
||||
reduceProgress, 0);
|
||||
}
|
||||
|
||||
public void verifyAMJobGenericSecure(Job job, int mapsPending,
|
||||
int mapsRunning, int reducesPending, int reducesRunning,
|
||||
Boolean uberized, String diagnostics, int newReduceAttempts,
|
||||
int runningReduceAttempts, int failedReduceAttempts,
|
||||
int killedReduceAttempts, int successfulReduceAttempts,
|
||||
int newMapAttempts, int runningMapAttempts, int failedMapAttempts,
|
||||
int killedMapAttempts, int successfulMapAttempts) {
|
||||
|
||||
String diagString = "";
|
||||
List<String> diagList = job.getDiagnostics();
|
||||
if (diagList != null && !diagList.isEmpty()) {
|
||||
StringBuffer b = new StringBuffer();
|
||||
for (String diag : diagList) {
|
||||
b.append(diag);
|
||||
}
|
||||
diagString = b.toString();
|
||||
}
|
||||
WebServicesTestUtils.checkStringMatch("diagnostics", diagString,
|
||||
diagnostics);
|
||||
|
||||
assertEquals("isUber incorrect", job.isUber(), uberized);
|
||||
|
||||
// unfortunately the following fields are all calculated in JobInfo
|
||||
// so not easily accessible without doing all the calculations again.
|
||||
// For now just make sure they are present.
|
||||
assertTrue("mapsPending not >= 0", mapsPending >= 0);
|
||||
assertTrue("mapsRunning not >= 0", mapsRunning >= 0);
|
||||
assertTrue("reducesPending not >= 0", reducesPending >= 0);
|
||||
assertTrue("reducesRunning not >= 0", reducesRunning >= 0);
|
||||
|
||||
assertTrue("newReduceAttempts not >= 0", newReduceAttempts >= 0);
|
||||
assertTrue("runningReduceAttempts not >= 0", runningReduceAttempts >= 0);
|
||||
assertTrue("failedReduceAttempts not >= 0", failedReduceAttempts >= 0);
|
||||
assertTrue("killedReduceAttempts not >= 0", killedReduceAttempts >= 0);
|
||||
assertTrue("successfulReduceAttempts not >= 0",
|
||||
successfulReduceAttempts >= 0);
|
||||
|
||||
assertTrue("newMapAttempts not >= 0", newMapAttempts >= 0);
|
||||
assertTrue("runningMapAttempts not >= 0", runningMapAttempts >= 0);
|
||||
assertTrue("failedMapAttempts not >= 0", failedMapAttempts >= 0);
|
||||
assertTrue("killedMapAttempts not >= 0", killedMapAttempts >= 0);
|
||||
assertTrue("successfulMapAttempts not >= 0", successfulMapAttempts >= 0);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobCounters() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("counters")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobCounters");
|
||||
verifyAMJobCounters(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobCountersSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("counters/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobCounters");
|
||||
verifyAMJobCounters(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobCountersDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("counters/").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobCounters");
|
||||
verifyAMJobCounters(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobCountersXML() throws Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("counters")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList info = dom.getElementsByTagName("jobCounters");
|
||||
verifyAMJobCountersXML(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMJobCounters(JSONObject info, Job job)
|
||||
throws JSONException {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, info.length());
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
|
||||
info.getString("id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
JSONArray counterGroups = info.getJSONArray("counterGroup");
|
||||
for (int i = 0; i < counterGroups.length(); i++) {
|
||||
JSONObject counterGroup = counterGroups.getJSONObject(i);
|
||||
String name = counterGroup.getString("counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
JSONArray counters = counterGroup.getJSONArray("counter");
|
||||
for (int j = 0; j < counters.length(); j++) {
|
||||
JSONObject counter = counters.getJSONObject(i);
|
||||
String counterName = counter.getString("name");
|
||||
assertTrue("counter name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
|
||||
long mapValue = counter.getLong("mapCounterValue");
|
||||
assertTrue("mapCounterValue >= 0", mapValue >= 0);
|
||||
|
||||
long reduceValue = counter.getLong("reduceCounterValue");
|
||||
assertTrue("reduceCounterValue >= 0", reduceValue >= 0);
|
||||
|
||||
long totalValue = counter.getLong("totalCounterValue");
|
||||
assertTrue("totalCounterValue >= 0", totalValue >= 0);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMJobCountersXML(NodeList nodes, Job job) {
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
|
||||
assertNotNull("Job not found - output incorrect", job);
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
|
||||
WebServicesTestUtils.getXmlString(element, "id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
NodeList groups = element.getElementsByTagName("counterGroup");
|
||||
|
||||
for (int j = 0; j < groups.getLength(); j++) {
|
||||
Element counters = (Element) groups.item(j);
|
||||
assertNotNull("should have counters in the web service info", counters);
|
||||
String name = WebServicesTestUtils.getXmlString(counters,
|
||||
"counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
NodeList counterArr = counters.getElementsByTagName("counter");
|
||||
for (int z = 0; z < counterArr.getLength(); z++) {
|
||||
Element counter = (Element) counterArr.item(z);
|
||||
String counterName = WebServicesTestUtils.getXmlString(counter,
|
||||
"name");
|
||||
assertTrue("counter name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
|
||||
long mapValue = WebServicesTestUtils.getXmlLong(counter,
|
||||
"mapCounterValue");
|
||||
assertTrue("mapCounterValue not >= 0", mapValue >= 0);
|
||||
|
||||
long reduceValue = WebServicesTestUtils.getXmlLong(counter,
|
||||
"reduceCounterValue");
|
||||
assertTrue("reduceCounterValue >= 0", reduceValue >= 0);
|
||||
|
||||
long totalValue = WebServicesTestUtils.getXmlLong(counter,
|
||||
"totalCounterValue");
|
||||
assertTrue("totalCounterValue >= 0", totalValue >= 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,821 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.app.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
|
||||
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.app.job.Task;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONArray;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
import com.sun.jersey.api.client.UniformInterfaceException;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the app master web service Rest API for getting tasks, a specific task,
|
||||
* and task counters.
|
||||
*
|
||||
* /ws/v1/mapreduce/jobs/{jobid}/tasks
|
||||
* /ws/v1/mapreduce/jobs/{jobid}/tasks/{taskid}
|
||||
* /ws/v1/mapreduce/jobs/{jobid}/tasks/{taskid}/counters
|
||||
*/
|
||||
public class TestAMWebServicesTasks extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
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);
|
||||
}
|
||||
|
||||
TestAppContext() {
|
||||
this(0, 1, 2, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
appContext = new TestAppContext();
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(AMWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
}
|
||||
|
||||
public TestAMWebServicesTasks() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.app.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasks() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 2, arr.length());
|
||||
|
||||
verifyAMTask(arr, jobsMap.get(id), null);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 2, arr.length());
|
||||
|
||||
verifyAMTask(arr, jobsMap.get(id), null);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 2, arr.length());
|
||||
|
||||
verifyAMTask(arr, jobsMap.get(id), null);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksXML() throws JSONException, Exception {
|
||||
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList tasks = dom.getElementsByTagName("tasks");
|
||||
assertEquals("incorrect number of elements", 1, tasks.getLength());
|
||||
NodeList task = dom.getElementsByTagName("task");
|
||||
verifyAMTaskXML(task, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksQueryMap() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String type = "m";
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").queryParam("type", type)
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 1, arr.length());
|
||||
verifyAMTask(arr, jobsMap.get(id), type);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksQueryReduce() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String type = "r";
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").queryParam("type", type)
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 1, arr.length());
|
||||
verifyAMTask(arr, jobsMap.get(id), type);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksQueryInvalid() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
// tasktype must be exactly either "m" or "r"
|
||||
String tasktype = "reduce";
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs").path(jobId)
|
||||
.path("tasks").queryParam("type", tasktype)
|
||||
.accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: tasktype must be either m or r", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskId() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("task");
|
||||
verifyAMSingleTask(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid + "/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("task");
|
||||
verifyAMSingleTask(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("task");
|
||||
verifyAMSingleTask(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdBogus() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "bogustaskid";
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs").path(jobId)
|
||||
.path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: Error parsing task ID: bogustaskid", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdNonExist() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "task_1234_0_0_m_0";
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs").path(jobId)
|
||||
.path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: task not found with id task_1234_0_0_m_0",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdInvalid() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "task_1234_0_0_d_0";
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs").path(jobId)
|
||||
.path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: Unknown task symbol: d", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdInvalid2() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "task_1234_0_m_0";
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs").path(jobId)
|
||||
.path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: For input string: \"m\"", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdInvalid3() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "task_1234_0_0_m";
|
||||
try {
|
||||
r.path("ws").path("v1").path("mapreduce").path("jobs").path(jobId)
|
||||
.path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: Error parsing task ID: task_1234_0_0_m",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList nodes = dom.getElementsByTagName("task");
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
verifyAMSingleTaskXML(element, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMSingleTask(JSONObject info, Task task)
|
||||
throws JSONException {
|
||||
assertEquals("incorrect number of elements", 8, info.length());
|
||||
|
||||
verifyTaskGeneric(task, info.getString("id"), info.getString("state"),
|
||||
info.getString("type"), info.getString("successfulAttempt"),
|
||||
info.getLong("startTime"), info.getLong("finishTime"),
|
||||
info.getLong("elapsedTime"), (float) info.getDouble("progress"));
|
||||
}
|
||||
|
||||
public void verifyAMTask(JSONArray arr, Job job, String type)
|
||||
throws JSONException {
|
||||
for (Task task : job.getTasks().values()) {
|
||||
TaskId id = task.getID();
|
||||
String tid = MRApps.toString(id);
|
||||
Boolean found = false;
|
||||
if (type != null && task.getType() == MRApps.taskType(type)) {
|
||||
|
||||
for (int i = 0; i < arr.length(); i++) {
|
||||
JSONObject info = arr.getJSONObject(i);
|
||||
if (tid.matches(info.getString("id"))) {
|
||||
found = true;
|
||||
verifyAMSingleTask(info, task);
|
||||
}
|
||||
}
|
||||
assertTrue("task with id: " + tid + " not in web service output", found);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyTaskGeneric(Task task, String id, String state,
|
||||
String type, String successfulAttempt, long startTime, long finishTime,
|
||||
long elapsedTime, float progress) {
|
||||
|
||||
TaskId taskid = task.getID();
|
||||
String tid = MRApps.toString(taskid);
|
||||
TaskReport report = task.getReport();
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", tid, id);
|
||||
WebServicesTestUtils.checkStringMatch("type", task.getType().toString(),
|
||||
type);
|
||||
WebServicesTestUtils.checkStringMatch("state", report.getTaskState()
|
||||
.toString(), state);
|
||||
// not easily checked without duplicating logic, just make sure its here
|
||||
assertNotNull("successfulAttempt null", successfulAttempt);
|
||||
assertEquals("startTime wrong", report.getStartTime(), startTime);
|
||||
assertEquals("finishTime wrong", report.getFinishTime(), finishTime);
|
||||
assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime);
|
||||
assertEquals("progress wrong", report.getProgress() * 100, progress, 1e-3f);
|
||||
}
|
||||
|
||||
public void verifyAMSingleTaskXML(Element element, Task task) {
|
||||
verifyTaskGeneric(task, WebServicesTestUtils.getXmlString(element, "id"),
|
||||
WebServicesTestUtils.getXmlString(element, "state"),
|
||||
WebServicesTestUtils.getXmlString(element, "type"),
|
||||
WebServicesTestUtils.getXmlString(element, "successfulAttempt"),
|
||||
WebServicesTestUtils.getXmlLong(element, "startTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "finishTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedTime"),
|
||||
WebServicesTestUtils.getXmlFloat(element, "progress"));
|
||||
}
|
||||
|
||||
public void verifyAMTaskXML(NodeList nodes, Job job) {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, nodes.getLength());
|
||||
|
||||
for (Task task : job.getTasks().values()) {
|
||||
TaskId id = task.getID();
|
||||
String tid = MRApps.toString(id);
|
||||
Boolean found = false;
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
|
||||
if (tid.matches(WebServicesTestUtils.getXmlString(element, "id"))) {
|
||||
found = true;
|
||||
verifyAMSingleTaskXML(element, task);
|
||||
}
|
||||
}
|
||||
assertTrue("task with id: " + tid + " not in web service output", found);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdCounters() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid).path("counters")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobTaskCounters");
|
||||
verifyAMJobTaskCounters(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdCountersSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid).path("counters/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobTaskCounters");
|
||||
verifyAMJobTaskCounters(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdCountersDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid).path("counters")
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobTaskCounters");
|
||||
verifyAMJobTaskCounters(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobTaskCountersXML() throws Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid).path("counters")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList info = dom.getElementsByTagName("jobTaskCounters");
|
||||
verifyAMTaskCountersXML(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMJobTaskCounters(JSONObject info, Task task)
|
||||
throws JSONException {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, info.length());
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(task.getID()),
|
||||
info.getString("id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
JSONArray counterGroups = info.getJSONArray("taskCounterGroup");
|
||||
for (int i = 0; i < counterGroups.length(); i++) {
|
||||
JSONObject counterGroup = counterGroups.getJSONObject(i);
|
||||
String name = counterGroup.getString("counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
JSONArray counters = counterGroup.getJSONArray("counter");
|
||||
for (int j = 0; j < counters.length(); j++) {
|
||||
JSONObject counter = counters.getJSONObject(i);
|
||||
String counterName = counter.getString("name");
|
||||
assertTrue("name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
long value = counter.getLong("value");
|
||||
assertTrue("value >= 0", value >= 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyAMTaskCountersXML(NodeList nodes, Task task) {
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
|
||||
Element element = (Element) nodes.item(i);
|
||||
WebServicesTestUtils.checkStringMatch("id",
|
||||
MRApps.toString(task.getID()),
|
||||
WebServicesTestUtils.getXmlString(element, "id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
NodeList groups = element.getElementsByTagName("taskCounterGroup");
|
||||
|
||||
for (int j = 0; j < groups.getLength(); j++) {
|
||||
Element counters = (Element) groups.item(j);
|
||||
assertNotNull("should have counters in the web service info", counters);
|
||||
String name = WebServicesTestUtils.getXmlString(counters,
|
||||
"counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
NodeList counterArr = counters.getElementsByTagName("counter");
|
||||
for (int z = 0; z < counterArr.getLength(); z++) {
|
||||
Element counter = (Element) counterArr.item(z);
|
||||
String counterName = WebServicesTestUtils.getXmlString(counter,
|
||||
"name");
|
||||
assertTrue("counter name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
|
||||
long value = WebServicesTestUtils.getXmlLong(counter, "value");
|
||||
assertTrue("value not >= 0", value >= 0);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -31,14 +31,13 @@ import javax.ws.rs.core.UriInfo;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
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.TaskType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
||||
import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebServices;
|
||||
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskAttemptCounterInfo;
|
||||
|
@ -131,7 +130,7 @@ public class HsWebServices {
|
|||
try {
|
||||
sBegin = Long.parseLong(startedBegin);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new BadRequestException(e.getMessage());
|
||||
throw new BadRequestException("Invalid number format: " + e.getMessage());
|
||||
}
|
||||
if (sBegin < 0) {
|
||||
throw new BadRequestException("startedTimeBegin must be greater than 0");
|
||||
|
@ -142,7 +141,7 @@ public class HsWebServices {
|
|||
try {
|
||||
sEnd = Long.parseLong(startedEnd);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new BadRequestException(e.getMessage());
|
||||
throw new BadRequestException("Invalid number format: " + e.getMessage());
|
||||
}
|
||||
if (sEnd < 0) {
|
||||
throw new BadRequestException("startedTimeEnd must be greater than 0");
|
||||
|
@ -158,10 +157,10 @@ public class HsWebServices {
|
|||
try {
|
||||
fBegin = Long.parseLong(finishBegin);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new BadRequestException(e.getMessage());
|
||||
throw new BadRequestException("Invalid number format: " + e.getMessage());
|
||||
}
|
||||
if (fBegin < 0) {
|
||||
throw new BadRequestException("finishTimeBegin must be greater than 0");
|
||||
throw new BadRequestException("finishedTimeBegin must be greater than 0");
|
||||
}
|
||||
}
|
||||
if (finishEnd != null && !finishEnd.isEmpty()) {
|
||||
|
@ -169,15 +168,15 @@ public class HsWebServices {
|
|||
try {
|
||||
fEnd = Long.parseLong(finishEnd);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new BadRequestException(e.getMessage());
|
||||
throw new BadRequestException("Invalid number format: " + e.getMessage());
|
||||
}
|
||||
if (fEnd < 0) {
|
||||
throw new BadRequestException("finishTimeEnd must be greater than 0");
|
||||
throw new BadRequestException("finishedTimeEnd must be greater than 0");
|
||||
}
|
||||
}
|
||||
if (fBegin > fEnd) {
|
||||
throw new BadRequestException(
|
||||
"finishTimeEnd must be greater than finishTimeBegin");
|
||||
"finishedTimeEnd must be greater than finishedTimeBegin");
|
||||
}
|
||||
|
||||
for (Job job : appCtx.getAllJobs().values()) {
|
||||
|
@ -200,7 +199,7 @@ public class HsWebServices {
|
|||
}
|
||||
|
||||
if (userQuery != null && !userQuery.isEmpty()) {
|
||||
if (!jobInfo.getName().equals(userQuery)) {
|
||||
if (!jobInfo.getUserName().equals(userQuery)) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
@ -224,14 +223,8 @@ public class HsWebServices {
|
|||
@Path("/mapreduce/jobs/{jobid}")
|
||||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public JobInfo getJob(@PathParam("jobid") String jid) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
return new JobInfo(job);
|
||||
}
|
||||
|
||||
|
@ -239,14 +232,8 @@ public class HsWebServices {
|
|||
@Path("/mapreduce/jobs/{jobid}/attempts")
|
||||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public AMAttemptsInfo getJobAttempts(@PathParam("jobid") String jid) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
AMAttemptsInfo amAttempts = new AMAttemptsInfo();
|
||||
for (AMInfo amInfo : job.getAMInfos()) {
|
||||
AMAttemptInfo attempt = new AMAttemptInfo(amInfo, MRApps.toString(job
|
||||
|
@ -261,53 +248,17 @@ public class HsWebServices {
|
|||
@Path("/mapreduce/jobs/{jobid}/counters")
|
||||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public JobCounterInfo getJobCounters(@PathParam("jobid") String jid) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
return new JobCounterInfo(this.appCtx, job);
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/counters")
|
||||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public JobTaskCounterInfo getSingleTaskCounters(
|
||||
@PathParam("jobid") String jid, @PathParam("taskid") String tid) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = this.appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
Task task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
return new JobTaskCounterInfo(task);
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
return new JobCounterInfo(this.appCtx, job);
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/mapreduce/jobs/{jobid}/conf")
|
||||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public ConfInfo getJobConf(@PathParam("jobid") String jid) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
ConfInfo info;
|
||||
try {
|
||||
info = new ConfInfo(job, this.conf);
|
||||
|
@ -315,7 +266,6 @@ public class HsWebServices {
|
|||
throw new NotFoundException("unable to load configuration for job: "
|
||||
+ jid);
|
||||
}
|
||||
|
||||
return info;
|
||||
}
|
||||
|
||||
|
@ -324,10 +274,8 @@ public class HsWebServices {
|
|||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public TasksInfo getJobTasks(@PathParam("jobid") String jid,
|
||||
@QueryParam("type") String type) {
|
||||
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
TasksInfo allTasks = new TasksInfo();
|
||||
for (Task task : job.getTasks().values()) {
|
||||
TaskType ttype = null;
|
||||
|
@ -351,10 +299,20 @@ public class HsWebServices {
|
|||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public TaskInfo getJobTask(@PathParam("jobid") String jid,
|
||||
@PathParam("taskid") String tid) {
|
||||
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
Task task = AMWebServices.getTaskFromTaskIdString(tid, job);
|
||||
return new TaskInfo(task);
|
||||
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/counters")
|
||||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public JobTaskCounterInfo getSingleTaskCounters(
|
||||
@PathParam("jobid") String jid, @PathParam("taskid") String tid) {
|
||||
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
|
@ -363,8 +321,7 @@ public class HsWebServices {
|
|||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
return new TaskInfo(task);
|
||||
|
||||
return new JobTaskCounterInfo(task);
|
||||
}
|
||||
|
||||
@GET
|
||||
|
@ -372,19 +329,10 @@ public class HsWebServices {
|
|||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public TaskAttemptsInfo getJobTaskAttempts(@PathParam("jobid") String jid,
|
||||
@PathParam("taskid") String tid) {
|
||||
|
||||
TaskAttemptsInfo attempts = new TaskAttemptsInfo();
|
||||
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
Task task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
Task task = AMWebServices.getTaskFromTaskIdString(tid, job);
|
||||
for (TaskAttempt ta : task.getAttempts().values()) {
|
||||
if (ta != null) {
|
||||
if (task.getType() == TaskType.REDUCE) {
|
||||
|
@ -402,28 +350,11 @@ public class HsWebServices {
|
|||
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
|
||||
public TaskAttemptInfo getJobTaskAttemptId(@PathParam("jobid") String jid,
|
||||
@PathParam("taskid") String tid, @PathParam("attemptid") String attId) {
|
||||
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
Task task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
|
||||
if (attemptId == null) {
|
||||
throw new NotFoundException("task attempt id " + attId
|
||||
+ " not found or invalid");
|
||||
}
|
||||
TaskAttempt ta = task.getAttempt(attemptId);
|
||||
if (ta == null) {
|
||||
throw new NotFoundException("Error getting info on task attempt id "
|
||||
+ attId);
|
||||
}
|
||||
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
Task task = AMWebServices.getTaskFromTaskIdString(tid, job);
|
||||
TaskAttempt ta = AMWebServices.getTaskAttemptFromTaskAttemptString(attId,
|
||||
task);
|
||||
if (task.getType() == TaskType.REDUCE) {
|
||||
return new ReduceTaskAttemptInfo(ta, task.getType());
|
||||
} else {
|
||||
|
@ -437,32 +368,11 @@ public class HsWebServices {
|
|||
public JobTaskAttemptCounterInfo getJobTaskAttemptIdCounters(
|
||||
@PathParam("jobid") String jid, @PathParam("taskid") String tid,
|
||||
@PathParam("attemptid") String attId) {
|
||||
JobId jobId = MRApps.toJobID(jid);
|
||||
if (jobId == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
Job job = this.appCtx.getJob(jobId);
|
||||
if (job == null) {
|
||||
throw new NotFoundException("job, " + jid + ", is not found");
|
||||
}
|
||||
TaskId taskID = MRApps.toTaskID(tid);
|
||||
if (taskID == null) {
|
||||
throw new NotFoundException("taskid " + tid + " not found or invalid");
|
||||
}
|
||||
Task task = job.getTask(taskID);
|
||||
if (task == null) {
|
||||
throw new NotFoundException("task not found with id " + tid);
|
||||
}
|
||||
TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
|
||||
if (attemptId == null) {
|
||||
throw new NotFoundException("task attempt id " + attId
|
||||
+ " not found or invalid");
|
||||
}
|
||||
TaskAttempt ta = task.getAttempt(attemptId);
|
||||
if (ta == null) {
|
||||
throw new NotFoundException("Error getting info on task attempt id "
|
||||
+ attId);
|
||||
}
|
||||
|
||||
Job job = AMWebServices.getJobFromJobIdString(jid, appCtx);
|
||||
Task task = AMWebServices.getTaskFromTaskIdString(tid, job);
|
||||
TaskAttempt ta = AMWebServices.getTaskAttemptFromTaskAttemptString(attId,
|
||||
task);
|
||||
return new JobTaskAttemptCounterInfo(ta);
|
||||
}
|
||||
|
||||
|
|
|
@ -42,6 +42,8 @@ import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptsInfo;
|
|||
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterGroupInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
|
||||
|
||||
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptsInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.HistoryInfo;
|
||||
|
@ -57,13 +59,12 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
|
|||
|
||||
// you have to specify all the dao classes here
|
||||
private final Class[] cTypes = { HistoryInfo.class, JobInfo.class,
|
||||
JobsInfo.class, TasksInfo.class, TaskAttemptsInfo.class, ConfInfo.class,
|
||||
CounterInfo.class, JobTaskCounterInfo.class,
|
||||
JobTaskAttemptCounterInfo.class,
|
||||
TaskCounterInfo.class, JobCounterInfo.class, ReduceTaskAttemptInfo.class,
|
||||
TaskAttemptInfo.class, TaskAttemptsInfo.class, CounterGroupInfo.class,
|
||||
TaskCounterGroupInfo.class,
|
||||
AMAttemptInfo.class, AMAttemptsInfo.class};
|
||||
JobsInfo.class, TaskInfo.class, TasksInfo.class, TaskAttemptsInfo.class,
|
||||
ConfInfo.class, CounterInfo.class, JobTaskCounterInfo.class,
|
||||
JobTaskAttemptCounterInfo.class, TaskCounterInfo.class,
|
||||
JobCounterInfo.class, ReduceTaskAttemptInfo.class, TaskAttemptInfo.class,
|
||||
TaskAttemptsInfo.class, CounterGroupInfo.class,
|
||||
TaskCounterGroupInfo.class, AMAttemptInfo.class, AMAttemptsInfo.class };
|
||||
|
||||
public JAXBContextResolver() throws Exception {
|
||||
this.types = new HashSet<Class>(Arrays.asList(cTypes));
|
||||
|
|
|
@ -26,6 +26,7 @@ import javax.xml.bind.annotation.XmlRootElement;
|
|||
import javax.xml.bind.annotation.XmlTransient;
|
||||
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
|
||||
|
@ -48,21 +49,28 @@ public class AMAttemptInfo {
|
|||
|
||||
public AMAttemptInfo(AMInfo amInfo, String jobId, String user, String host,
|
||||
String pathPrefix) {
|
||||
this.nodeHttpAddress = amInfo.getNodeManagerHost() + ":"
|
||||
+ amInfo.getNodeManagerHttpPort();
|
||||
NodeId nodeId = BuilderUtils.newNodeId(amInfo.getNodeManagerHost(),
|
||||
amInfo.getNodeManagerPort());
|
||||
this.nodeHttpAddress = "";
|
||||
this.nodeId = "";
|
||||
String nmHost = amInfo.getNodeManagerHost();
|
||||
int nmPort = amInfo.getNodeManagerHttpPort();
|
||||
if (nmHost != null) {
|
||||
this.nodeHttpAddress = nmHost + ":" + nmPort;
|
||||
NodeId nodeId = BuilderUtils.newNodeId(nmHost, nmPort);
|
||||
this.nodeId = nodeId.toString();
|
||||
}
|
||||
this.id = amInfo.getAppAttemptId().getAttemptId();
|
||||
this.startTime = amInfo.getStartTime();
|
||||
this.containerId = amInfo.getContainerId().toString();
|
||||
this.logsLink = join(
|
||||
host,
|
||||
pathPrefix,
|
||||
ujoin("logs", nodeId.toString(), amInfo.getContainerId().toString(),
|
||||
jobId, user));
|
||||
this.shortLogsLink = ujoin("logs", nodeId.toString(), amInfo
|
||||
.getContainerId().toString(), jobId, user);
|
||||
this.containerId = "";
|
||||
this.logsLink = "";
|
||||
this.shortLogsLink = "";
|
||||
ContainerId containerId = amInfo.getContainerId();
|
||||
if (containerId != null) {
|
||||
this.containerId = containerId.toString();
|
||||
this.logsLink = join(host, pathPrefix,
|
||||
ujoin("logs", this.nodeId, this.containerId, jobId, user));
|
||||
this.shortLogsLink = ujoin("logs", this.nodeId, this.containerId,
|
||||
jobId, user);
|
||||
}
|
||||
}
|
||||
|
||||
public String getNodeHttpAddress() {
|
||||
|
|
|
@ -92,6 +92,7 @@ public class JobInfo {
|
|||
this.user = job.getUserName();
|
||||
this.state = job.getState().toString();
|
||||
this.uberized = job.isUber();
|
||||
this.diagnostics = "";
|
||||
List<String> diagnostics = job.getDiagnostics();
|
||||
if (diagnostics != null && !diagnostics.isEmpty()) {
|
||||
StringBuffer b = new StringBuffer();
|
||||
|
|
|
@ -0,0 +1,360 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.hs.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.HistoryContext;
|
||||
import org.apache.hadoop.mapreduce.v2.hs.JobHistory;
|
||||
import org.apache.hadoop.util.VersionInfo;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebApp;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
import com.sun.jersey.api.client.UniformInterfaceException;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the History Server info web services api's. Also test non-existent urls.
|
||||
*
|
||||
* /ws/v1/history
|
||||
* /ws/v1/history/info
|
||||
*/
|
||||
public class TestHsWebServices extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
private static HsWebApp webApp;
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
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);
|
||||
}
|
||||
|
||||
TestAppContext() {
|
||||
this(0, 1, 1, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
appContext = new TestAppContext();
|
||||
JobHistory jobHistoryService = new JobHistory();
|
||||
HistoryContext historyContext = (HistoryContext) jobHistoryService;
|
||||
webApp = new HsWebApp(historyContext);
|
||||
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(HsWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(WebApp.class).toInstance(webApp);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
}
|
||||
|
||||
public TestHsWebServices() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.hs.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHS() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHSSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHSDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history/")
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHSXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
verifyHSInfoXML(xml, appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInfo() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("info").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInfoSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("info/").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInfoDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("info/").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
verifyHSInfo(json.getJSONObject("historyInfo"), appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInfoXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("info/").accept(MediaType.APPLICATION_XML)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
verifyHSInfoXML(xml, appContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidUri() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
String responseStr = "";
|
||||
try {
|
||||
responseStr = r.path("ws").path("v1").path("history").path("bogus")
|
||||
.accept(MediaType.APPLICATION_JSON).get(String.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
WebServicesTestUtils.checkStringMatch(
|
||||
"error string exists and shouldn't", "", responseStr);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidUri2() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
String responseStr = "";
|
||||
try {
|
||||
responseStr = r.path("ws").path("v1").path("invalid")
|
||||
.accept(MediaType.APPLICATION_JSON).get(String.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
WebServicesTestUtils.checkStringMatch(
|
||||
"error string exists and shouldn't", "", responseStr);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidAccept() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
String responseStr = "";
|
||||
try {
|
||||
responseStr = r.path("ws").path("v1").path("history")
|
||||
.accept(MediaType.TEXT_PLAIN).get(String.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.INTERNAL_SERVER_ERROR,
|
||||
response.getClientResponseStatus());
|
||||
WebServicesTestUtils.checkStringMatch(
|
||||
"error string exists and shouldn't", "", responseStr);
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsInfoGeneric(String hadoopVersionBuiltOn,
|
||||
String hadoopBuildVersion, String hadoopVersion) {
|
||||
WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn",
|
||||
VersionInfo.getDate(), hadoopVersionBuiltOn);
|
||||
WebServicesTestUtils.checkStringMatch("hadoopBuildVersion",
|
||||
VersionInfo.getBuildVersion(), hadoopBuildVersion);
|
||||
WebServicesTestUtils.checkStringMatch("hadoopVersion",
|
||||
VersionInfo.getVersion(), hadoopVersion);
|
||||
}
|
||||
|
||||
public void verifyHSInfo(JSONObject info, TestAppContext ctx)
|
||||
throws JSONException {
|
||||
assertEquals("incorrect number of elements", 3, info.length());
|
||||
|
||||
verifyHsInfoGeneric(info.getString("hadoopVersionBuiltOn"),
|
||||
info.getString("hadoopBuildVersion"), info.getString("hadoopVersion"));
|
||||
}
|
||||
|
||||
public void verifyHSInfoXML(String xml, TestAppContext ctx)
|
||||
throws JSONException, Exception {
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList nodes = dom.getElementsByTagName("historyInfo");
|
||||
assertEquals("incorrect number of elements", 1, nodes.getLength());
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
verifyHsInfoGeneric(
|
||||
WebServicesTestUtils.getXmlString(element, "hadoopVersionBuiltOn"),
|
||||
WebServicesTestUtils.getXmlString(element, "hadoopBuildVersion"),
|
||||
WebServicesTestUtils.getXmlString(element, "hadoopVersion"));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,745 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.hs.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
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.app.job.Task;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebApp;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONArray;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
import com.sun.jersey.api.client.UniformInterfaceException;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the history server Rest API for getting task attempts, a
|
||||
* specific task attempt, and task attempt counters
|
||||
*
|
||||
* /ws/v1/history/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts
|
||||
* /ws/v1/history/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}
|
||||
* /ws/v1/history/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/
|
||||
* counters
|
||||
*/
|
||||
public class TestHsWebServicesAttempts extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
private static HsWebApp webApp;
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
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);
|
||||
}
|
||||
|
||||
TestAppContext() {
|
||||
this(0, 1, 2, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
appContext = new TestAppContext();
|
||||
webApp = mock(HsWebApp.class);
|
||||
when(webApp.name()).thenReturn("hsmockwebapp");
|
||||
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(HsWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(WebApp.class).toInstance(webApp);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
}
|
||||
|
||||
public TestHsWebServicesAttempts() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.hs.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttempts() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
verifyHsTaskAttempts(json, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptsSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts/").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
verifyHsTaskAttempts(json, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptsDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
verifyHsTaskAttempts(json, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptsXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").accept(MediaType.APPLICATION_XML)
|
||||
.get(ClientResponse.class);
|
||||
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList attempts = dom.getElementsByTagName("taskAttempts");
|
||||
assertEquals("incorrect number of elements", 1, attempts.getLength());
|
||||
|
||||
NodeList nodes = dom.getElementsByTagName("taskAttempt");
|
||||
verifyHsTaskAttemptsXML(nodes, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptId() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.path(tid).path("attempts").path(attid)
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("taskAttempt");
|
||||
verifyHsTaskAttempt(info, att, task.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.path(tid).path("attempts").path(attid + "/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("taskAttempt");
|
||||
verifyHsTaskAttempt(info, att, task.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.path(tid).path("attempts").path(attid).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("taskAttempt");
|
||||
verifyHsTaskAttempt(info, att, task.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.path(tid).path("attempts").path(attid)
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList nodes = dom.getElementsByTagName("taskAttempt");
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
verifyHsTaskAttemptXML(element, att, task.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdBogus() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric("bogusid",
|
||||
"java.lang.Exception: Error parsing attempt ID: bogusid");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdNonExist() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric(
|
||||
"attempt_12345_0_0_r_1_0",
|
||||
"java.lang.Exception: Error getting info on task attempt id attempt_12345_0_0_r_1_0");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdInvalid() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric("attempt_12345_0_0_d_1_0",
|
||||
"java.lang.Exception: Unknown task symbol: d");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdInvalid2() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric("attempt_12345_0_r_1_0",
|
||||
"java.lang.Exception: For input string: \"r\"");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdInvalid3() throws JSONException, Exception {
|
||||
|
||||
testTaskAttemptIdErrorGeneric("attempt_12345_0_0_r_1",
|
||||
"java.lang.Exception: Error parsing attempt ID: attempt_12345_0_0_r_1");
|
||||
}
|
||||
|
||||
private void testTaskAttemptIdErrorGeneric(String attid, String error)
|
||||
throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("attempts").path(attid).accept(MediaType.APPLICATION_JSON)
|
||||
.get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message", error,
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsTaskAttemptXML(Element element, TaskAttempt att,
|
||||
TaskType ttype) {
|
||||
verifyTaskAttemptGeneric(att, ttype,
|
||||
WebServicesTestUtils.getXmlString(element, "id"),
|
||||
WebServicesTestUtils.getXmlString(element, "state"),
|
||||
WebServicesTestUtils.getXmlString(element, "type"),
|
||||
WebServicesTestUtils.getXmlString(element, "rack"),
|
||||
WebServicesTestUtils.getXmlString(element, "nodeHttpAddress"),
|
||||
WebServicesTestUtils.getXmlString(element, "diagnostics"),
|
||||
WebServicesTestUtils.getXmlString(element, "assignedContainerId"),
|
||||
WebServicesTestUtils.getXmlLong(element, "startTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "finishTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedTime"),
|
||||
WebServicesTestUtils.getXmlFloat(element, "progress"));
|
||||
|
||||
if (ttype == TaskType.REDUCE) {
|
||||
verifyReduceTaskAttemptGeneric(att,
|
||||
WebServicesTestUtils.getXmlLong(element, "shuffleFinishTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "mergeFinishTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedShuffleTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedMergeTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedReduceTime"));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsTaskAttempt(JSONObject info, TaskAttempt att,
|
||||
TaskType ttype) throws JSONException {
|
||||
if (ttype == TaskType.REDUCE) {
|
||||
assertEquals("incorrect number of elements", 16, info.length());
|
||||
} else {
|
||||
assertEquals("incorrect number of elements", 11, info.length());
|
||||
}
|
||||
|
||||
verifyTaskAttemptGeneric(att, ttype, info.getString("id"),
|
||||
info.getString("state"), info.getString("type"),
|
||||
info.getString("rack"), info.getString("nodeHttpAddress"),
|
||||
info.getString("diagnostics"), info.getString("assignedContainerId"),
|
||||
info.getLong("startTime"), info.getLong("finishTime"),
|
||||
info.getLong("elapsedTime"), (float) info.getDouble("progress"));
|
||||
|
||||
if (ttype == TaskType.REDUCE) {
|
||||
verifyReduceTaskAttemptGeneric(att, info.getLong("shuffleFinishTime"),
|
||||
info.getLong("mergeFinishTime"), info.getLong("elapsedShuffleTime"),
|
||||
info.getLong("elapsedMergeTime"), info.getLong("elapsedReduceTime"));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsTaskAttempts(JSONObject json, Task task)
|
||||
throws JSONException {
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject attempts = json.getJSONObject("taskAttempts");
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONArray arr = attempts.getJSONArray("taskAttempt");
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId id = att.getID();
|
||||
String attid = MRApps.toString(id);
|
||||
Boolean found = false;
|
||||
|
||||
for (int i = 0; i < arr.length(); i++) {
|
||||
JSONObject info = arr.getJSONObject(i);
|
||||
if (attid.matches(info.getString("id"))) {
|
||||
found = true;
|
||||
verifyHsTaskAttempt(info, att, task.getType());
|
||||
}
|
||||
}
|
||||
assertTrue("task attempt with id: " + attid
|
||||
+ " not in web service output", found);
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsTaskAttemptsXML(NodeList nodes, Task task) {
|
||||
assertEquals("incorrect number of elements", 1, nodes.getLength());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId id = att.getID();
|
||||
String attid = MRApps.toString(id);
|
||||
Boolean found = false;
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
|
||||
if (attid.matches(WebServicesTestUtils.getXmlString(element, "id"))) {
|
||||
found = true;
|
||||
verifyHsTaskAttemptXML(element, att, task.getType());
|
||||
}
|
||||
}
|
||||
assertTrue("task with id: " + attid + " not in web service output", found);
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyTaskAttemptGeneric(TaskAttempt ta, TaskType ttype,
|
||||
String id, String state, String type, String rack,
|
||||
String nodeHttpAddress, String diagnostics, String assignedContainerId,
|
||||
long startTime, long finishTime, long elapsedTime, float progress) {
|
||||
|
||||
TaskAttemptId attid = ta.getID();
|
||||
String attemptId = MRApps.toString(attid);
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", attemptId, id);
|
||||
WebServicesTestUtils.checkStringMatch("type", ttype.toString(), type);
|
||||
WebServicesTestUtils.checkStringMatch("state", ta.getState().toString(),
|
||||
state);
|
||||
WebServicesTestUtils.checkStringMatch("rack", ta.getNodeRackName(), rack);
|
||||
WebServicesTestUtils.checkStringMatch("nodeHttpAddress",
|
||||
ta.getNodeHttpAddress(), nodeHttpAddress);
|
||||
|
||||
String expectDiag = "";
|
||||
List<String> diagnosticsList = ta.getDiagnostics();
|
||||
if (diagnosticsList != null && !diagnostics.isEmpty()) {
|
||||
StringBuffer b = new StringBuffer();
|
||||
for (String diag : diagnosticsList) {
|
||||
b.append(diag);
|
||||
}
|
||||
expectDiag = b.toString();
|
||||
}
|
||||
WebServicesTestUtils.checkStringMatch("diagnostics", expectDiag,
|
||||
diagnostics);
|
||||
WebServicesTestUtils.checkStringMatch("assignedContainerId",
|
||||
ConverterUtils.toString(ta.getAssignedContainerID()),
|
||||
assignedContainerId);
|
||||
|
||||
assertEquals("startTime wrong", ta.getLaunchTime(), startTime);
|
||||
assertEquals("finishTime wrong", ta.getFinishTime(), finishTime);
|
||||
assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime);
|
||||
assertEquals("progress wrong", ta.getProgress() * 100, progress, 1e-3f);
|
||||
}
|
||||
|
||||
public void verifyReduceTaskAttemptGeneric(TaskAttempt ta,
|
||||
long shuffleFinishTime, long mergeFinishTime, long elapsedShuffleTime,
|
||||
long elapsedMergeTime, long elapsedReduceTime) {
|
||||
|
||||
assertEquals("shuffleFinishTime wrong", ta.getShuffleFinishTime(),
|
||||
shuffleFinishTime);
|
||||
assertEquals("mergeFinishTime wrong", ta.getSortFinishTime(),
|
||||
mergeFinishTime);
|
||||
assertEquals("elapsedShuffleTime wrong",
|
||||
ta.getLaunchTime() - ta.getShuffleFinishTime(), elapsedShuffleTime);
|
||||
assertEquals("elapsedMergeTime wrong",
|
||||
ta.getShuffleFinishTime() - ta.getSortFinishTime(), elapsedMergeTime);
|
||||
assertEquals("elapsedReduceTime wrong",
|
||||
ta.getSortFinishTime() - ta.getFinishTime(), elapsedReduceTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdCounters() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
String tid = MRApps.toString(task.getID());
|
||||
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.path(tid).path("attempts").path(attid).path("counters")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("JobTaskAttemptCounters");
|
||||
verifyHsJobTaskAttemptCounters(info, att);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskAttemptIdXMLCounters() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
for (TaskAttempt att : task.getAttempts().values()) {
|
||||
TaskAttemptId attemptid = att.getID();
|
||||
String attid = MRApps.toString(attemptid);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.path(tid).path("attempts").path(attid).path("counters")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList nodes = dom.getElementsByTagName("JobTaskAttemptCounters");
|
||||
|
||||
verifyHsTaskCountersXML(nodes, att);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsJobTaskAttemptCounters(JSONObject info, TaskAttempt att)
|
||||
throws JSONException {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, info.length());
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(att.getID()),
|
||||
info.getString("id"));
|
||||
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
JSONArray counterGroups = info.getJSONArray("taskAttemptCounterGroup");
|
||||
for (int i = 0; i < counterGroups.length(); i++) {
|
||||
JSONObject counterGroup = counterGroups.getJSONObject(i);
|
||||
String name = counterGroup.getString("counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
JSONArray counters = counterGroup.getJSONArray("counter");
|
||||
for (int j = 0; j < counters.length(); j++) {
|
||||
JSONObject counter = counters.getJSONObject(i);
|
||||
String counterName = counter.getString("name");
|
||||
assertTrue("name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
long value = counter.getLong("value");
|
||||
assertTrue("value >= 0", value >= 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsTaskCountersXML(NodeList nodes, TaskAttempt att) {
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
|
||||
Element element = (Element) nodes.item(i);
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(att.getID()),
|
||||
WebServicesTestUtils.getXmlString(element, "id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
NodeList groups = element.getElementsByTagName("taskAttemptCounterGroup");
|
||||
|
||||
for (int j = 0; j < groups.getLength(); j++) {
|
||||
Element counters = (Element) groups.item(j);
|
||||
assertNotNull("should have counters in the web service info", counters);
|
||||
String name = WebServicesTestUtils.getXmlString(counters,
|
||||
"counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
NodeList counterArr = counters.getElementsByTagName("counter");
|
||||
for (int z = 0; z < counterArr.getLength(); z++) {
|
||||
Element counter = (Element) counterArr.item(z);
|
||||
String counterName = WebServicesTestUtils.getXmlString(counter,
|
||||
"name");
|
||||
assertTrue("counter name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
|
||||
long value = WebServicesTestUtils.getXmlLong(counter, "value");
|
||||
assertTrue("value not >= 0", value >= 0);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,345 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.hs.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.io.StringReader;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
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.util.MRApps;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebApp;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONArray;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the history server Rest API for getting the job conf. This
|
||||
* requires created a temporary configuration file.
|
||||
*
|
||||
* /ws/v1/history/mapreduce/jobs/{jobid}/conf
|
||||
*/
|
||||
public class TestHsWebServicesJobConf extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
private static HsWebApp webApp;
|
||||
|
||||
private static File testConfDir = new File("target",
|
||||
TestHsWebServicesJobConf.class.getSimpleName() + "confDir");
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
final long startTime = System.currentTimeMillis();
|
||||
|
||||
TestAppContext(int appid, int numTasks, int numAttempts, Path confPath) {
|
||||
appID = MockJobs.newAppID(appid);
|
||||
appAttemptID = MockJobs.newAppAttemptID(appID, 0);
|
||||
Map<JobId, Job> map = Maps.newHashMap();
|
||||
Job job = MockJobs.newJob(appID, 0, numTasks, numAttempts, confPath);
|
||||
map.put(job.getID(), job);
|
||||
jobs = map;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
Path confPath = new Path(testConfDir.toString(),
|
||||
MRJobConfig.JOB_CONF_FILE);
|
||||
Configuration config = new Configuration();
|
||||
|
||||
FileSystem localFs;
|
||||
try {
|
||||
localFs = FileSystem.getLocal(config);
|
||||
confPath = localFs.makeQualified(confPath);
|
||||
|
||||
OutputStream out = localFs.create(confPath);
|
||||
try {
|
||||
conf.writeXml(out);
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
if (!localFs.exists(confPath)) {
|
||||
fail("error creating config file: " + confPath);
|
||||
}
|
||||
|
||||
} catch (IOException e) {
|
||||
fail("error creating config file: " + e.getMessage());
|
||||
}
|
||||
|
||||
appContext = new TestAppContext(0, 2, 1, confPath);
|
||||
|
||||
webApp = mock(HsWebApp.class);
|
||||
when(webApp.name()).thenReturn("hsmockwebapp");
|
||||
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(HsWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(WebApp.class).toInstance(webApp);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
testConfDir.mkdir();
|
||||
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
static public void stop() {
|
||||
FileUtil.fullyDelete(testConfDir);
|
||||
}
|
||||
|
||||
public TestHsWebServicesJobConf() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.hs.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobConf() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce")
|
||||
.path("jobs").path(jobId).path("conf")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("conf");
|
||||
verifyHsJobConf(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobConfSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("conf/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("conf");
|
||||
verifyHsJobConf(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobConfDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("conf").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("conf");
|
||||
verifyHsJobConf(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobConfXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history").path("mapreduce")
|
||||
.path("jobs").path(jobId).path("conf")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList info = dom.getElementsByTagName("conf");
|
||||
verifyHsJobConfXML(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsJobConf(JSONObject info, Job job) throws JSONException {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, info.length());
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("path", job.getConfFile().toString(),
|
||||
info.getString("path"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
JSONArray properties = info.getJSONArray("property");
|
||||
for (int i = 0; i < properties.length(); i++) {
|
||||
JSONObject prop = properties.getJSONObject(i);
|
||||
String name = prop.getString("name");
|
||||
String value = prop.getString("value");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
assertTrue("value not set", (value != null && !value.isEmpty()));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsJobConfXML(NodeList nodes, Job job) {
|
||||
|
||||
assertEquals("incorrect number of elements", 1, nodes.getLength());
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
WebServicesTestUtils.checkStringMatch("path", job.getConfFile()
|
||||
.toString(), WebServicesTestUtils.getXmlString(element, "path"));
|
||||
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
NodeList properties = element.getElementsByTagName("property");
|
||||
|
||||
for (int j = 0; j < properties.getLength(); j++) {
|
||||
Element property = (Element) properties.item(j);
|
||||
assertNotNull("should have counters in the web service info", property);
|
||||
String name = WebServicesTestUtils.getXmlString(property, "name");
|
||||
String value = WebServicesTestUtils.getXmlString(property, "value");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
assertTrue("name not set", (value != null && !value.isEmpty()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,755 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.hs.webapp;
|
||||
|
||||
import static org.apache.hadoop.yarn.util.StringHelper.join;
|
||||
import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
||||
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.util.MRApps;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebApp;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONArray;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
import com.sun.jersey.api.client.UniformInterfaceException;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the history server Rest API for getting jobs, a specific job, job
|
||||
* counters, and job attempts.
|
||||
*
|
||||
* /ws/v1/history/mapreduce/jobs /ws/v1/history/mapreduce/jobs/{jobid}
|
||||
* /ws/v1/history/mapreduce/jobs/{jobid}/counters
|
||||
* /ws/v1/history/mapreduce/jobs/{jobid}/attempts
|
||||
*/
|
||||
public class TestHsWebServicesJobs extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
private static HsWebApp webApp;
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
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);
|
||||
}
|
||||
|
||||
TestAppContext() {
|
||||
this(0, 1, 2, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
appContext = new TestAppContext();
|
||||
webApp = mock(HsWebApp.class);
|
||||
when(webApp.name()).thenReturn("hsmockwebapp");
|
||||
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(HsWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(WebApp.class).toInstance(webApp);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
}
|
||||
|
||||
public TestHsWebServicesJobs() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.hs.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobs() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
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);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs/").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
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);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
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);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsXML() throws Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").accept(MediaType.APPLICATION_XML)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList jobs = dom.getElementsByTagName("jobs");
|
||||
assertEquals("incorrect number of elements", 1, jobs.getLength());
|
||||
NodeList job = dom.getElementsByTagName("job");
|
||||
assertEquals("incorrect number of elements", 1, job.getLength());
|
||||
verifyHsJobXML(job, appContext);
|
||||
|
||||
}
|
||||
|
||||
public void verifyHsJobXML(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.getJob(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"));
|
||||
|
||||
// restricted access fields - if security and acls set
|
||||
VerifyJobsUtils.verifyHsJobGenericSecure(job,
|
||||
WebServicesTestUtils.getXmlBoolean(element, "uberized"),
|
||||
WebServicesTestUtils.getXmlString(element, "diagnostics"),
|
||||
WebServicesTestUtils.getXmlLong(element, "avgMapTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "avgReduceTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "avgShuffleTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "avgMergeTime"),
|
||||
WebServicesTestUtils.getXmlInt(element, "failedReduceAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "killedReduceAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "successfulReduceAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "failedMapAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "killedMapAttempts"),
|
||||
WebServicesTestUtils.getXmlInt(element, "successfulMapAttempts"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobId() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId)
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
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));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId + "/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
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));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
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));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdNonExist() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce").path("jobs")
|
||||
.path("job_1234_1_2").get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: job, job_1234_1_2, is not found", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdInvalid() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce").path("jobs")
|
||||
.path("job_foo").get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"For input string: \"foo\"", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NumberFormatException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"java.lang.NumberFormatException", classname);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdInvalidBogus() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce").path("jobs")
|
||||
.path("bogusfoo").get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: Error parsing job ID: bogusfoo", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobIdXML() throws Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId)
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList job = dom.getElementsByTagName("job");
|
||||
verifyHsJobXML(job, appContext);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobCounters() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("counters")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobCounters");
|
||||
verifyHsJobCounters(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobCountersSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("counters/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobCounters");
|
||||
verifyHsJobCounters(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobCountersDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("counters/")
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobCounters");
|
||||
verifyHsJobCounters(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobCountersXML() throws Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("counters")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList info = dom.getElementsByTagName("jobCounters");
|
||||
verifyHsJobCountersXML(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsJobCounters(JSONObject info, Job job)
|
||||
throws JSONException {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, info.length());
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
|
||||
info.getString("id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
JSONArray counterGroups = info.getJSONArray("counterGroup");
|
||||
for (int i = 0; i < counterGroups.length(); i++) {
|
||||
JSONObject counterGroup = counterGroups.getJSONObject(i);
|
||||
String name = counterGroup.getString("counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
JSONArray counters = counterGroup.getJSONArray("counter");
|
||||
for (int j = 0; j < counters.length(); j++) {
|
||||
JSONObject counter = counters.getJSONObject(i);
|
||||
String counterName = counter.getString("name");
|
||||
assertTrue("counter name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
|
||||
long mapValue = counter.getLong("mapCounterValue");
|
||||
assertTrue("mapCounterValue >= 0", mapValue >= 0);
|
||||
|
||||
long reduceValue = counter.getLong("reduceCounterValue");
|
||||
assertTrue("reduceCounterValue >= 0", reduceValue >= 0);
|
||||
|
||||
long totalValue = counter.getLong("totalCounterValue");
|
||||
assertTrue("totalCounterValue >= 0", totalValue >= 0);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsJobCountersXML(NodeList nodes, Job job) {
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
|
||||
assertNotNull("Job not found - output incorrect", job);
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
|
||||
WebServicesTestUtils.getXmlString(element, "id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
NodeList groups = element.getElementsByTagName("counterGroup");
|
||||
|
||||
for (int j = 0; j < groups.getLength(); j++) {
|
||||
Element counters = (Element) groups.item(j);
|
||||
assertNotNull("should have counters in the web service info", counters);
|
||||
String name = WebServicesTestUtils.getXmlString(counters,
|
||||
"counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
NodeList counterArr = counters.getElementsByTagName("counter");
|
||||
for (int z = 0; z < counterArr.getLength(); z++) {
|
||||
Element counter = (Element) counterArr.item(z);
|
||||
String counterName = WebServicesTestUtils.getXmlString(counter,
|
||||
"name");
|
||||
assertTrue("counter name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
|
||||
long mapValue = WebServicesTestUtils.getXmlLong(counter,
|
||||
"mapCounterValue");
|
||||
assertTrue("mapCounterValue not >= 0", mapValue >= 0);
|
||||
|
||||
long reduceValue = WebServicesTestUtils.getXmlLong(counter,
|
||||
"reduceCounterValue");
|
||||
assertTrue("reduceCounterValue >= 0", reduceValue >= 0);
|
||||
|
||||
long totalValue = WebServicesTestUtils.getXmlLong(counter,
|
||||
"totalCounterValue");
|
||||
assertTrue("totalCounterValue >= 0", totalValue >= 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobAttempts() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("attempts")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("attempts");
|
||||
verifyHsJobAttempts(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobAttemptsSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("attempts/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("attempts");
|
||||
verifyHsJobAttempts(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobAttemptsDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("attempts")
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("attempts");
|
||||
verifyHsJobAttempts(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobAttemptsXML() throws Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("attempts")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList attempts = dom.getElementsByTagName("attempts");
|
||||
assertEquals("incorrect number of elements", 1, attempts.getLength());
|
||||
NodeList info = dom.getElementsByTagName("attempt");
|
||||
verifyHsJobAttemptsXML(info, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsJobAttempts(JSONObject info, Job job)
|
||||
throws JSONException {
|
||||
|
||||
JSONArray attempts = info.getJSONArray("attempt");
|
||||
assertEquals("incorrect number of elements", 2, attempts.length());
|
||||
for (int i = 0; i < attempts.length(); i++) {
|
||||
JSONObject attempt = attempts.getJSONObject(i);
|
||||
verifyHsJobAttemptsGeneric(job, attempt.getString("nodeHttpAddress"),
|
||||
attempt.getString("nodeId"), attempt.getInt("id"),
|
||||
attempt.getLong("startTime"), attempt.getString("containerId"),
|
||||
attempt.getString("logsLink"));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsJobAttemptsXML(NodeList nodes, Job job) {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, nodes.getLength());
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
verifyHsJobAttemptsGeneric(job,
|
||||
WebServicesTestUtils.getXmlString(element, "nodeHttpAddress"),
|
||||
WebServicesTestUtils.getXmlString(element, "nodeId"),
|
||||
WebServicesTestUtils.getXmlInt(element, "id"),
|
||||
WebServicesTestUtils.getXmlLong(element, "startTime"),
|
||||
WebServicesTestUtils.getXmlString(element, "containerId"),
|
||||
WebServicesTestUtils.getXmlString(element, "logsLink"));
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsJobAttemptsGeneric(Job job, String nodeHttpAddress,
|
||||
String nodeId, int id, long startTime, String containerId, String logsLink) {
|
||||
boolean attemptFound = false;
|
||||
for (AMInfo amInfo : job.getAMInfos()) {
|
||||
if (amInfo.getAppAttemptId().getAttemptId() == id) {
|
||||
attemptFound = true;
|
||||
String nmHost = amInfo.getNodeManagerHost();
|
||||
int nmPort = amInfo.getNodeManagerHttpPort();
|
||||
WebServicesTestUtils.checkStringMatch("nodeHttpAddress", nmHost + ":"
|
||||
+ nmPort, nodeHttpAddress);
|
||||
WebServicesTestUtils.checkStringMatch("nodeId",
|
||||
BuilderUtils.newNodeId(nmHost, nmPort).toString(), nodeId);
|
||||
assertTrue("startime not greater than 0", startTime > 0);
|
||||
WebServicesTestUtils.checkStringMatch("containerId", amInfo
|
||||
.getContainerId().toString(), containerId);
|
||||
|
||||
String localLogsLink = join(
|
||||
"hsmockwebapp",
|
||||
ujoin("logs", nodeId, containerId, MRApps.toString(job.getID()),
|
||||
job.getUserName()));
|
||||
|
||||
assertTrue("logsLink", logsLink.contains(localLogsLink));
|
||||
}
|
||||
}
|
||||
assertTrue("attempt: " + id + " was not found", attemptFound);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,656 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.hs.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.util.MRApps;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebApp;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONArray;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the history server Rest API for getting jobs with various query
|
||||
* parameters.
|
||||
*
|
||||
* /ws/v1/history/mapreduce/jobs?{query=value}
|
||||
*/
|
||||
public class TestHsWebServicesJobsQuery extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
private static HsWebApp webApp;
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
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);
|
||||
}
|
||||
|
||||
TestAppContext() {
|
||||
this(0, 3, 2, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
appContext = new TestAppContext();
|
||||
webApp = mock(HsWebApp.class);
|
||||
when(webApp.name()).thenReturn("hsmockwebapp");
|
||||
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(HsWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(WebApp.class).toInstance(webApp);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
}
|
||||
|
||||
public TestHsWebServicesJobsQuery() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.hs.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryUserNone() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("user", "bogus")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryUser() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("user", "mock")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
assertEquals("incorrect number of elements", 3, arr.length());
|
||||
// just verify one of them.
|
||||
JSONObject info = arr.getJSONObject(0);
|
||||
Job job = appContext.getJob(MRApps.toJobID(info.getString("id")));
|
||||
VerifyJobsUtils.verifyHsJob(info, job);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryLimit() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("limit", "2")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
// make sure we get 2 back
|
||||
assertEquals("incorrect number of elements", 2, arr.length());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryLimitInvalid() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("limit", "-1")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: limit value must be greater then 0", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryQueue() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("queue", "mockqueue")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
assertEquals("incorrect number of elements", 3, arr.length());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryQueueNonExist() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("queue", "bogus")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryStartTimeEnd() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
// the mockJobs start time is the current time - some random amount
|
||||
Long now = System.currentTimeMillis();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("startedTimeEnd", String.valueOf(now))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
assertEquals("incorrect number of elements", 3, arr.length());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryStartTimeBegin() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
// the mockJobs start time is the current time - some random amount
|
||||
Long now = System.currentTimeMillis();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("startedTimeBegin", String.valueOf(now))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryStartTimeBeginEnd() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
int size = jobsMap.size();
|
||||
ArrayList<Long> startTime = new ArrayList<Long>(size);
|
||||
// figure out the middle start Time
|
||||
for (Map.Entry<JobId, Job> entry : jobsMap.entrySet()) {
|
||||
startTime.add(entry.getValue().getReport().getStartTime());
|
||||
}
|
||||
Collections.sort(startTime);
|
||||
|
||||
assertTrue("Error we must have atleast 3 jobs", size >= 3);
|
||||
long midStartTime = startTime.get(size - 2);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("startedTimeBegin", String.valueOf(40000))
|
||||
.queryParam("startedTimeEnd", String.valueOf(midStartTime))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
assertEquals("incorrect number of elements", size - 1, arr.length());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryStartTimeBeginEndInvalid() throws JSONException,
|
||||
Exception {
|
||||
WebResource r = resource();
|
||||
Long now = System.currentTimeMillis();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("startedTimeBegin", String.valueOf(now))
|
||||
.queryParam("startedTimeEnd", String.valueOf(40000))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils
|
||||
.checkStringMatch(
|
||||
"exception message",
|
||||
"java.lang.Exception: startedTimeEnd must be greater than startTimeBegin",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryStartTimeInvalidformat() throws JSONException,
|
||||
Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("startedTimeBegin", "efsd")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils
|
||||
.checkStringMatch(
|
||||
"exception message",
|
||||
"java.lang.Exception: Invalid number format: For input string: \"efsd\"",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryStartTimeEndInvalidformat() throws JSONException,
|
||||
Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("startedTimeEnd", "efsd")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils
|
||||
.checkStringMatch(
|
||||
"exception message",
|
||||
"java.lang.Exception: Invalid number format: For input string: \"efsd\"",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryStartTimeNegative() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("startedTimeBegin", String.valueOf(-1000))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils
|
||||
.checkStringMatch("exception message",
|
||||
"java.lang.Exception: startedTimeBegin must be greater than 0",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryStartTimeEndNegative() throws JSONException,
|
||||
Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("startedTimeEnd", String.valueOf(-1000))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: startedTimeEnd must be greater than 0", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryFinishTimeEndNegative() throws JSONException,
|
||||
Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("finishedTimeEnd", String.valueOf(-1000))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: finishedTimeEnd must be greater than 0", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryFinishTimeBeginNegative() throws JSONException,
|
||||
Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("finishedTimeBegin", String.valueOf(-1000))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: finishedTimeBegin must be greater than 0",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryFinishTimeBeginEndInvalid() throws JSONException,
|
||||
Exception {
|
||||
WebResource r = resource();
|
||||
Long now = System.currentTimeMillis();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("finishedTimeBegin", String.valueOf(now))
|
||||
.queryParam("finishedTimeEnd", String.valueOf(40000))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils
|
||||
.checkStringMatch(
|
||||
"exception message",
|
||||
"java.lang.Exception: finishedTimeEnd must be greater than finishedTimeBegin",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryFinishTimeInvalidformat() throws JSONException,
|
||||
Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("finishedTimeBegin", "efsd")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils
|
||||
.checkStringMatch(
|
||||
"exception message",
|
||||
"java.lang.Exception: Invalid number format: For input string: \"efsd\"",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryFinishTimeEndInvalidformat() throws JSONException,
|
||||
Exception {
|
||||
WebResource r = resource();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").queryParam("finishedTimeEnd", "efsd")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils
|
||||
.checkStringMatch(
|
||||
"exception message",
|
||||
"java.lang.Exception: Invalid number format: For input string: \"efsd\"",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryFinishTimeBegin() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
// the mockJobs finish time is the current time + some random amount
|
||||
Long now = System.currentTimeMillis();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("finishedTimeBegin", String.valueOf(now))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
assertEquals("incorrect number of elements", 3, arr.length());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryFinishTimeEnd() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
// the mockJobs finish time is the current time + some random amount
|
||||
Long now = System.currentTimeMillis();
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("finishedTimeEnd", String.valueOf(now))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobsQueryFinishTimeBeginEnd() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
int size = jobsMap.size();
|
||||
// figure out the mid end time - we expect atleast 3 jobs
|
||||
ArrayList<Long> finishTime = new ArrayList<Long>(size);
|
||||
for (Map.Entry<JobId, Job> entry : jobsMap.entrySet()) {
|
||||
finishTime.add(entry.getValue().getReport().getFinishTime());
|
||||
}
|
||||
Collections.sort(finishTime);
|
||||
|
||||
assertTrue("Error we must have atleast 3 jobs", size >= 3);
|
||||
long midFinishTime = finishTime.get(size - 2);
|
||||
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs")
|
||||
.queryParam("finishedTimeBegin", String.valueOf(40000))
|
||||
.queryParam("finishedTimeEnd", String.valueOf(midFinishTime))
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject jobs = json.getJSONObject("jobs");
|
||||
JSONArray arr = jobs.getJSONArray("job");
|
||||
assertEquals("incorrect number of elements", size - 1, arr.length());
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,835 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.hs.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
|
||||
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.app.job.Task;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
||||
import org.apache.hadoop.yarn.webapp.WebApp;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONArray;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
import com.google.inject.servlet.ServletModule;
|
||||
import com.sun.jersey.api.client.ClientResponse;
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
import com.sun.jersey.api.client.UniformInterfaceException;
|
||||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import com.sun.jersey.test.framework.JerseyTest;
|
||||
import com.sun.jersey.test.framework.WebAppDescriptor;
|
||||
|
||||
/**
|
||||
* Test the history server Rest API for getting tasks, a specific task,
|
||||
* and task counters.
|
||||
*
|
||||
* /ws/v1/history/mapreduce/jobs/{jobid}/tasks
|
||||
* /ws/v1/history/mapreduce/jobs/{jobid}/tasks/{taskid}
|
||||
* /ws/v1/history/mapreduce/jobs/{jobid}/tasks/{taskid}/counters
|
||||
*/
|
||||
public class TestHsWebServicesTasks extends JerseyTest {
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
private static TestAppContext appContext;
|
||||
private static HsWebApp webApp;
|
||||
|
||||
static class TestAppContext implements AppContext {
|
||||
final ApplicationAttemptId appAttemptID;
|
||||
final ApplicationId appID;
|
||||
final String user = MockJobs.newUserName();
|
||||
final Map<JobId, Job> jobs;
|
||||
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);
|
||||
}
|
||||
|
||||
TestAppContext() {
|
||||
this(0, 1, 2, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationAttemptId getApplicationAttemptId() {
|
||||
return appAttemptID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationId getApplicationID() {
|
||||
return appID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Job getJob(JobId jobID) {
|
||||
return jobs.get(jobID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<JobId, Job> getAllJobs() {
|
||||
return jobs; // OK
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getApplicationName() {
|
||||
return "TestApp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
}
|
||||
|
||||
private Injector injector = Guice.createInjector(new ServletModule() {
|
||||
@Override
|
||||
protected void configureServlets() {
|
||||
|
||||
appContext = new TestAppContext();
|
||||
webApp = mock(HsWebApp.class);
|
||||
when(webApp.name()).thenReturn("hsmockwebapp");
|
||||
|
||||
bind(JAXBContextResolver.class);
|
||||
bind(HsWebServices.class);
|
||||
bind(GenericExceptionHandler.class);
|
||||
bind(WebApp.class).toInstance(webApp);
|
||||
bind(AppContext.class).toInstance(appContext);
|
||||
bind(Configuration.class).toInstance(conf);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
});
|
||||
|
||||
public class GuiceServletConfig extends GuiceServletContextListener {
|
||||
|
||||
@Override
|
||||
protected Injector getInjector() {
|
||||
return injector;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
}
|
||||
|
||||
public TestHsWebServicesTasks() {
|
||||
super(new WebAppDescriptor.Builder(
|
||||
"org.apache.hadoop.mapreduce.v2.hs.webapp")
|
||||
.contextListenerClass(GuiceServletConfig.class)
|
||||
.filterClass(com.google.inject.servlet.GuiceFilter.class)
|
||||
.contextPath("jersey-guice-filter").servletPath("/").build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasks() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 2, arr.length());
|
||||
|
||||
verifyHsTask(arr, jobsMap.get(id), null);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 2, arr.length());
|
||||
|
||||
verifyHsTask(arr, jobsMap.get(id), null);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks/")
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 2, arr.length());
|
||||
|
||||
verifyHsTask(arr, jobsMap.get(id), null);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksXML() throws JSONException, Exception {
|
||||
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList tasks = dom.getElementsByTagName("tasks");
|
||||
assertEquals("incorrect number of elements", 1, tasks.getLength());
|
||||
NodeList task = dom.getElementsByTagName("task");
|
||||
verifyHsTaskXML(task, jobsMap.get(id));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksQueryMap() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String type = "m";
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.queryParam("type", type).accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 1, arr.length());
|
||||
verifyHsTask(arr, jobsMap.get(id), type);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksQueryReduce() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String type = "r";
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.queryParam("type", type).accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject tasks = json.getJSONObject("tasks");
|
||||
JSONArray arr = tasks.getJSONArray("task");
|
||||
assertEquals("incorrect number of elements", 1, arr.length());
|
||||
verifyHsTask(arr, jobsMap.get(id), type);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTasksQueryInvalid() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
// tasktype must be exactly either "m" or "r"
|
||||
String tasktype = "reduce";
|
||||
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce").path("jobs")
|
||||
.path(jobId).path("tasks").queryParam("type", tasktype)
|
||||
.accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: tasktype must be either m or r", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"BadRequestException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.BadRequestException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskId() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("task");
|
||||
verifyHsSingleTask(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks")
|
||||
.path(tid + "/").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("task");
|
||||
verifyHsSingleTask(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("task");
|
||||
verifyHsSingleTask(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdBogus() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "bogustaskid";
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce").path("jobs")
|
||||
.path(jobId).path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: Error parsing task ID: bogustaskid", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdNonExist() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "task_1234_0_0_m_0";
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce").path("jobs")
|
||||
.path(jobId).path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: task not found with id task_1234_0_0_m_0",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdInvalid() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "task_1234_0_0_d_0";
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce").path("jobs")
|
||||
.path(jobId).path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: Unknown task symbol: d", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdInvalid2() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "task_1234_0_m_0";
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce").path("jobs")
|
||||
.path(jobId).path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: For input string: \"m\"", message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdInvalid3() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
String tid = "task_1234_0_0_m";
|
||||
try {
|
||||
r.path("ws").path("v1").path("history").path("mapreduce").path("jobs")
|
||||
.path(jobId).path("tasks").path(tid).get(JSONObject.class);
|
||||
fail("should have thrown exception on invalid uri");
|
||||
} catch (UniformInterfaceException ue) {
|
||||
ClientResponse response = ue.getResponse();
|
||||
assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject msg = response.getEntity(JSONObject.class);
|
||||
JSONObject exception = msg.getJSONObject("RemoteException");
|
||||
assertEquals("incorrect number of elements", 3, exception.length());
|
||||
String message = exception.getString("message");
|
||||
String type = exception.getString("exception");
|
||||
String classname = exception.getString("javaClassName");
|
||||
WebServicesTestUtils.checkStringMatch("exception message",
|
||||
"java.lang.Exception: Error parsing task ID: task_1234_0_0_m",
|
||||
message);
|
||||
WebServicesTestUtils.checkStringMatch("exception type",
|
||||
"NotFoundException", type);
|
||||
WebServicesTestUtils.checkStringMatch("exception classname",
|
||||
"org.apache.hadoop.yarn.webapp.NotFoundException", classname);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdXML() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
|
||||
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList nodes = dom.getElementsByTagName("task");
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
verifyHsSingleTaskXML(element, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsSingleTask(JSONObject info, Task task)
|
||||
throws JSONException {
|
||||
assertEquals("incorrect number of elements", 8, info.length());
|
||||
|
||||
verifyTaskGeneric(task, info.getString("id"), info.getString("state"),
|
||||
info.getString("type"), info.getString("successfulAttempt"),
|
||||
info.getLong("startTime"), info.getLong("finishTime"),
|
||||
info.getLong("elapsedTime"), (float) info.getDouble("progress"));
|
||||
}
|
||||
|
||||
public void verifyHsTask(JSONArray arr, Job job, String type)
|
||||
throws JSONException {
|
||||
for (Task task : job.getTasks().values()) {
|
||||
TaskId id = task.getID();
|
||||
String tid = MRApps.toString(id);
|
||||
Boolean found = false;
|
||||
if (type != null && task.getType() == MRApps.taskType(type)) {
|
||||
|
||||
for (int i = 0; i < arr.length(); i++) {
|
||||
JSONObject info = arr.getJSONObject(i);
|
||||
if (tid.matches(info.getString("id"))) {
|
||||
found = true;
|
||||
verifyHsSingleTask(info, task);
|
||||
}
|
||||
}
|
||||
assertTrue("task with id: " + tid + " not in web service output", found);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyTaskGeneric(Task task, String id, String state,
|
||||
String type, String successfulAttempt, long startTime, long finishTime,
|
||||
long elapsedTime, float progress) {
|
||||
|
||||
TaskId taskid = task.getID();
|
||||
String tid = MRApps.toString(taskid);
|
||||
TaskReport report = task.getReport();
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", tid, id);
|
||||
WebServicesTestUtils.checkStringMatch("type", task.getType().toString(),
|
||||
type);
|
||||
WebServicesTestUtils.checkStringMatch("state", report.getTaskState()
|
||||
.toString(), state);
|
||||
// not easily checked without duplicating logic, just make sure its here
|
||||
assertNotNull("successfulAttempt null", successfulAttempt);
|
||||
assertEquals("startTime wrong", report.getStartTime(), startTime);
|
||||
assertEquals("finishTime wrong", report.getFinishTime(), finishTime);
|
||||
assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime);
|
||||
assertEquals("progress wrong", report.getProgress() * 100, progress, 1e-3f);
|
||||
}
|
||||
|
||||
public void verifyHsSingleTaskXML(Element element, Task task) {
|
||||
verifyTaskGeneric(task, WebServicesTestUtils.getXmlString(element, "id"),
|
||||
WebServicesTestUtils.getXmlString(element, "state"),
|
||||
WebServicesTestUtils.getXmlString(element, "type"),
|
||||
WebServicesTestUtils.getXmlString(element, "successfulAttempt"),
|
||||
WebServicesTestUtils.getXmlLong(element, "startTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "finishTime"),
|
||||
WebServicesTestUtils.getXmlLong(element, "elapsedTime"),
|
||||
WebServicesTestUtils.getXmlFloat(element, "progress"));
|
||||
}
|
||||
|
||||
public void verifyHsTaskXML(NodeList nodes, Job job) {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, nodes.getLength());
|
||||
|
||||
for (Task task : job.getTasks().values()) {
|
||||
TaskId id = task.getID();
|
||||
String tid = MRApps.toString(id);
|
||||
Boolean found = false;
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
Element element = (Element) nodes.item(i);
|
||||
|
||||
if (tid.matches(WebServicesTestUtils.getXmlString(element, "id"))) {
|
||||
found = true;
|
||||
verifyHsSingleTaskXML(element, task);
|
||||
}
|
||||
}
|
||||
assertTrue("task with id: " + tid + " not in web service output", found);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdCounters() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("counters").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobTaskCounters");
|
||||
verifyHsJobTaskCounters(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdCountersSlash() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("counters/").accept(MediaType.APPLICATION_JSON)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobTaskCounters");
|
||||
verifyHsJobTaskCounters(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTaskIdCountersDefault() throws JSONException, Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("counters").get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
||||
JSONObject json = response.getEntity(JSONObject.class);
|
||||
assertEquals("incorrect number of elements", 1, json.length());
|
||||
JSONObject info = json.getJSONObject("jobTaskCounters");
|
||||
verifyHsJobTaskCounters(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJobTaskCountersXML() throws Exception {
|
||||
WebResource r = resource();
|
||||
Map<JobId, Job> jobsMap = appContext.getAllJobs();
|
||||
for (JobId id : jobsMap.keySet()) {
|
||||
String jobId = MRApps.toString(id);
|
||||
for (Task task : jobsMap.get(id).getTasks().values()) {
|
||||
|
||||
String tid = MRApps.toString(task.getID());
|
||||
ClientResponse response = r.path("ws").path("v1").path("history")
|
||||
.path("mapreduce").path("jobs").path(jobId).path("tasks").path(tid)
|
||||
.path("counters").accept(MediaType.APPLICATION_XML)
|
||||
.get(ClientResponse.class);
|
||||
assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
|
||||
String xml = response.getEntity(String.class);
|
||||
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
||||
DocumentBuilder db = dbf.newDocumentBuilder();
|
||||
InputSource is = new InputSource();
|
||||
is.setCharacterStream(new StringReader(xml));
|
||||
Document dom = db.parse(is);
|
||||
NodeList info = dom.getElementsByTagName("jobTaskCounters");
|
||||
verifyHsTaskCountersXML(info, task);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsJobTaskCounters(JSONObject info, Task task)
|
||||
throws JSONException {
|
||||
|
||||
assertEquals("incorrect number of elements", 2, info.length());
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(task.getID()),
|
||||
info.getString("id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
JSONArray counterGroups = info.getJSONArray("taskCounterGroup");
|
||||
for (int i = 0; i < counterGroups.length(); i++) {
|
||||
JSONObject counterGroup = counterGroups.getJSONObject(i);
|
||||
String name = counterGroup.getString("counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
JSONArray counters = counterGroup.getJSONArray("counter");
|
||||
for (int j = 0; j < counters.length(); j++) {
|
||||
JSONObject counter = counters.getJSONObject(i);
|
||||
String counterName = counter.getString("name");
|
||||
assertTrue("name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
long value = counter.getLong("value");
|
||||
assertTrue("value >= 0", value >= 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void verifyHsTaskCountersXML(NodeList nodes, Task task) {
|
||||
|
||||
for (int i = 0; i < nodes.getLength(); i++) {
|
||||
|
||||
Element element = (Element) nodes.item(i);
|
||||
WebServicesTestUtils.checkStringMatch("id",
|
||||
MRApps.toString(task.getID()),
|
||||
WebServicesTestUtils.getXmlString(element, "id"));
|
||||
// just do simple verification of fields - not data is correct
|
||||
// in the fields
|
||||
NodeList groups = element.getElementsByTagName("taskCounterGroup");
|
||||
|
||||
for (int j = 0; j < groups.getLength(); j++) {
|
||||
Element counters = (Element) groups.item(j);
|
||||
assertNotNull("should have counters in the web service info", counters);
|
||||
String name = WebServicesTestUtils.getXmlString(counters,
|
||||
"counterGroupName");
|
||||
assertTrue("name not set", (name != null && !name.isEmpty()));
|
||||
NodeList counterArr = counters.getElementsByTagName("counter");
|
||||
for (int z = 0; z < counterArr.getLength(); z++) {
|
||||
Element counter = (Element) counterArr.item(z);
|
||||
String counterName = WebServicesTestUtils.getXmlString(counter,
|
||||
"name");
|
||||
assertTrue("counter name not set",
|
||||
(counterName != null && !counterName.isEmpty()));
|
||||
|
||||
long value = WebServicesTestUtils.getXmlLong(counter, "value");
|
||||
assertTrue("value not >= 0", value >= 0);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,133 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.hs.webapp;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
||||
import org.codehaus.jettison.json.JSONException;
|
||||
import org.codehaus.jettison.json.JSONObject;
|
||||
|
||||
public class VerifyJobsUtils {
|
||||
|
||||
public static void verifyHsJob(JSONObject info, Job job) throws JSONException {
|
||||
|
||||
// this is 23 instead of 24 because acls not being checked since
|
||||
// we are using mock job instead of CompletedJob
|
||||
assertEquals("incorrect number of elements", 23, info.length());
|
||||
|
||||
// everyone access fields
|
||||
verifyHsJobGeneric(job, info.getString("id"), info.getString("user"),
|
||||
info.getString("name"), info.getString("state"),
|
||||
info.getString("queue"), info.getLong("startTime"),
|
||||
info.getLong("finishTime"), info.getInt("mapsTotal"),
|
||||
info.getInt("mapsCompleted"), info.getInt("reducesTotal"),
|
||||
info.getInt("reducesCompleted"));
|
||||
|
||||
String diagnostics = "";
|
||||
if (info.has("diagnostics")) {
|
||||
diagnostics = info.getString("diagnostics");
|
||||
}
|
||||
|
||||
// restricted access fields - if security and acls set
|
||||
verifyHsJobGenericSecure(job, info.getBoolean("uberized"), diagnostics,
|
||||
info.getLong("avgMapTime"), info.getLong("avgReduceTime"),
|
||||
info.getLong("avgShuffleTime"), info.getLong("avgMergeTime"),
|
||||
info.getInt("failedReduceAttempts"),
|
||||
info.getInt("killedReduceAttempts"),
|
||||
info.getInt("successfulReduceAttempts"),
|
||||
info.getInt("failedMapAttempts"), info.getInt("killedMapAttempts"),
|
||||
info.getInt("successfulMapAttempts"));
|
||||
|
||||
// acls not being checked since
|
||||
// we are using mock job instead of CompletedJob
|
||||
}
|
||||
|
||||
public static void verifyHsJobGeneric(Job job, String id, String user,
|
||||
String name, String state, String queue, long startTime, long finishTime,
|
||||
int mapsTotal, int mapsCompleted, int reducesTotal, int reducesCompleted) {
|
||||
JobReport report = job.getReport();
|
||||
|
||||
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
|
||||
id);
|
||||
WebServicesTestUtils.checkStringMatch("user", job.getUserName().toString(),
|
||||
user);
|
||||
WebServicesTestUtils.checkStringMatch("name", job.getName(), name);
|
||||
WebServicesTestUtils.checkStringMatch("state", job.getState().toString(),
|
||||
state);
|
||||
WebServicesTestUtils.checkStringMatch("queue", job.getQueueName(), queue);
|
||||
|
||||
assertEquals("startTime incorrect", report.getStartTime(), startTime);
|
||||
assertEquals("finishTime incorrect", report.getFinishTime(), finishTime);
|
||||
|
||||
assertEquals("mapsTotal incorrect", job.getTotalMaps(), mapsTotal);
|
||||
assertEquals("mapsCompleted incorrect", job.getCompletedMaps(),
|
||||
mapsCompleted);
|
||||
assertEquals("reducesTotal incorrect", job.getTotalReduces(), reducesTotal);
|
||||
assertEquals("reducesCompleted incorrect", job.getCompletedReduces(),
|
||||
reducesCompleted);
|
||||
}
|
||||
|
||||
public static void verifyHsJobGenericSecure(Job job, Boolean uberized,
|
||||
String diagnostics, long avgMapTime, long avgReduceTime,
|
||||
long avgShuffleTime, long avgMergeTime, int failedReduceAttempts,
|
||||
int killedReduceAttempts, int successfulReduceAttempts,
|
||||
int failedMapAttempts, int killedMapAttempts, int successfulMapAttempts) {
|
||||
|
||||
String diagString = "";
|
||||
List<String> diagList = job.getDiagnostics();
|
||||
if (diagList != null && !diagList.isEmpty()) {
|
||||
StringBuffer b = new StringBuffer();
|
||||
for (String diag : diagList) {
|
||||
b.append(diag);
|
||||
}
|
||||
diagString = b.toString();
|
||||
}
|
||||
WebServicesTestUtils.checkStringMatch("diagnostics", diagString,
|
||||
diagnostics);
|
||||
|
||||
assertEquals("isUber incorrect", job.isUber(), uberized);
|
||||
|
||||
// unfortunately the following fields are all calculated in JobInfo
|
||||
// so not easily accessible without doing all the calculations again.
|
||||
// For now just make sure they are present.
|
||||
|
||||
assertTrue("failedReduceAttempts not >= 0", failedReduceAttempts >= 0);
|
||||
assertTrue("killedReduceAttempts not >= 0", killedReduceAttempts >= 0);
|
||||
assertTrue("successfulReduceAttempts not >= 0",
|
||||
successfulReduceAttempts >= 0);
|
||||
|
||||
assertTrue("failedMapAttempts not >= 0", failedMapAttempts >= 0);
|
||||
assertTrue("killedMapAttempts not >= 0", killedMapAttempts >= 0);
|
||||
assertTrue("successfulMapAttempts not >= 0", successfulMapAttempts >= 0);
|
||||
|
||||
assertTrue("avgMapTime not >= 0", avgMapTime >= 0);
|
||||
assertTrue("avgReduceTime not >= 0", avgReduceTime >= 0);
|
||||
assertTrue("avgShuffleTime not >= 0", avgShuffleTime >= 0);
|
||||
assertTrue("avgMergeTime not >= 0", avgMergeTime >= 0);
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -50,6 +50,9 @@ public class WebServicesTestUtils {
|
|||
public static String getXmlString(Element element, String name) {
|
||||
NodeList id = element.getElementsByTagName(name);
|
||||
Element line = (Element) id.item(0);
|
||||
if (line == null) {
|
||||
return null;
|
||||
}
|
||||
Node first = line.getFirstChild();
|
||||
// handle empty <key></key>
|
||||
if (first == null) {
|
||||
|
|
|
@ -1,83 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager;
|
||||
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
|
||||
public class MockApp implements Application {
|
||||
|
||||
final String user;
|
||||
final ApplicationId appId;
|
||||
Map<ContainerId, Container> containers = new HashMap<ContainerId, Container>();
|
||||
ApplicationState appState;
|
||||
Application app;
|
||||
|
||||
public MockApp(int uniqId) {
|
||||
this("mockUser", 1234, uniqId);
|
||||
}
|
||||
|
||||
public MockApp(String user, long clusterTimeStamp, int uniqId) {
|
||||
super();
|
||||
this.user = user;
|
||||
// Add an application and the corresponding containers
|
||||
RecordFactory recordFactory = RecordFactoryProvider
|
||||
.getRecordFactory(new Configuration());
|
||||
this.appId = BuilderUtils.newApplicationId(recordFactory, clusterTimeStamp,
|
||||
uniqId);
|
||||
appState = ApplicationState.NEW;
|
||||
}
|
||||
|
||||
public void setState(ApplicationState state) {
|
||||
this.appState = state;
|
||||
}
|
||||
|
||||
public String getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
public Map<ContainerId, Container> getContainers() {
|
||||
return containers;
|
||||
}
|
||||
|
||||
public ApplicationId getAppId() {
|
||||
return appId;
|
||||
}
|
||||
|
||||
public ApplicationState getApplicationState() {
|
||||
return appState;
|
||||
}
|
||||
|
||||
public void handle(ApplicationEvent event) {}
|
||||
|
||||
}
|
|
@ -1,120 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
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.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
|
||||
public class MockContainer implements Container {
|
||||
|
||||
private ContainerId id;
|
||||
private ContainerState state;
|
||||
private String user;
|
||||
private ContainerLaunchContext launchContext;
|
||||
private final Map<Path, String> resource = new HashMap<Path, String>();
|
||||
private RecordFactory recordFactory;
|
||||
|
||||
public MockContainer(ApplicationAttemptId appAttemptId,
|
||||
Dispatcher dispatcher, Configuration conf, String user,
|
||||
ApplicationId appId, int uniqId) {
|
||||
|
||||
this.user = user;
|
||||
this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
|
||||
this.id = BuilderUtils.newContainerId(recordFactory, appId, appAttemptId,
|
||||
uniqId);
|
||||
this.launchContext = recordFactory
|
||||
.newRecordInstance(ContainerLaunchContext.class);
|
||||
launchContext.setContainerId(id);
|
||||
launchContext.setUser(user);
|
||||
this.state = ContainerState.NEW;
|
||||
|
||||
}
|
||||
|
||||
public void setState(ContainerState state) {
|
||||
this.state = state;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerId getContainerID() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerState getContainerState() {
|
||||
return state;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerLaunchContext getLaunchContext() {
|
||||
return launchContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Credentials getCredentials() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Path, String> getLocalizedResources() {
|
||||
return resource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerStatus cloneAndGetContainerStatus() {
|
||||
ContainerStatus containerStatus = recordFactory
|
||||
.newRecordInstance(ContainerStatus.class);
|
||||
containerStatus
|
||||
.setState(org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
|
||||
containerStatus.setContainerId(this.launchContext.getContainerId());
|
||||
containerStatus.setDiagnostics("testing");
|
||||
containerStatus.setExitStatus(0);
|
||||
return containerStatus;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handle(ContainerEvent event) {
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.webapp;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
|
||||
public class MockApp implements Application {
|
||||
|
||||
final String user;
|
||||
final ApplicationId appId;
|
||||
Map<ContainerId, Container> containers = new HashMap<ContainerId, Container>();
|
||||
ApplicationState appState;
|
||||
Application app;
|
||||
|
||||
public MockApp(int uniqId) {
|
||||
this("mockUser", 1234, uniqId);
|
||||
}
|
||||
|
||||
public MockApp(String user, long clusterTimeStamp, int uniqId) {
|
||||
super();
|
||||
this.user = user;
|
||||
// Add an application and the corresponding containers
|
||||
RecordFactory recordFactory = RecordFactoryProvider
|
||||
.getRecordFactory(new Configuration());
|
||||
this.appId = BuilderUtils.newApplicationId(recordFactory, clusterTimeStamp,
|
||||
uniqId);
|
||||
appState = ApplicationState.NEW;
|
||||
}
|
||||
|
||||
public void setState(ApplicationState state) {
|
||||
this.appState = state;
|
||||
}
|
||||
|
||||
public String getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
public Map<ContainerId, Container> getContainers() {
|
||||
return containers;
|
||||
}
|
||||
|
||||
public ApplicationId getAppId() {
|
||||
return appId;
|
||||
}
|
||||
|
||||
public ApplicationState getApplicationState() {
|
||||
return appState;
|
||||
}
|
||||
|
||||
public void handle(ApplicationEvent event) {}
|
||||
|
||||
}
|
|
@ -0,0 +1,120 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.webapp;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
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.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
|
||||
public class MockContainer implements Container {
|
||||
|
||||
private ContainerId id;
|
||||
private ContainerState state;
|
||||
private String user;
|
||||
private ContainerLaunchContext launchContext;
|
||||
private final Map<Path, String> resource = new HashMap<Path, String>();
|
||||
private RecordFactory recordFactory;
|
||||
|
||||
public MockContainer(ApplicationAttemptId appAttemptId,
|
||||
Dispatcher dispatcher, Configuration conf, String user,
|
||||
ApplicationId appId, int uniqId) {
|
||||
|
||||
this.user = user;
|
||||
this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
|
||||
this.id = BuilderUtils.newContainerId(recordFactory, appId, appAttemptId,
|
||||
uniqId);
|
||||
this.launchContext = recordFactory
|
||||
.newRecordInstance(ContainerLaunchContext.class);
|
||||
launchContext.setContainerId(id);
|
||||
launchContext.setUser(user);
|
||||
this.state = ContainerState.NEW;
|
||||
|
||||
}
|
||||
|
||||
public void setState(ContainerState state) {
|
||||
this.state = state;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerId getContainerID() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getUser() {
|
||||
return user;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerState getContainerState() {
|
||||
return state;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerLaunchContext getLaunchContext() {
|
||||
return launchContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Credentials getCredentials() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Path, String> getLocalizedResources() {
|
||||
return resource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerStatus cloneAndGetContainerStatus() {
|
||||
ContainerStatus containerStatus = recordFactory
|
||||
.newRecordInstance(ContainerStatus.class);
|
||||
containerStatus
|
||||
.setState(org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
|
||||
containerStatus.setContainerId(this.launchContext.getContainerId());
|
||||
containerStatus.setDiagnostics("testing");
|
||||
containerStatus.setExitStatus(0);
|
||||
return containerStatus;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handle(ContainerEvent event) {
|
||||
}
|
||||
|
||||
}
|
|
@ -38,8 +38,6 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.MockApp;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.MockContainer;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
|
||||
|
|
|
@ -39,8 +39,6 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.MockApp;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.MockContainer;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
|
||||
|
|
Loading…
Reference in New Issue