You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by vi...@apache.org on 2011/09/29 10:08:50 UTC

svn commit: r1177203 - in /hadoop/common/trunk/hadoop-mapreduce-project: ./ hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/o...

Author: vinodkv
Date: Thu Sep 29 08:08:49 2011
New Revision: 1177203

URL: http://svn.apache.org/viewvc?rev=1177203&view=rev
Log:
MAPREDUCE-3001. Added task-specific counters to AppMaster and JobHistory web-UIs. Contributed by Robert Joseph Evans.

Added:
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
Modified:
    hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMParams.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebApp.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java

Modified: hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt Thu Sep 29 08:08:49 2011
@@ -321,6 +321,9 @@ Release 0.23.0 - Unreleased
     MAPREDUCE-3099. Add docs for setting up a single node MRv2 cluster.
     (mahadev)
 
+    MAPREDUCE-3001. Added task-specific counters to AppMaster and JobHistory
+    web-UIs. (Robert Joseph Evans via vinodkv)
+
   OPTIMIZATIONS
 
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMParams.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMParams.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMParams.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMParams.java Thu Sep 29 08:08:49 2011
@@ -28,4 +28,6 @@ public interface AMParams {
   static final String TASK_ID = "task.id";
   static final String TASK_TYPE = "task.type";
   static final String ATTEMPT_STATE = "attempt.state";
+  static final String COUNTER_GROUP = "counter.group";
+  static final String COUNTER_NAME = "counter.name";
 }

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebApp.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebApp.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebApp.java Thu Sep 29 08:08:49 2011
@@ -34,9 +34,14 @@ public class AMWebApp extends WebApp imp
     route(pajoin("/job", JOB_ID), AppController.class, "job");
     route(pajoin("/conf", JOB_ID), AppController.class, "conf");
     route(pajoin("/jobcounters", JOB_ID), AppController.class, "jobCounters");
+    route(pajoin("/singlejobcounter",JOB_ID, COUNTER_GROUP, COUNTER_NAME),
+        AppController.class, "singleJobCounter");
     route(pajoin("/tasks", JOB_ID, TASK_TYPE), AppController.class, "tasks");
     route(pajoin("/attempts", JOB_ID, TASK_TYPE, ATTEMPT_STATE),
         AppController.class, "attempts");
     route(pajoin("/task", TASK_ID), AppController.class, "task");
+    route(pajoin("/taskcounters", TASK_ID), AppController.class, "taskCounters");
+    route(pajoin("/singletaskcounter",TASK_ID, COUNTER_GROUP, COUNTER_NAME),
+        AppController.class, "singleTaskCounter");
   }
 }

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java Thu Sep 29 08:08:49 2011
@@ -20,6 +20,8 @@ package org.apache.hadoop.mapreduce.v2.a
 
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 
+import java.io.IOException;
+import java.net.URLDecoder;
 import java.util.Locale;
 
 import javax.servlet.http.HttpServletResponse;
@@ -30,7 +32,7 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.util.Apps;
+import org.apache.hadoop.yarn.util.StringHelper;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.Controller;
 import org.apache.hadoop.yarn.webapp.View;
@@ -41,7 +43,7 @@ import com.google.inject.Inject;
  * This class renders the various pages that the web app supports.
  */
 public class AppController extends Controller implements AMParams {
-  final App app;
+  protected final App app;
   
   protected AppController(App app, Configuration conf, RequestContext ctx,
       String title) {
@@ -109,6 +111,54 @@ public class AppController extends Contr
     }
     render(countersPage());
   }
