Browse Source

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
Arun Murthy 13 years ago
parent
commit
2281b945d8
36 changed files with 7395 additions and 614 deletions
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 116 132
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
  3. 1 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/AppInfo.java
  4. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/ConfInfo.java
  5. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/CounterInfo.java
  6. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobCounterInfo.java
  7. 9 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
  8. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobTaskAttemptCounterInfo.java
  9. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobTaskCounterInfo.java
  10. 4 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java
  11. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptsInfo.java
  12. 4 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskInfo.java
  13. 105 80
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
  14. 359 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
  15. 732 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
  16. 336 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java
  17. 780 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
  18. 821 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
  19. 47 137
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
  20. 8 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/JAXBContextResolver.java
  21. 21 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/AMAttemptInfo.java
  22. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobInfo.java
  23. 360 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServices.java
  24. 745 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAttempts.java
  25. 345 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobConf.java
  26. 755 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobs.java
  27. 656 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java
  28. 835 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesTasks.java
  29. 133 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/VerifyJobsUtils.java
  30. 3 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java
  31. 0 83
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockApp.java
  32. 0 120
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockContainer.java
  33. 80 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java
  34. 120 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
  35. 0 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java
  36. 0 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -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

+ 116 - 132
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java

@@ -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");
-    }
+
+    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);
     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);
   }
 }

+ 1 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/AppInfo.java

@@ -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() {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/ConfInfo.java

@@ -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 {
 

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/CounterInfo.java

@@ -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();

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobCounterInfo.java

@@ -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);
         }
       }
     }

+ 9 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java

@@ -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() {
@@ -267,13 +265,11 @@ public class JobInfo {
   /**
    * Go through a job and update the member variables with counts for
    * information to output in the page.
-   * 
+   *
    * @param job
    *          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;

+ 3 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobTaskAttemptCounterInfo.java

@@ -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);
           }
         }
       }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobTaskCounterInfo.java

@@ -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);
         }
       }
     }

+ 4 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java

@@ -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() {

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptsInfo.java

@@ -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;
   }
 
 }

+ 4 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskInfo.java

@@ -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() {

+ 105 - 80
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java

@@ -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,33 +64,38 @@ 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(
-      "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_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<Phase> PHASES = Iterators.cycle(Phase.values());
   static final Iterator<String> DIAGS = Iterators.cycle(
       "Error: java.lang.OutOfMemoryError: Java heap space",
       "Lost task tracker: tasktracker.domain/127.0.0.1:40879");
-  
+
   public static final String NM_HOST = "localhost";
   public static final int NM_PORT = 1234;
   public static final int NM_HTTP_PORT = 9999;
@@ -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,41 +152,42 @@ 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);
   }
 
   public static Map<TaskAttemptId, TaskAttempt> newTaskAttempts(TaskId tid,
-                                                                int m) {
+      int m) {
     Map<TaskAttemptId, TaskAttempt> map = Maps.newHashMap();
     for (int i = 0; i < m; ++i) {
       TaskAttempt ta = newTaskAttempt(tid, i);
@@ -237,9 +247,10 @@ public class MockJobs extends MockApps {
       @Override
       public boolean isFinished() {
         switch (report.getTaskAttemptState()) {
-          case SUCCEEDED:
-          case FAILED:
-          case KILLED: return true;
+        case SUCCEEDED:
+        case FAILED:
+        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);
@@ -280,10 +291,10 @@ public class MockJobs extends MockApps {
         return 0;
       }
 
-	@Override
-	public String getNodeRackName() {
-		return "/default-rack";
-	}
+      @Override
+      public String getNodeRackName() {
+        return "/default-rack";
+      }
     };
   }
 
@@ -342,9 +353,10 @@ public class MockJobs extends MockApps {
       @Override
       public boolean isFinished() {
         switch (report.getTaskState()) {
-          case SUCCEEDED:
-          case KILLED:
-          case FAILED: return true;
+        case SUCCEEDED:
+        case KILLED:
+        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
@@ -521,11 +547,10 @@ public class MockJobs extends MockApps {
       }
     };
   }
-  
+
   private static AMInfo createAMInfo(int attempt) {
-    ApplicationAttemptId appAttemptId =
-        BuilderUtils.newApplicationAttemptId(
-            BuilderUtils.newApplicationId(100, 1), attempt);
+    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
+        BuilderUtils.newApplicationId(100, 1), attempt);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     return MRBuilderUtils.newAMInfo(appAttemptId, System.currentTimeMillis(),
         containerId, NM_HOST, NM_PORT, NM_HTTP_PORT);

+ 359 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java

@@ -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));
+
+  }
+}

+ 732 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java

@@ -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);
+
+        }
+      }
+    }
+  }
+
+}

+ 336 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java

@@ -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()));
+      }
+    }
+  }
+
+}

+ 780 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java

@@ -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);
+        }
+      }
+    }
+  }
+
+}

+ 821 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java

@@ -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);
+
+        }
+      }
+    }
+  }
+
+}

+ 47 - 137
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java

@@ -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);
   }
 

+ 8 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/JAXBContextResolver.java

@@ -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));

+ 21 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/AMAttemptInfo.java

@@ -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.nodeId = nodeId.toString();
+    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() {

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobInfo.java

@@ -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();

+ 360 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServices.java

@@ -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"));
+    }
+  }
+
+}

+ 745 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAttempts.java

@@ -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);
+
+        }
+      }
+    }
+  }
+
+}

+ 345 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobConf.java

@@ -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()));
+      }
+    }
+  }
+
+}

+ 755 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobs.java

@@ -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);
+  }
+
+}

+ 656 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java

@@ -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());
+  }
+
+}

+ 835 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesTasks.java

@@ -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);
+
+        }
+      }
+    }
+  }
+
+}

+ 133 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/VerifyJobsUtils.java

@@ -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);
+
+  }
+
+}

+ 3 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java

@@ -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) {

+ 0 - 83
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockApp.java

@@ -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) {}
-
-}

+ 0 - 120
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockContainer.java

@@ -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) {
-  }
-
-}

+ 80 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java

@@ -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) {}
+
+}

+ 120 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java

@@ -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) {
+  }
+
+}

+ 0 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java

@@ -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;

+ 0 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java

@@ -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;