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/12 16:24:32 UTC

svn commit: r1169763 - 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-hs/src/main/java/or...

Author: vinodkv
Date: Mon Sep 12 14:24:32 2011
New Revision: 1169763

URL: http://svn.apache.org/viewvc?rev=1169763&view=rev
Log:
MAPREDUCE-2675. Reformat JobHistory Server main page to be more useful. Contributed by Robert Joseph Evans.

Added:
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.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/AppController.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.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/HsView.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.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=1169763&r1=1169762&r2=1169763&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt Mon Sep 12 14:24:32 2011
@@ -272,6 +272,9 @@ Release 0.23.0 - Unreleased
     MAPREDUCE-2933. Change allocate call to return ContainerStatus for
     completed containers rather than Container. (acmurthy) 
 
+    MAPREDUCE-2675. Reformat JobHistory Server main page to be more
+    useful. (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/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=1169763&r1=1169762&r2=1169763&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 Mon Sep 12 14:24:32 2011
@@ -22,6 +22,8 @@ import static org.apache.hadoop.yarn.uti
 
 import java.util.Locale;
 
+import javax.servlet.http.HttpServletResponse;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -187,7 +189,7 @@ public class AppController extends Contr
    * @param s the error message to include.
    */
   void badRequest(String s) {
-    setStatus(response().SC_BAD_REQUEST);
+    setStatus(HttpServletResponse.SC_BAD_REQUEST);
     setTitle(join("Bad request: ", s));
   }
 
@@ -196,7 +198,7 @@ public class AppController extends Contr
    * @param s the error message to include.
    */
   void notFound(String s) {
-    setStatus(response().SC_NOT_FOUND);
+    setStatus(HttpServletResponse.SC_NOT_FOUND);
     setTitle(join("Not found: ", s));
   }
 

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.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/JobHistory.java?rev=1169763&r1=1169762&r2=1169763&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java Mon Sep 12 14:24:32 2011
@@ -224,7 +224,7 @@ public class JobHistory extends Abstract
             DEFAULT_MOVE_THREAD_INTERVAL);
     numMoveThreads = conf.getInt(JHAdminConfig.MR_HISTORY_MOVE_THREAD_COUNT, DEFAULT_MOVE_THREAD_COUNT);
     try {
-    initExisting();
+      initExisting();
     } catch (IOException e) {
       throw new YarnException("Failed to intialize existing directories", e);
     }
@@ -319,6 +319,7 @@ public class JobHistory extends Abstract
    */
   @SuppressWarnings("unchecked")
   private void initExisting() throws IOException {
+    LOG.info("Initializing Existing Jobs...");
     List<FileStatus> timestampedDirList = findTimestampedDirectories();
     Collections.sort(timestampedDirList);
     for (FileStatus fs : timestampedDirList) {
@@ -350,6 +351,9 @@ public class JobHistory extends Abstract
   }
   
   private void addDirectoryToSerialNumberIndex(Path serialDirPath) {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Adding "+serialDirPath+" to serial index");
+    }
     String serialPart = serialDirPath.getName();
     String timestampPart = JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
     if (timestampPart == null) {
@@ -374,9 +378,15 @@ public class JobHistory extends Abstract
   }
   
   private void addDirectoryToJobListCache(Path path) throws IOException {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Adding "+path+" to job list cache.");
+    }
     List<FileStatus> historyFileList = scanDirectoryForHistoryFiles(path,
         doneDirFc);
     for (FileStatus fs : historyFileList) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Adding in history for "+fs.getPath());
+      }
       JobIndexInfo jobIndexInfo = FileNameIndexUtils.getIndexInfo(fs.getPath()
           .getName());
       String confFileName = JobHistoryUtils