+  
+  /**
+   * Display a page showing a task's counters
+   */
+  public void taskCounters() {
+    requireTask();
+    if (app.getTask() != null) {
+      setTitle(StringHelper.join("Counters for ", $(TASK_ID)));
+    }
+    render(countersPage());
+  }
+  
+  /**
+   * @return the class that will render the /singlejobcounter page
+   */
+  protected Class<? extends View> singleCounterPage() {
+    return SingleCounterPage.class;
+  }
+  
+  /**
+   * Render the /singlejobcounter page
+   * @throws IOException on any error.
+   */
+  public void singleJobCounter() throws IOException{
+    requireJob();
+    set(COUNTER_GROUP, URLDecoder.decode($(COUNTER_GROUP), "UTF-8"));
+    set(COUNTER_NAME, URLDecoder.decode($(COUNTER_NAME), "UTF-8"));
+    if (app.getJob() != null) {
+      setTitle(StringHelper.join($(COUNTER_GROUP)," ",$(COUNTER_NAME),
+          " for ", $(JOB_ID)));
+    }
+    render(singleCounterPage());
+  }
+  
+  /**
+   * Render the /singletaskcounter page
+   * @throws IOException on any error.
+   */
+  public void singleTaskCounter() throws IOException{
+    requireTask();
+    set(COUNTER_GROUP, URLDecoder.decode($(COUNTER_GROUP), "UTF-8"));
+    set(COUNTER_NAME, URLDecoder.decode($(COUNTER_NAME), "UTF-8"));
+    if (app.getTask() != null) {
+      setTitle(StringHelper.join($(COUNTER_GROUP)," ",$(COUNTER_NAME),
+          " for ", $(TASK_ID)));
+    }
+    render(singleCounterPage());
+  }
 
   /**
    * @return the class that will render the /tasks page

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java Thu Sep 29 08:08:49 2011
@@ -61,6 +61,29 @@ public class CountersBlock extends HtmlB
         p()._("Sorry, no counters for nonexistent", $(TASK_ID, "task"))._();
       return;
     }
+    
+    if(total == null || total.getAllCounterGroups() == null || 
+        total.getAllCounterGroups().size() <= 0) {
+      String type = $(TASK_ID);
+      if(type == null || type.isEmpty()) {
+        type = $(JOB_ID, "the job");
+      }
+      html.
+        p()._("Sorry it looks like ",type," has no counters.")._();
+      return;
+    }
+    
+    String urlBase;
+    String urlId;
+    if(task != null) {
+      urlBase = "singletaskcounter";
+      urlId = MRApps.toString(task.getID());
+    } else {
+      urlBase = "singlejobcounter";
+      urlId = MRApps.toString(job.getID());
+    }
+    
+    
     int numGroups = 0;
     TBODY<TABLE<DIV<Hamlet>>> tbody = html.
       div(_INFO_WRAP).
@@ -79,12 +102,13 @@ public class CountersBlock extends HtmlB
       // serves as an indicator of where we're in the tag hierarchy.
       TR<THEAD<TABLE<TD<TR<TBODY<TABLE<DIV<Hamlet>>>>>>>> groupHeadRow = tbody.
         tr().
-          th().$title(g.getName()).
+          th().$title(g.getName()).$class("ui-state-default").
             _(fixGroupDisplayName(g.getDisplayName()))._().
           td().$class(C_TABLE).
             table(".dt-counters").
               thead().
                 tr().th(".name", "Name");
+
       if (map != null) {
         groupHeadRow.th("Map").th("Reduce");
       }
@@ -97,7 +121,9 @@ public class CountersBlock extends HtmlB
         TR<TBODY<TABLE<TD<TR<TBODY<TABLE<DIV<Hamlet>>>>>>>> groupRow = group.
           tr().
             td().$title(counter.getName()).
-              _(counter.getDisplayName())._();
+              a(url(urlBase,urlId,g.getName(), 
+                  counter.getName()), counter.getDisplayName()).
+            _();
         if (map != null) {
           Counter mc = mg == null ? null : mg.getCounter(counter.getName());
           Counter rc = rg == null ? null : rg.getCounter(counter.getName());
@@ -121,7 +147,7 @@ public class CountersBlock extends HtmlB
       jobID = taskID.getJobId();
     } else {
       String jid = $(JOB_ID);
-      if (!jid.isEmpty()) {
+      if (jid != null && !jid.isEmpty()) {
         jobID = MRApps.toJobID(jid);
       }
     }

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java Thu Sep 29 08:08:49 2011
@@ -20,13 +20,19 @@ package org.apache.hadoop.mapreduce.v2.a
 
 import org.apache.hadoop.yarn.webapp.SubView;
 
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 
 public class CountersPage extends AppView {
 
   @Override protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
-    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:2}");
+    String tid = $(TASK_ID);
+    String activeNav = "3";
+    if(tid == null || tid.isEmpty()) {
+      activeNav = "2";
+    }
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
     set(DATATABLES_SELECTOR, "#counters .dt-counters");
     set(initSelector(DATATABLES),
         "{bJQueryUI:true, sDom:'t', iDisplayLength:-1}");
@@ -35,9 +41,9 @@ public class CountersPage extends AppVie
   @Override protected void postHead(Page.HTML<_> html) {
     html.
       style("#counters, .dt-counters { table-layout: fixed }",
-            "#counters th { overflow: hidden; vertical-align: center }",
+            "#counters th { overflow: hidden; vertical-align: middle }",
             "#counters .dataTables_wrapper { min-height: 1em }",
-            "#counters .group { width: 10em }",
+            "#counters .group { width: 15em }",
             "#counters .name { width: 30em }");
   }
 

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java Thu Sep 29 08:08:49 2011
@@ -55,6 +55,14 @@ public class NavBlock extends HtmlBlock 
           li().a(url("conf", jobid), "Configuration")._().
           li().a(url("tasks", jobid, "m"), "Map tasks")._().
           li().a(url("tasks", jobid, "r"), "Reduce tasks")._()._();
+      if (app.getTask() != null) {
+        String taskid = MRApps.toString(app.getTask().getID());
+        nav.
+          h3("Task").
+          ul().
+            li().a(url("task", taskid), "Task Overview")._().
+            li().a(url("taskcounters", taskid), "Counters")._()._();
+      }
     }
     nav.
       h3("Tools").

Added: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java?rev=1177203&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java (added)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java Thu Sep 29 08:08:49 2011
@@ -0,0 +1,151 @@
+/**
+* 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 com.google.inject.Inject;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.mapreduce.v2.api.records.Counter;
+import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
+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.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.util.MRApps;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp.*;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
+
+public class SingleCounterBlock extends HtmlBlock {
+  protected TreeMap<String, Long> values = new TreeMap<String, Long>(); 
+  protected Job job;
+  protected Task task;
+  
+  @Inject SingleCounterBlock(AppContext appCtx, ViewContext ctx) {
+    super(ctx);
+    this.populateMembers(appCtx);
+  }
+
+  @Override protected void render(Block html) {
+    if (job == null) {
+      html.
+        p()._("Sorry, no counters for nonexistent", $(JOB_ID, "job"))._();
+      return;
+    }
+    if (!$(TASK_ID).isEmpty() && task == null) {
+      html.
+        p()._("Sorry, no counters for nonexistent", $(TASK_ID, "task"))._();
+      return;
+    }
+    
+    String columnType = task == null ? "Task" : "Task Attempt";
+    
+    TBODY<TABLE<DIV<Hamlet>>> tbody = html.
+      div(_INFO_WRAP).
+      table("#singleCounter").
+        thead().
+          tr().
+            th(".ui-state-default", columnType).
+            th(".ui-state-default", "Value")._()._().
+          tbody();
+    for (Map.Entry<String, Long> entry : values.entrySet()) {
+      TR<TBODY<TABLE<DIV<Hamlet>>>> row = tbody.tr();
+      String id = entry.getKey();
+      String val = entry.getValue().toString();
+      if(task != null) {
+        row.td(id);
+        row.td().br().$title(val)._()._(val)._();
+      } else {
+        row.td().a(url("singletaskcounter",entry.getKey(),
+            $(COUNTER_GROUP), $(COUNTER_NAME)), id)._();
+        row.td().br().$title(val)._().a(url("singletaskcounter",entry.getKey(),
+            $(COUNTER_GROUP), $(COUNTER_NAME)), val)._();
+      }
+      row._();
+    }
+    tbody._()._()._();
+  }
+
+  private void populateMembers(AppContext ctx) {
+    JobId jobID = null;
+    TaskId taskID = null;
+    String tid = $(TASK_ID);
+    if (!tid.isEmpty()) {
+      taskID = MRApps.toTaskID(tid);
+      jobID = taskID.getJobId();
+    } else {
+      String jid = $(JOB_ID);
+      if (!jid.isEmpty()) {
+        jobID = MRApps.toJobID(jid);
+      }
+    }
+    if (jobID == null) {
+      return;
+    }
+    job = ctx.getJob(jobID);
+    if (job == null) {
+      return;
+    }
+    if (taskID != null) {
+      task = job.getTask(taskID);
+      if (task == null) {
+        return;
+      }
+      for(Map.Entry<TaskAttemptId, TaskAttempt> entry : 
+        task.getAttempts().entrySet()) {
+        long value = 0;
+        CounterGroup group = entry.getValue().getCounters()
+        .getCounterGroup($(COUNTER_GROUP));
+        if(group != null)  {
+          Counter c = group.getCounter($(COUNTER_NAME));
+          if(c != null) {
+            value = c.getValue();
+          }
+        }
+        values.put(MRApps.toString(entry.getKey()), value);
+      }
+      
+      return;
+    }
+    // Get all types of counters
+    Map<TaskId, Task> tasks = job.getTasks();
+    for(Map.Entry<TaskId, Task> entry : tasks.entrySet()) {
+      long value = 0;
+      CounterGroup group = entry.getValue().getCounters()
+      .getCounterGroup($(COUNTER_GROUP));
+      if(group != null)  {
+        Counter c = group.getCounter($(COUNTER_NAME));
+        if(c != null) {
+          value = c.getValue();
+        }
+      }
+      values.put(MRApps.toString(entry.getKey()), value);
+    }
+  }
+}

Added: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java?rev=1177203&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java (added)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java Thu Sep 29 08:08:49 2011
@@ -0,0 +1,69 @@
+/**
+* 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.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
+
+import org.apache.hadoop.mapreduce.v2.app.webapp.SingleCounterBlock;
+import org.apache.hadoop.yarn.webapp.SubView;
+
+/**
+ * Render the counters page
+ */
+public class SingleCounterPage extends AppView {
+
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
+   */
+  @Override protected void preHead(Page.HTML<_> html) {
+    commonPreHead(html);
+    String tid = $(TASK_ID);
+    String activeNav = "3";
+    if(tid == null || tid.isEmpty()) {
+      activeNav = "2";
+    }
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
+    set(DATATABLES_ID, "singleCounter");
+    set(initID(DATATABLES, "singleCounter"), counterTableInit());
+    setTableStyles(html, "singleCounter");
+  }
+
+  /**
+   * @return The end of a javascript map that is the jquery datatable 
+   * configuration for the jobs table.  the Jobs table is assumed to be
+   * rendered by the class returned from {@link #content()} 
+   */
+  private String counterTableInit() {
+    return tableInit().
+        append(",aoColumnDefs:[").
+        append("{'sType':'title-numeric', 'aTargets': [ 1 ] }").
+        append("]}").
+        toString();
+  }
+  
+  /**
+   * The content of this page is the CountersBlock now.
+   * @return CountersBlock.class
+   */
+  @Override protected Class<? extends SubView> content() {
+    return SingleCounterBlock.class;
+  }
+}

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java Thu Sep 29 08:08:49 2011
@@ -108,7 +108,7 @@ public class TaskPage extends AppView {
 
   @Override protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
-    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:2}");
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:3}");
     set(DATATABLES_ID, "attempts");
     set(initID(DATATABLES, "attempts"), attemptsTableInit());
     setTableStyles(html, "attempts");

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java Thu Sep 29 08:08:49 2011
@@ -36,7 +36,6 @@ 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.Apps;
 import org.apache.hadoop.yarn.webapp.WebApps;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
 import org.junit.Test;