@@ -423,6 +433,9 @@ public class JobHistory extends Abstract
    * Adds an entry to the job list cache. Maintains the size.
    */
   private void addToJobListCache(JobId jobId, MetaInfo metaInfo) {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Adding "+jobId+" to job list cache with "+metaInfo.getJobIndexInfo());
+    }
     jobListCache.put(jobId, metaInfo);
     if (jobListCache.size() > jobListCacheSize) {
       jobListCache.remove(jobListCache.firstKey());
@@ -432,7 +445,10 @@ public class JobHistory extends Abstract
   /**
    * Adds an entry to the loaded job cache. Maintains the size.
    */
-  private void  addToLoadedJobCache(Job job) {
+  private void addToLoadedJobCache(Job job) {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Adding "+job.getID()+" to loaded job cache");
+    }
     loadedJobCache.put(job.getID(), job);
     if (loadedJobCache.size() > loadedJobCacheSize ) {
       loadedJobCache.remove(loadedJobCache.firstKey());
@@ -967,6 +983,9 @@ public class JobHistory extends Abstract
 
   @Override
   public synchronized Job getJob(JobId jobId) {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Looking for Job "+jobId);
+    }
     Job job = null;
     try {
       job = findJob(jobId);
@@ -979,7 +998,9 @@ public class JobHistory extends Abstract
 
   @Override
   public Map<JobId, Job> getAllJobs(ApplicationId appID) {
-    LOG.info("Called getAllJobs(AppId): " + appID);
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Called getAllJobs(AppId): " + appID);
+    }
 //    currently there is 1 to 1 mapping between app and job id
     org.apache.hadoop.mapreduce.JobID oldJobID = TypeConverter.fromYarn(appID);
     Map<JobId, Job> jobs = new HashMap<JobId, Job>();
@@ -1002,12 +1023,9 @@ public class JobHistory extends Abstract
    * This does involve a DFS oepration of scanning the intermediate directory.
    */
   public Map<JobId, Job> getAllJobs() {
+    LOG.debug("Called getAllJobs()");
     return getAllJobsInternal();
-        }
-
-  
-  
-  
+  }
   
   static class MetaInfo {
     private Path historyFile;

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.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/PartialJob.java?rev=1169763&r1=1169762&r2=1169763&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java Mon Sep 12 14:24:32 2011
@@ -46,6 +46,8 @@ public class PartialJob implements org.a
     this.jobIndexInfo = jobIndexInfo;
     this.jobId = jobId;
     jobReport = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(JobReport.class);
+    jobReport.setStartTime(jobIndexInfo.getSubmitTime());
+    jobReport.setFinishTime(jobIndexInfo.getFinishTime());
   }
   
   @Override
@@ -142,7 +144,7 @@ public class PartialJob implements org.a
   
   @Override
   public String getUserName() {
-    return null;
+    return jobIndexInfo.getUser();
   }
 
 }

Added: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.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/HsJobBlock.java?rev=1169763&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/HsJobBlock.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/HsJobBlock.java Mon Sep 12 14:24:32 2011
@@ -0,0 +1,266 @@
+/**
+* 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 com.google.inject.Inject;
+import java.util.Date;
+import java.util.Map;
+
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
+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.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.InfoBlock;
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp.*;
+import static org.apache.hadoop.yarn.util.StringHelper.*;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
+
+/**
+ * Render a block of HTML for a give job.
+ */
+public class HsJobBlock extends HtmlBlock {
+  final AppContext appContext;
+
+  int runningMapTasks = 0;
+  int pendingMapTasks = 0;
+  int runningReduceTasks = 0;
+  int pendingReduceTasks = 0;
+
+  int newMapAttempts = 0;
+  int runningMapAttempts = 0;
+  int killedMapAttempts = 0;
+  int failedMapAttempts = 0;
+  int successfulMapAttempts = 0;
+  int newReduceAttempts = 0;
+  int runningReduceAttempts = 0;
+  int killedReduceAttempts = 0;
+  int failedReduceAttempts = 0;
+  int successfulReduceAttempts = 0;
+
+  @Inject HsJobBlock(AppContext appctx) {
+    appContext = appctx;
+  }
+
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.yarn.webapp.view.HtmlBlock#render(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block)
+   */
+  @Override protected void render(Block html) {
+    String jid = $(JOB_ID);
+    if (jid.isEmpty()) {
+      html.
+        p()._("Sorry, can't do anything without a JobID.")._();
+      return;
+    }
+    JobId jobID = MRApps.toJobID(jid);
+    Job job = appContext.getJob(jobID);
+    if (job == null) {
+      html.
+        p()._("Sorry, ", jid, " not found.")._();
+      return;
+    }
+    JobReport jobReport = job.getReport();
+    String mapPct = percent(jobReport.getMapProgress());
+    String reducePct = percent(jobReport.getReduceProgress());
+    int mapTasks = job.getTotalMaps();
+    int mapTasksComplete = job.getCompletedMaps();
+    int reduceTasks = job.getTotalReduces();
+    int reducesTasksComplete = job.getCompletedReduces();
+    long startTime = jobReport.getStartTime();
+    long finishTime = jobReport.getFinishTime();
+    countTasksAndAttempts(job);
+    info("Job Overview").
+        _("Job Name:", job.getName()).
+        _("State:", job.getState()).
+        _("Uberized:", job.isUber()).
+        _("Started:", new Date(startTime)).
+        _("Elapsed:", StringUtils.formatTime(
+            Times.elapsed(startTime, finishTime)));
+    html.
+      _(InfoBlock.class).
+      div(_INFO_WRAP).
+
+      // Tasks table
+        table("#job").
+          tr().
+            th(_TH, "Task Type").
+            th(_TH, "Progress").
+            th(_TH, "Total").
+            th(_TH, "Pending").
+            th(_TH, "Running").
+            th(_TH, "Complete")._().
+          tr(_ODD).
+            th().
+              a(url("tasks", jid, "m"), "Map")._().
+            td().
+              div(_PROGRESSBAR).
+                $title(join(mapPct, '%')). // tooltip
+                div(_PROGRESSBAR_VALUE).
+                  $style(join("width:", mapPct, '%'))._()._()._().
+            td(String.valueOf(mapTasks)).
+            td(String.valueOf(pendingMapTasks)).
+            td(String.valueOf(runningMapTasks)).
+            td(String.valueOf(mapTasksComplete))._().
+          tr(_EVEN).
+            th().
+              a(url("tasks", jid, "r"), "Reduce")._().
+            td().
+              div(_PROGRESSBAR).
+                $title(join(reducePct, '%')). // tooltip
+                div(_PROGRESSBAR_VALUE).
+                  $style(join("width:", reducePct, '%'))._()._()._().
+            td(String.valueOf(reduceTasks)).
+            td(String.valueOf(pendingReduceTasks)).
+            td(String.valueOf(runningReduceTasks)).
+            td(String.valueOf(reducesTasksComplete))._()
+          ._().
+
+        // Attempts table
+        table("#job").
+        tr().
+          th(_TH, "Attempt Type").
+          th(_TH, "New").
+          th(_TH, "Running").
+          th(_TH, "Failed").
+          th(_TH, "Killed").
+          th(_TH, "Successful")._().
+        tr(_ODD).
+          th("Maps").
+          td().a(url("attempts", jid, "m",
+              TaskAttemptStateUI.NEW.toString()), 
+              String.valueOf(newMapAttempts))._().
+          td().a(url("attempts", jid, "m",
+              TaskAttemptStateUI.RUNNING.toString()), 
+              String.valueOf(runningMapAttempts))._().
+          td().a(url("attempts", jid, "m",
+              TaskAttemptStateUI.FAILED.toString()), 
+              String.valueOf(failedMapAttempts))._().
+          td().a(url("attempts", jid, "m",
+              TaskAttemptStateUI.KILLED.toString()), 
+              String.valueOf(killedMapAttempts))._().
+          td().a(url("attempts", jid, "m",
+              TaskAttemptStateUI.SUCCESSFUL.toString()), 
+              String.valueOf(successfulMapAttempts))._().
+        _().
+        tr(_EVEN).
+          th("Reduces").
+          td().a(url("attempts", jid, "r",
+              TaskAttemptStateUI.NEW.toString()), 
+              String.valueOf(newReduceAttempts))._().
+          td().a(url("attempts", jid, "r",
+              TaskAttemptStateUI.RUNNING.toString()), 
+              String.valueOf(runningReduceAttempts))._().
+          td().a(url("attempts", jid, "r",
+              TaskAttemptStateUI.FAILED.toString()), 
+              String.valueOf(failedReduceAttempts))._().
+          td().a(url("attempts", jid, "r",
+              TaskAttemptStateUI.KILLED.toString()), 
+              String.valueOf(killedReduceAttempts))._().
+          td().a(url("attempts", jid, "r",
+              TaskAttemptStateUI.SUCCESSFUL.toString()), 
+              String.valueOf(successfulReduceAttempts))._().
+         _().
+       _().
+     _();
+  }
+
+  /**
+   * 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) {
+    Map<TaskId, Task> tasks = job.getTasks();
+    for (Task task : tasks.values()) {
+      switch (task.getType()) {
+      case MAP:
+        // Task counts
+        switch (task.getState()) {
+        case RUNNING:
+          ++runningMapTasks;
+          break;
+        case SCHEDULED:
+          ++pendingMapTasks;
+          break;
+        }
+        break;
+      case REDUCE:
+        // Task counts
+        switch (task.getState()) {
+        case RUNNING:
+          ++runningReduceTasks;
+          break;
+        case SCHEDULED:
+          ++pendingReduceTasks;
+          break;
+        }
+        break;
+      }
+
+      // Attempts counts
+      Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
+      for (TaskAttempt attempt : attempts.values()) {
+
+        int newAttempts = 0, running = 0, successful = 0, failed = 0, killed =0;
+
+        if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
+          ++newAttempts;
+        } else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt
+            .getState())) {
+          ++running;
+        } else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
+            .getState())) {
+          ++successful;
+        } else if (TaskAttemptStateUI.FAILED
+            .correspondsTo(attempt.getState())) {
+          ++failed;
+        } else if (TaskAttemptStateUI.KILLED
+            .correspondsTo(attempt.getState())) {
+          ++killed;
+        }
+
+        switch (task.getType()) {
+        case MAP:
+          newMapAttempts += newAttempts;
+          runningMapAttempts += running;
+          successfulMapAttempts += successful;
+          failedMapAttempts += failed;
+          killedMapAttempts += killed;
+          break;
+        case REDUCE:
+          newReduceAttempts += newAttempts;
+          runningReduceAttempts += running;
+          successfulReduceAttempts += successful;
+          failedReduceAttempts += failed;
+          killedReduceAttempts += killed;
+          break;
+        }
+      }
+    }
+  }
+}

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.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/HsJobPage.java?rev=1169763&r1=1169762&r2=1169763&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/HsJobPage.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/HsJobPage.java Mon Sep 12 14:24:32 2011
@@ -23,7 +23,6 @@ import static org.apache.hadoop.yarn.uti
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
 
-import org.apache.hadoop.mapreduce.v2.app.webapp.JobBlock;
 import org.apache.hadoop.yarn.webapp.SubView;
 
 /**
@@ -46,9 +45,9 @@ public class HsJobPage extends HsView {
 
   /**
    * The content of this page is the JobBlock
-   * @return JobBlock.class
+   * @return HsJobBlock.class
    */
   @Override protected Class<? extends SubView> content() {
-    return JobBlock.class;
+    return HsJobBlock.class;
   }
 }

Added: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.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/HsJobsBlock.java?rev=1169763&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/HsJobsBlock.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/HsJobsBlock.java Mon Sep 12 14:24:32 2011
@@ -0,0 +1,109 @@
+/**
+* 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 java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+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.HamletSpec.InputType;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+
+import com.google.inject.Inject;
+
+/**
+ * Render all of the jobs that the history server is aware of.
+ */
+public class HsJobsBlock extends HtmlBlock {
+  final AppContext appContext;
+  static final SimpleDateFormat dateFormat = 
+    new SimpleDateFormat("yyyy.MM.dd HH:mm:ss z"); 
+
+  @Inject HsJobsBlock(AppContext appCtx) {
+    appContext = appCtx;
+  }
+
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.yarn.webapp.view.HtmlBlock#render(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block)
+   */
+  @Override protected void render(Block html) {
+    TBODY<TABLE<Hamlet>> tbody = html.
+      h2("Retired Jobs").
+      table("#jobs").
+        thead().
+          tr().
+            th("Start Time").
+            th("Finish Time").
+            th(".id", "Job ID").
+            th(".name", "Name").
+            th("User").
+            th(".state", "State").
+            th("Maps Total").
+            th("Maps Completed").
+            th("Reduces Total").
+            th("Reduces Completed")._()._().
+        tbody();
+    LOG.info("Getting list of all Jobs.");
+    for (Job job : appContext.getAllJobs().values()) {
+      String jobID = MRApps.toString(job.getID());
+      JobReport report = job.getReport();
+      String mapsTotal = String.valueOf(job.getTotalMaps());
+      String mapsCompleted = String.valueOf(job.getCompletedMaps());
+      String reduceTotal = String.valueOf(job.getTotalReduces());
+      String reduceCompleted = String.valueOf(job.getCompletedReduces());
+      long startTime = report.getStartTime();
+      long finishTime = report.getFinishTime();
+      tbody.
+        tr().
+          td(dateFormat.format(new Date(startTime))).
+          td(dateFormat.format(new Date(finishTime))).
+          td().a(url("job", jobID), jobID)._().
+          td(job.getName().toString()).
+          td(job.getUserName()).
+          td(job.getState().toString()).
+          td(mapsTotal).
+          td(mapsCompleted).
+          td(reduceTotal).
+          td(reduceCompleted)._();
+    }
+    tbody._().
+    tfoot().
+      tr().
+        th().input("search_init").$type(InputType.text).$name("start_time").$value("Start Time")._()._().
+        th().input("search_init").$type(InputType.text).$name("finish_time").$value("Finish Time")._()._().
+        th().input("search_init").$type(InputType.text).$name("start_time").$value("Job ID")._()._().
+        th().input("search_init").$type(InputType.text).$name("start_time").$value("Name")._()._().
+        th().input("search_init").$type(InputType.text).$name("start_time").$value("User")._()._().
+        th().input("search_init").$type(InputType.text).$name("start_time").$value("State")._()._().
+        th().input("search_init").$type(InputType.text).$name("start_time").$value("Maps Total")._()._().
+        th().input("search_init").$type(InputType.text).$name("start_time").$value("Maps Completed")._()._().
+        th().input("search_init").$type(InputType.text).$name("start_time").$value("Reduces Total")._()._().
+        th().input("search_init").$type(InputType.text).$name("start_time").$value("Reduces Completed")._()._().
+        _().
+      _().
+    _();
+  }
+}

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=1169763&r1=1169762&r2=1169763&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 Mon Sep 12 14:24:32 2011
@@ -43,7 +43,7 @@ public class HsNavBlock extends HtmlBloc
       div("#nav").
       h3("Application").
         ul().
-          li().a("about", "About")._().
+          li().a(url("about"), "About")._().
           li().a(url("app"), "Jobs")._()._();
     if (app.getJob() != null) {
       String jobid = MRApps.toString(app.getJob().getID());

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.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/HsView.java?rev=1169763&r1=1169762&r2=1169763&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/HsView.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/HsView.java Mon Sep 12 14:24:32 2011
@@ -18,11 +18,18 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
-import org.apache.hadoop.mapreduce.v2.app.webapp.JobsBlock;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.THEMESWITCHER_ID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.postInitID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
+
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout;
 
-import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 
 /**
  * A view that should be used as the base class for all history server pages.
@@ -36,6 +43,7 @@ public class HsView extends TwoColumnLay
     commonPreHead(html);
     set(DATATABLES_ID, "jobs");
     set(initID(DATATABLES, "jobs"), jobsTableInit());
+    set(postInitID(DATATABLES, "jobs"), jobsPostTableInit());
     setTableStyles(html, "jobs");
   }
 
@@ -64,7 +72,7 @@ public class HsView extends TwoColumnLay
    */
   @Override
   protected Class<? extends SubView> content() {
-    return JobsBlock.class;
+    return HsJobsBlock.class;
   }
   
   //TODO We need a way to move all of the javascript/CSS that is for a subview
@@ -76,9 +84,40 @@ public class HsView extends TwoColumnLay
    */
   private String jobsTableInit() {
     return tableInit().
-        append(",aoColumns:[{sType:'title-numeric'},").
-        append("null,null,{sType:'title-numeric', bSearchable:false},null,").
-        append("null,{sType:'title-numeric',bSearchable:false}, null, null]}").
+        append(",aoColumnDefs:[").
+        append("{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 6 ] }").
+        append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 7 ] }").
+        append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 8 ] }").
+        append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 9 ] }").
+        append("]}").
         toString();
   }
+  
+  /**
+   * @return javascript to add into the jquery block after the table has
+   *  been initialized. This code adds in per field filtering.
+   */
+  private String jobsPostTableInit() {
+    return "var asInitVals = new Array();\n" +
+    		   "$('tfoot input').keyup( function () \n{"+
+           "  jobsDataTable.fnFilter( this.value, $('tfoot input').index(this) );\n"+
+           "} );\n"+
+           "$('tfoot input').each( function (i) {\n"+
+           "  asInitVals[i] = this.value;\n"+
+           "} );\n"+
+           "$('tfoot input').focus( function () {\n"+
+           "  if ( this.className == 'search_init' )\n"+
+           "  {\n"+
+           "    this.className = '';\n"+
+           "    this.value = '';\n"+
+           "  }\n"+
+           "} );\n"+
+           "$('tfoot input').blur( function (i) {\n"+
+           "  if ( this.value == '' )\n"+
+           "  {\n"+
+           "    this.className = 'search_init';\n"+
+           "    this.value = asInitVals[$('tfoot input').index(this)];\n"+
+           "  }\n"+
+           "} );\n";
+  }
 }

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java?rev=1169763&r1=1169762&r2=1169763&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java Mon Sep 12 14:24:32 2011
@@ -133,8 +133,12 @@ public class JQueryUI extends HtmlBlock 
         if (init.isEmpty()) {
           init = defaultInit;
         }
-        list.add(join("  $('#", id, "').dataTable(", init,
+        list.add(join(id,"DataTable =  $('#", id, "').dataTable(", init,
                       ").fnSetFilteringDelay(188);"));
+        String postInit = $(postInitID(DATATABLES, id));
+        if(!postInit.isEmpty()) {
+          list.add(postInit);
+        }
       }
     }
     String selector = $(DATATABLES_SELECTOR);
@@ -210,6 +214,10 @@ public class JQueryUI extends HtmlBlock 
   public static String initID(String name, String id) {
     return djoin(name, id, "init");
   }
+  
+  public static String postInitID(String name, String id) {
+    return djoin(name, id, "postinit");
+  }
 
   public static String initSelector(String name) {
     return djoin(name, "selector.init");