@@ -87,6 +86,7 @@ public class TestAMWebApp {
       return jobs; // OK
     }
 
+    @SuppressWarnings("rawtypes")
     @Override
     public EventHandler getEventHandler() {
       return null;
@@ -163,6 +163,23 @@ public class TestAMWebApp {
                          new TestAppContext());
   }
 
+  @Test public void testCountersView() {
+    AppContext appContext = new TestAppContext();
+    Map<String, String> params = getJobParams(appContext);
+    WebAppTests.testPage(CountersPage.class, AppContext.class,
+                         appContext, params);
+  }
+  
+  @Test public void testSingleCounterView() {
+    AppContext appContext = new TestAppContext();
+    Map<String, String> params = getJobParams(appContext);
+    params.put(AMParams.COUNTER_GROUP, 
+        "org.apache.hadoop.mapreduce.FileSystemCounter");
+    params.put(AMParams.COUNTER_NAME, "HDFS_WRITE_OPS");
+    WebAppTests.testPage(SingleCounterPage.class, AppContext.class,
+                         appContext, params);
+  }
+  
   public static void main(String[] args) {
     WebApps.$for("yarn", AppContext.class, new TestAppContext(0, 8, 88, 4)).
         at(58888).inDevMode().start(new AMWebApp()).joinThread();

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java Thu Sep 29 08:08:49 2011
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.app.webapp.App;
 import org.apache.hadoop.mapreduce.v2.app.webapp.AppController;
@@ -57,7 +59,7 @@ public class HsController extends AppCon
    * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#countersPage()
    */
   @Override
-  protected Class<? extends View> countersPage() {
+  public Class<? extends View> countersPage() {
     return HsCountersPage.class;
   }
   
@@ -108,7 +110,16 @@ public class HsController extends AppCon
   public void jobCounters() {
     super.jobCounters();
   }
-
+  
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#taskCounters()
+   */
+  @Override
+  public void taskCounters() {
+    super.taskCounters();
+  }
+  
   /*
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#tasks()
@@ -157,4 +168,31 @@ public class HsController extends AppCon
   public void about() {
     render(aboutPage());
   }
+  
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#singleCounterPage()
+   */
+  @Override
+  protected Class<? extends View> singleCounterPage() {
+    return HsSingleCounterPage.class;
+  }
+  
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#singleJobCounter()
+   */
+  @Override
+  public void singleJobCounter() throws IOException{
+    super.singleJobCounter();
+  }
+  
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#singleTaskCounter()
+   */
+  @Override
+  public void singleTaskCounter() throws IOException{
+    super.singleTaskCounter();
+  }
 }

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java Thu Sep 29 08:08:49 2011
@@ -18,11 +18,12 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
+
 import org.apache.hadoop.mapreduce.v2.app.webapp.CountersBlock;
 import org.apache.hadoop.yarn.webapp.SubView;
 
-import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
-
 /**
  * Render the counters page
  */
@@ -34,7 +35,12 @@ public class HsCountersPage extends HsVi
    */
   @Override protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
-    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
+    String tid = $(TASK_ID);
+    String activeNav = "2";
+    if(tid == null || tid.isEmpty()) {
+      activeNav = "1";
+    }
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
     set(DATATABLES_SELECTOR, "#counters .dt-counters");
     set(initSelector(DATATABLES),
         "{bJQueryUI:true, sDom:'t', iDisplayLength:-1}");
@@ -47,9 +53,9 @@ public class HsCountersPage extends HsVi
   @Override protected void postHead(Page.HTML<_> html) {
     html.
       style("#counters, .dt-counters { table-layout: fixed }",
-            "#counters th { overflow: hidden; vertical-align: center }",
+            "#counters th { overflow: hidden; vertical-align: middle }",
             "#counters .dataTables_wrapper { min-height: 1em }",
-            "#counters .group { width: 10em }",
+            "#counters .group { width: 15em }",
             "#counters .name { width: 30em }");
   }
 

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java Thu Sep 29 08:08:49 2011
@@ -55,6 +55,14 @@ public class HsNavBlock extends HtmlBloc
           li().a(url("conf", jobid), "Configuration")._().
           li().a(url("tasks", jobid, "m"), "Map tasks")._().
           li().a(url("tasks", jobid, "r"), "Reduce tasks")._()._();
+      if (app.getTask() != null) {
+        String taskid = MRApps.toString(app.getTask().getID());
+        nav.
+          h3("Task").
+          ul().
+            li().a(url("task", taskid), "Task Overview")._().
+            li().a(url("taskcounters", taskid), "Counters")._()._();
+      }
     }
     nav.
       h3("Tools").

Added: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java?rev=1177203&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java (added)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java Thu Sep 29 08:08:49 2011
@@ -0,0 +1,69 @@
+/**
+* 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.mapreduce.v2.app.webapp.AMParams.TASK_ID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
+
+import org.apache.hadoop.mapreduce.v2.app.webapp.SingleCounterBlock;
+import org.apache.hadoop.yarn.webapp.SubView;
+
+/**
+ * Render the counters page
+ */
+public class HsSingleCounterPage extends HsView {
+
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
+   */
+  @Override protected void preHead(Page.HTML<_> html) {
+    commonPreHead(html);
+    String tid = $(TASK_ID);
+    String activeNav = "2";
+    if(tid == null || tid.isEmpty()) {
+      activeNav = "1";
+    }
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
+    set(DATATABLES_ID, "singleCounter");
+    set(initID(DATATABLES, "singleCounter"), counterTableInit());
+    setTableStyles(html, "singleCounter");
+  }
+
+  /**
+   * @return The end of a javascript map that is the jquery datatable 
+   * configuration for the jobs table.  the Jobs table is assumed to be
+   * rendered by the class returned from {@link #content()} 
+   */
+  private String counterTableInit() {
+    return tableInit().
+        append(", aoColumnDefs:[").
+        append("{'sType':'title-numeric', 'aTargets': [ 1 ] }").
+        append("]}").
+        toString();
+  }
+  
+  /**
+   * The content of this page is the CountersBlock now.
+   * @return CountersBlock.class
+   */
+  @Override protected Class<? extends SubView> content() {
+    return SingleCounterBlock.class;
+  }
+}

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java Thu Sep 29 08:08:49 2011
@@ -250,7 +250,7 @@ public class HsTaskPage extends HsView {
   @Override protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
     //override the nav config from commonPReHead
-    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:2}");
     //Set up the java script and CSS for the attempts table
     set(DATATABLES_ID, "attempts");
     set(initID(DATATABLES, "attempts"), attemptsTableInit());

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java Thu Sep 29 08:08:49 2011
@@ -41,10 +41,15 @@ public class HsWebApp extends WebApp imp
     route(pajoin("/job", JOB_ID), HsController.class, "job");
     route(pajoin("/conf", JOB_ID), HsController.class, "conf");
     route(pajoin("/jobcounters", JOB_ID), HsController.class, "jobCounters");
+    route(pajoin("/singlejobcounter",JOB_ID, COUNTER_GROUP, COUNTER_NAME),
+        HsController.class, "singleJobCounter");
     route(pajoin("/tasks", JOB_ID, TASK_TYPE), HsController.class, "tasks");
     route(pajoin("/attempts", JOB_ID, TASK_TYPE, ATTEMPT_STATE),
         HsController.class, "attempts");
     route(pajoin("/task", TASK_ID), HsController.class, "task");
+    route(pajoin("/taskcounters", TASK_ID), HsController.class, "taskCounters");
+    route(pajoin("/singletaskcounter",TASK_ID, COUNTER_GROUP, COUNTER_NAME),
+        HsController.class, "singleTaskCounter");
     route("/about", HsController.class, "about");
   }
 }

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java?rev=1177203&r1=1177202&r2=1177203&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java Thu Sep 29 08:08:49 2011
@@ -26,17 +26,13 @@ import static org.junit.Assert.assertEqu
 
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 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.webapp.AMParams;
 import org.apache.hadoop.mapreduce.v2.app.webapp.TestAMWebApp;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -92,6 +88,7 @@ public class TestHSWebApp {
       return jobs; // OK
     }
 
+    @SuppressWarnings("rawtypes")
     @Override
     public EventHandler getEventHandler() {
       return null;
@@ -171,4 +168,16 @@ public class TestHSWebApp {
     WebAppTests.testPage(HsConfPage.class, AppContext.class,
                          new TestAppContext());
   }
+  
+  @Test public void testAboutView() {
+    LOG.info("HsAboutPage");
+    WebAppTests.testPage(HsAboutPage.class, AppContext.class,
+                         new TestAppContext());
+  }
+  
+  @Test public void testSingleCounterView() {
+    LOG.info("HsSingleCounterPage");
+    WebAppTests.testPage(HsSingleCounterPage.class, AppContext.class,
+                         new TestAppContext());
+  }
 }