Pārlūkot izejas kodu

commit cadb769b4fb37e0fabf3947f9bcf18e4c7bd0516
Author: Devaraj Das <ddas@yahoo-inc.com>
Date: Fri Jul 23 10:46:46 2010 -0700

MAPREDUCE:291 from

+++ b/YAHOO-CHANGES.txt
+ MAPREDUCE-291. Optionally a separate daemon should serve JobHistory.
+ (Srikanth Sundarrajan via ddas)
+


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1077590 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 gadi atpakaļ
vecāks
revīzija
dbec15175c

+ 4 - 0
bin/hadoop

@@ -74,6 +74,7 @@ if [ $# = 0 ]; then
   echo "  jobtracker           run the MapReduce job Tracker node" 
   echo "  jobtracker           run the MapReduce job Tracker node" 
   echo "  pipes                run a Pipes job"
   echo "  pipes                run a Pipes job"
   echo "  tasktracker          run a MapReduce task Tracker node" 
   echo "  tasktracker          run a MapReduce task Tracker node" 
+  echo "  historyserver        run job history servers as a standalone daemon"
   echo "  job                  manipulate MapReduce jobs"
   echo "  job                  manipulate MapReduce jobs"
   echo "  queue                get information regarding JobQueues" 
   echo "  queue                get information regarding JobQueues" 
   echo "  version              print the version"
   echo "  version              print the version"
@@ -244,6 +245,9 @@ elif [ "$COMMAND" = "fetchdt" ] ; then
 elif [ "$COMMAND" = "jobtracker" ] ; then
 elif [ "$COMMAND" = "jobtracker" ] ; then
   CLASS=org.apache.hadoop.mapred.JobTracker
   CLASS=org.apache.hadoop.mapred.JobTracker
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_JOBTRACKER_OPTS"
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_JOBTRACKER_OPTS"
+elif [ "$COMMAND" = "historyserver" ] ; then
+  CLASS=org.apache.hadoop.mapred.JobHistoryServer
+  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_JOB_HISTORYSERVER_OPTS"
 elif [ "$COMMAND" = "tasktracker" ] ; then
 elif [ "$COMMAND" = "tasktracker" ] ; then
   CLASS=org.apache.hadoop.mapred.TaskTracker
   CLASS=org.apache.hadoop.mapred.TaskTracker
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_TASKTRACKER_OPTS"
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_TASKTRACKER_OPTS"

+ 27 - 0
bin/start-jobhistoryserver.sh

@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+
+# 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.
+
+
+# Start hadoop job history daemons.  Run this on node where history server need to run
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/hadoop-config.sh
+
+# start daemon
+"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR start historyserver

+ 27 - 0
bin/stop-jobhistoryserver.sh

@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+
+# 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.
+
+
+# Stop hadoop job history daemon.  Run this on the node where history server is running
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/hadoop-config.sh
+
+"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop historyserver
+

+ 8 - 0
build.xml

@@ -325,6 +325,7 @@
     <mkdir dir="${build.src}"/>
     <mkdir dir="${build.src}"/>
     <mkdir dir="${build.webapps}/task/WEB-INF"/>
     <mkdir dir="${build.webapps}/task/WEB-INF"/>
     <mkdir dir="${build.webapps}/job/WEB-INF"/>
     <mkdir dir="${build.webapps}/job/WEB-INF"/>
+    <mkdir dir="${build.webapps}/history/WEB-INF"/>
     <mkdir dir="${build.webapps}/hdfs/WEB-INF"/>
     <mkdir dir="${build.webapps}/hdfs/WEB-INF"/>
     <mkdir dir="${build.webapps}/datanode/WEB-INF"/>
     <mkdir dir="${build.webapps}/datanode/WEB-INF"/>
     <mkdir dir="${build.webapps}/secondary/WEB-INF"/>
     <mkdir dir="${build.webapps}/secondary/WEB-INF"/>
@@ -442,6 +443,13 @@
      webxml="${build.webapps}/job/WEB-INF/web.xml">
      webxml="${build.webapps}/job/WEB-INF/web.xml">
     </jsp-compile>
     </jsp-compile>
 
 
+    <jsp-compile
+     uriroot="${src.webapps}/history"
+     outputdir="${build.src}"
+     package="org.apache.hadoop.mapred"
+     webxml="${build.webapps}/history/WEB-INF/web.xml">
+    </jsp-compile>
+
     <!-- Compile Java files (excluding JSPs) checking warnings -->
     <!-- Compile Java files (excluding JSPs) checking warnings -->
     <javac 
     <javac 
      encoding="${build.encoding}" 
      encoding="${build.encoding}" 

+ 42 - 1
src/core/org/apache/hadoop/http/HttpServer.java

@@ -267,6 +267,36 @@ public class HttpServer implements FilterContainer {
     defaultContexts.put(ctxt, isFiltered);
     defaultContexts.put(ctxt, isFiltered);
   }
   }
 
 
+  public WebAppContext addContext(String name, boolean isFiltered)
+      throws IOException {
+    if (0 == webServer.getHandlers().length) {
+      throw new RuntimeException("Couldn't find handler");
+    }
+    WebAppContext webAppCtx = new WebAppContext();
+    webAppCtx.setDisplayName("AppContext-" + name);
+    webAppCtx.setContextPath("/" + name);
+    webAppCtx.setWar(getWebAppsPath() + "/" + name);
+    setContextAttributes(webAppCtx);
+    addContext(webAppCtx, isFiltered);
+
+    if (isFiltered) {
+      defineFilter(webAppCtx, "krb5Filter",
+          Krb5AndCertsSslSocketConnector.Krb5SslFilter.class.getName(),
+          null, null);
+
+      defineFilter(webAppCtx, "safety", QuotingInputFilter.class.getName(), null,
+          new String[] {"/*"});
+
+      final FilterInitializer[] initializers = getFilterInitializers(conf);
+      if (initializers != null) {
+        for(FilterInitializer c : initializers) {
+          c.initFilter(this, conf);
+        }
+      }
+    }
+    return webAppCtx;
+  }
+
   /**
   /**
    * Add a context 
    * Add a context 
    * @param pathSpec The path spec for the context
    * @param pathSpec The path spec for the context
@@ -291,7 +321,18 @@ public class HttpServer implements FilterContainer {
    * @param value The value of the attribute
    * @param value The value of the attribute
    */
    */
   public void setAttribute(String name, Object value) {
   public void setAttribute(String name, Object value) {
-    webAppContext.setAttribute(name, value);
+    setAttribute(webAppContext, name, value);
+  }
+
+  /**
+   * Set a value in the webapp context. These values are available to the jsp
+   * pages as "application.getAttribute(name)".
+   * @param context Context to add attribute
+   * @param name The name of the attribute
+   * @param value The value of the attribute
+   */
+  public void setAttribute(Context context, String name, Object value) {
+    context.setAttribute(name, value);
   }
   }
 
 
   /**
   /**

+ 41 - 22
src/mapred/org/apache/hadoop/mapred/JSPUtil.java

@@ -402,7 +402,8 @@ class JSPUtil {
             "<td id=\"job_" + rowId + "\">" + 
             "<td id=\"job_" + rowId + "\">" + 
             
             
               (historyFileUrl == null ? "" :
               (historyFileUrl == null ? "" :
-              "<a href=\"jobdetailshistory.jsp?logFile=" + historyFileUrl + "\">") + 
+              "<a href=\"" + JobHistoryServer.getHistoryUrlPrefix(tracker.conf) +
+                  "/jobdetailshistory.jsp?logFile=" + historyFileUrl + "\">") +
               
               
               info.status.getJobId() + "</a></td>" +
               info.status.getJobId() + "</a></td>" +
             
             
@@ -446,12 +447,11 @@ class JSPUtil {
    * 
    * 
    * @param logFile
    * @param logFile
    * @param fs
    * @param fs
-   * @param jobTracker
    * @return JobInfo
    * @return JobInfo
    * @throws IOException
    * @throws IOException
    */
    */
   static JobInfo getJobInfo(Path logFile, FileSystem fs,
   static JobInfo getJobInfo(Path logFile, FileSystem fs,
-      JobTracker jobTracker, String user) throws IOException {
+      JobConf jobConf, ACLsManager acLsManager, String user) throws IOException {
     String jobid = getJobID(logFile.getName());
     String jobid = getJobID(logFile.getName());
     JobInfo jobInfo = null;
     JobInfo jobInfo = null;
     synchronized(jobHistoryCache) {
     synchronized(jobHistoryCache) {
@@ -465,7 +465,7 @@ class JSPUtil {
       }
       }
       jobHistoryCache.put(jobid, jobInfo);
       jobHistoryCache.put(jobid, jobInfo);
       int CACHE_SIZE = 
       int CACHE_SIZE = 
-        jobTracker.conf.getInt("mapred.job.tracker.jobhistory.lru.cache.size", 5);
+        jobConf.getInt("mapred.job.tracker.jobhistory.lru.cache.size", 5);
       if (jobHistoryCache.size() > CACHE_SIZE) {
       if (jobHistoryCache.size() > CACHE_SIZE) {
         Iterator<Map.Entry<String, JobInfo>> it = 
         Iterator<Map.Entry<String, JobInfo>> it = 
           jobHistoryCache.entrySet().iterator();
           jobHistoryCache.entrySet().iterator();
@@ -483,7 +483,7 @@ class JSPUtil {
     }
     }
 
 
     // Authorize the user for view access of this job
     // Authorize the user for view access of this job
-    jobTracker.getACLsManager().checkAccess(jobid, currentUser,
+    acLsManager.checkAccess(jobid, currentUser,
         jobInfo.getJobQueue(), Operation.VIEW_JOB_DETAILS,
         jobInfo.getJobQueue(), Operation.VIEW_JOB_DETAILS,
         jobInfo.get(Keys.USER), jobInfo.getJobACLs().get(JobACL.VIEW_JOB));
         jobInfo.get(Keys.USER), jobInfo.getJobACLs().get(JobACL.VIEW_JOB));
 
 
@@ -495,7 +495,6 @@ class JSPUtil {
    * 
    * 
    * @param request
    * @param request
    * @param response
    * @param response
-   * @param jobTracker
    * @param fs
    * @param fs
    * @param logFile
    * @param logFile
    * @return the job if authorization is disabled or if the authorization checks
    * @return the job if authorization is disabled or if the authorization checks
@@ -505,29 +504,32 @@ class JSPUtil {
    * @throws ServletException
    * @throws ServletException
    */
    */
   static JobInfo checkAccessAndGetJobInfo(HttpServletRequest request,
   static JobInfo checkAccessAndGetJobInfo(HttpServletRequest request,
-      HttpServletResponse response, final JobTracker jobTracker,
-      final FileSystem fs, final Path logFile) throws IOException,
+      HttpServletResponse response, final JobConf jobConf,
+      final ACLsManager acLsManager, final FileSystem fs,
+      final Path logFile) throws IOException,
       InterruptedException, ServletException {
       InterruptedException, ServletException {
     String jobid = getJobID(logFile.getName());
     String jobid = getJobID(logFile.getName());
     String user = request.getRemoteUser();
     String user = request.getRemoteUser();
     JobInfo job = null;
     JobInfo job = null;
     if (user != null) {
     if (user != null) {
       try {
       try {
-        job = JSPUtil.getJobInfo(logFile, fs, jobTracker, user);
+        job = JSPUtil.getJobInfo(logFile, fs, jobConf, acLsManager, user);
       } catch (AccessControlException e) {
       } catch (AccessControlException e) {
+        String trackerAddress = jobConf.get("mapred.job.tracker.http.address");
         String errMsg =
         String errMsg =
             String.format(
             String.format(
                 "User %s failed to view %s!<br><br>%s"
                 "User %s failed to view %s!<br><br>%s"
                     + "<hr>"
                     + "<hr>"
                     + "<a href=\"jobhistory.jsp\">Go back to JobHistory</a><br>"
                     + "<a href=\"jobhistory.jsp\">Go back to JobHistory</a><br>"
-                    + "<a href=\"jobtracker.jsp\">Go back to JobTracker</a>",
+                    + "<a href=\"http://" + trackerAddress +
+                    "/jobtracker.jsp\">Go back to JobTracker</a>",
                 user, jobid, e.getMessage());
                 user, jobid, e.getMessage());
         JSPUtil.setErrorAndForward(errMsg, request, response);
         JSPUtil.setErrorAndForward(errMsg, request, response);
         return null;
         return null;
       }
       }
     } else {
     } else {
       // no authorization needed
       // no authorization needed
-      job = JSPUtil.getJobInfo(logFile, fs, jobTracker, null);
+      job = JSPUtil.getJobInfo(logFile, fs, jobConf, acLsManager, null);
     }
     }
     return job;
     return job;
   }
   }
@@ -555,17 +557,7 @@ class JSPUtil {
       Map<JobACL, AccessControlList> jobAcls, JspWriter out)
       Map<JobACL, AccessControlList> jobAcls, JspWriter out)
       throws IOException {
       throws IOException {
     if (tracker.areACLsEnabled()) {
     if (tracker.areACLsEnabled()) {
-      // Display job-view-acls and job-modify-acls configured for this job
-      out.print("<b>Job-ACLs:</b><br>");
-      for (JobACL aclName : JobACL.values()) {
-        String aclConfigName = aclName.getAclName();
-        AccessControlList aclConfigured = jobAcls.get(aclName);
-        if (aclConfigured != null) {
-          String aclStr = aclConfigured.toString();
-          out.print("&nbsp;&nbsp;&nbsp;&nbsp;" + aclConfigName + ": "
-              + aclStr + "<br>");
-        }
-      }
+      printJobACLsInternal(jobAcls, out);
     }
     }
     else {
     else {
       out.print("<b>Job-ACLs: " + new AccessControlList("*").toString()
       out.print("<b>Job-ACLs: " + new AccessControlList("*").toString()
@@ -573,6 +565,33 @@ class JSPUtil {
     }
     }
   }
   }
 
 
+  static void printJobACLs(JobConf conf,
+      Map<JobACL, AccessControlList> jobAcls, JspWriter out)
+      throws IOException {
+    if (conf.getBoolean(JobConf.MR_ACLS_ENABLED, false)) {
+      printJobACLsInternal(jobAcls, out);
+    }
+    else {
+      out.print("<b>Job-ACLs: " + new AccessControlList("*").toString()
+          + "</b><br>");
+    }
+  }
+
+  private static void printJobACLsInternal(Map<JobACL, AccessControlList> jobAcls,
+                                           JspWriter out) throws IOException {
+    // Display job-view-acls and job-modify-acls configured for this job
+    out.print("<b>Job-ACLs:</b><br>");
+    for (JobACL aclName : JobACL.values()) {
+      String aclConfigName = aclName.getAclName();
+      AccessControlList aclConfigured = jobAcls.get(aclName);
+      if (aclConfigured != null) {
+        String aclStr = aclConfigured.toString();
+        out.print("&nbsp;&nbsp;&nbsp;&nbsp;" + aclConfigName + ": "
+            + aclStr + "<br>");
+      }
+    }
+  }
+
   static boolean privateActionsAllowed(JobConf conf) {
   static boolean privateActionsAllowed(JobConf conf) {
     return conf.getBoolean(PRIVATE_ACTIONS_KEY, false);
     return conf.getBoolean(PRIVATE_ACTIONS_KEY, false);
   }
   }

+ 10 - 0
src/mapred/org/apache/hadoop/mapred/JobHistory.java

@@ -518,6 +518,12 @@ public class JobHistory {
   }
   }
 
 
   static void initDone(JobConf conf, FileSystem fs) throws IOException {
   static void initDone(JobConf conf, FileSystem fs) throws IOException {
+    initDone(conf, fs, true);
+  }
+
+  static void initDone(JobConf conf, FileSystem fs,
+                                     boolean setup)
+      throws IOException {
     //if completed job history location is set, use that
     //if completed job history location is set, use that
     String doneLocation = conf.
     String doneLocation = conf.
                      get("mapred.job.tracker.history.completed.location");
                      get("mapred.job.tracker.history.completed.location");
@@ -529,6 +535,10 @@ public class JobHistory {
       DONEDIR_FS = LOGDIR_FS;
       DONEDIR_FS = LOGDIR_FS;
     }
     }
 
 
+    if (!setup) {
+        return;
+    }
+
     //If not already present create the done folder with appropriate 
     //If not already present create the done folder with appropriate 
     //permission
     //permission
     if (!DONEDIR_FS.exists(DONE)) {
     if (!DONEDIR_FS.exists(DONE)) {

+ 257 - 0
src/mapred/org/apache/hadoop/mapred/JobHistoryServer.java

@@ -0,0 +1,257 @@
+/**
+ * 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.mapred;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.mortbay.jetty.webapp.WebAppContext;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.security.PrivilegedExceptionAction;
+
+/******************************************************************
+ * {@link JobHistoryServer} is responsible for servicing all job history
+ * related requests from client.
+ *
+ * History Server can be initialized in one of two modes
+ *   * Emdedded within {@link JobTracker}
+ *   * External daemon, can either be run on the job tracker node or outside
+ *
+ * Two {@link Configuration} entries in mapred-site.xml govern the functioning
+ * of the history server
+ *
+ * mapred.job.history.server.http.address is address to which history web
+ *  server is bound to. If operating in embedded mode, the hostname on
+ *  history address has to be same as the job tracker host name
+ *
+ * mapred.job.history.server.embedded (default is true) will cause job tracker
+ *  to init history server, else the server need to be started as a
+ *  separate daemon process
+ *****************************************************************/
+public class JobHistoryServer {
+  private static final Log LOG = LogFactory.getLog(JobHistoryServer.class);
+
+  static{
+    Configuration.addDefaultResource("mapred-default.xml");
+    Configuration.addDefaultResource("mapred-site.xml");
+  }
+
+  private static final String JH_USER_NAME =
+      "mapreduce.jobhistory.kerberos.principal";
+  private static final String JH_KEYTAB_FILE =
+      "mapreduce.jobhistory.keytab.file";
+  public static final String MAPRED_HISTORY_SERVER_HTTP_ADDRESS =
+      "mapreduce.history.server.http.address";
+  public static final String MAPRED_HISTORY_SERVER_EMBEDDED =
+      "mapreduce.history.server.embedded";
+
+  private HttpServer historyServer;
+  private JobConf conf;
+  private String historyInfoAddr;
+  private WebAppContext context;
+
+  /**
+   * Starts job history server as a independent process
+   *  * Initializes ACL Manager
+   *  * Starts a webapp to service history requests
+   *
+   * @param conf - Mr Cluster configuration
+   * @throws IOException - any exception starting history server
+   */
+  public JobHistoryServer(JobConf conf) throws IOException {
+
+    if (isEmbedded(conf)) {
+      throw new IllegalStateException("History server is configured to run " +
+          "within JobTracker. Aborting..");
+    }
+
+    historyInfoAddr = getBindAddress(conf);
+    login(conf);
+    ACLsManager aclsManager = initializeACLsManager(conf);
+    historyServer = initializeWebContainer(conf, aclsManager);
+    initializeWebServer(conf, aclsManager);
+  }
+
+  /**
+   * Starts job history server as a embedded server within job tracker
+   *  * Starts a webapp to service history requests
+   *
+   * @param conf - MR Cluster configuration
+   * @param aclsManager - ACLs Manager for user authentication
+   * @param httpServer - Http Server instance
+   * @throws IOException - any exception starting history server
+   */
+  public JobHistoryServer(JobConf conf,
+                          ACLsManager aclsManager,
+                          HttpServer httpServer) throws IOException {
+    historyInfoAddr = getBindAddress(conf);
+    this.historyServer = httpServer;
+    initializeWebServer(conf, aclsManager);
+  }
+
+  private void login(JobConf conf) throws IOException {
+    UserGroupInformation.setConfiguration(conf);
+    InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(historyInfoAddr);
+
+    SecurityUtil.login(conf, JH_KEYTAB_FILE, JH_USER_NAME, infoSocAddr.getHostName());
+    LOG.info("History server login successful");
+  }
+
+  private ACLsManager initializeACLsManager(JobConf conf)
+      throws IOException {
+    LOG.info("Initializing ACLs Manager");
+
+    Configuration queuesConf = new Configuration(conf);
+    QueueManager queueManager = new QueueManager(queuesConf);
+
+    return new ACLsManager(conf,
+        new JobACLsManager(conf), queueManager);
+  }
+
+  /**
+   * Start embedded jetty server to host history servlets/pages
+   *  - Push history file system, acl Manager and cluster conf for future
+   *    reference by the servlets/pages
+   *
+   * @param conf - Cluster configuration
+   * @param aclsManager - ACLs Manager for validating user request
+   * @throws IOException - Any exception while starting web server
+   */
+  private void initializeWebServer(final JobConf conf,
+                                            ACLsManager aclsManager)
+      throws IOException {
+
+    this.conf = conf;
+
+    FileSystem fs;
+    try {
+      fs = aclsManager.getMROwner().
+        doAs(new PrivilegedExceptionAction<FileSystem>() {
+        public FileSystem run() throws IOException {
+          return FileSystem.get(conf);
+      }});
+    } catch (InterruptedException e) {
+      throw new IOException("Operation interrupted", e);
+    }
+
+    if (!isEmbedded(conf)) {
+      JobHistory.initDone(conf, fs, false);
+    }
+    final String historyLogDir =
+      JobHistory.getCompletedJobHistoryLocation().toString();
+
+    context = historyServer.addContext("history", true);
+
+    historyServer.setAttribute(context, "historyLogDir", historyLogDir);
+    historyServer.setAttribute(context, "fileSys", fs);
+    historyServer.setAttribute(context, "jobConf", conf);
+    historyServer.setAttribute(context, "aclManager", aclsManager);
+
+    if (!isEmbedded(conf)) {
+      historyServer.setAttribute("historyLogDir", historyLogDir);
+      historyServer.setAttribute("fileSys", fs);
+      historyServer.setAttribute("jobConf", conf);
+      historyServer.setAttribute("aclManager", aclsManager);
+    }
+  }
+
+  private HttpServer initializeWebContainer(JobConf conf,
+                                      ACLsManager aclsManager)
+      throws IOException {
+    InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(historyInfoAddr);
+    int tmpInfoPort = infoSocAddr.getPort();
+    return new HttpServer("/", infoSocAddr.getHostName(),
+        tmpInfoPort, tmpInfoPort == 0, conf, aclsManager.getAdminsAcl());
+  }
+
+  public void start() throws IOException {
+    if (!isEmbedded(conf)) {
+      historyServer.start();
+    } else {
+      try {
+        context.start();
+      } catch (Exception e) {
+        throw new IOException("Unable to start history context", e);
+      }
+    }
+
+    InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(historyInfoAddr);
+    conf.set(MAPRED_HISTORY_SERVER_HTTP_ADDRESS, infoSocAddr.getHostName() +
+        ":" + historyServer.getPort());
+    LOG.info("Started job history server at: " + getAddress(conf));
+  }
+
+  public void join() throws InterruptedException {
+    historyServer.join();
+  }
+
+  /**
+   * Shutsdown the history server if already initialized
+   * @throws Exception - Any exception during shutdown
+   */
+  public void shutdown() throws Exception {
+    if (historyServer != null && !isEmbedded(conf)) {
+      LOG.info("Shutting down history server");
+      historyServer.stop();
+    }
+  }
+
+  /**
+   * Start job history server as an independent process
+   *
+   * @param args - Command line arguments
+   */
+  public static void main(String[] args) {
+    StringUtils.startupShutdownMessage(JobHistoryServer.class, args, LOG);
+
+    try {
+      JobHistoryServer server = new JobHistoryServer(new JobConf());
+      server.start();
+      server.join();
+    } catch (Throwable e) {
+      LOG.fatal(StringUtils.stringifyException(e));
+      System.exit(-1);
+    }
+  }
+
+  static boolean isEmbedded(JobConf conf) {
+    return conf.getBoolean(MAPRED_HISTORY_SERVER_EMBEDDED, true);
+  }
+
+  static String getAddress(JobConf conf) {
+    return conf.get(MAPRED_HISTORY_SERVER_HTTP_ADDRESS);
+  }
+
+  static String getHistoryUrlPrefix(JobConf conf) {
+    return "http://" + getAddress(conf) + "/history";
+  }
+
+  private static String getBindAddress(JobConf conf) {
+    return conf.get(MAPRED_HISTORY_SERVER_HTTP_ADDRESS, "localhost:0");
+  }
+}

+ 19 - 1
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -1939,6 +1939,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   CompletedJobStatusStore completedJobStatusStore = null;
   CompletedJobStatusStore completedJobStatusStore = null;
   Thread completedJobsStoreThread = null;
   Thread completedJobsStoreThread = null;
   RecoveryManager recoveryManager;
   RecoveryManager recoveryManager;
+  JobHistoryServer jobHistoryServer;
 
 
   /**
   /**
    * It might seem like a bug to maintain a TreeSet of tasktracker objects,
    * It might seem like a bug to maintain a TreeSet of tasktracker objects,
@@ -2267,6 +2268,15 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
       }
       }
     });
     });
     infoServer.setAttribute("fileSys", historyFS);
     infoServer.setAttribute("fileSys", historyFS);
+    infoServer.setAttribute("jobConf", conf);
+    infoServer.setAttribute("aclManager", aclsManager);
+
+    if (JobHistoryServer.isEmbedded(conf)) {
+      LOG.info("History server being initialized in embedded mode");
+      jobHistoryServer = new JobHistoryServer(conf, aclsManager, infoServer);
+      jobHistoryServer.start();
+      LOG.info("Job History Server web address: " + JobHistoryServer.getAddress(conf));
+    }
 
 
     this.dnsToSwitchMapping = ReflectionUtils.newInstance(
     this.dnsToSwitchMapping = ReflectionUtils.newInstance(
         conf.getClass("topology.node.switch.mapping.impl", ScriptBasedMapping.class,
         conf.getClass("topology.node.switch.mapping.impl", ScriptBasedMapping.class,
@@ -2470,6 +2480,14 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
         ex.printStackTrace();
         ex.printStackTrace();
       }
       }
     }
     }
+    if (jobHistoryServer != null) {
+      LOG.info("Stopping job history server");
+      try {
+        jobHistoryServer.shutdown();
+      } catch (Exception ex) {
+        LOG.warn("Exception shutting down Job History server", ex);
+      }
+  }
     DelegationTokenRenewal.close();
     DelegationTokenRenewal.close();
     LOG.info("stopped all jobtracker services");
     LOG.info("stopped all jobtracker services");
     return;
     return;
@@ -2533,7 +2551,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
       trackerToMarkedTasksMap.put(taskTracker, taskset);
       trackerToMarkedTasksMap.put(taskTracker, taskset);
     }
     }
     taskset.add(taskid);
     taskset.add(taskid);
-      
+
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Marked '" + taskid + "' from '" + taskTracker + "'");
       LOG.debug("Marked '" + taskid + "' from '" + taskTracker + "'");
     }
     }

+ 168 - 0
src/test/org/apache/hadoop/mapred/TestJobHistoryServer.java

@@ -0,0 +1,168 @@
+/**
+ * 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.mapred;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.mapred.lib.NullOutputFormat;
+import org.junit.Assert;
+import junit.framework.TestCase;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.net.MalformedURLException;
+
+public class TestJobHistoryServer extends TestCase {
+  private static final Log LOG = LogFactory.getLog(TestJobHistoryServer.class);
+
+  public void testHistoryServerEmbedded() {
+
+    MiniMRCluster mrCluster = null;
+    JobConf conf = new JobConf();
+    try {
+      conf.setLong("mapred.job.tracker.retiredjobs.cache.size", 1);
+      conf.setLong("mapred.jobtracker.retirejob.interval", 0);
+      conf.setLong("mapred.jobtracker.retirejob.check", 0);
+      conf.setLong("mapred.jobtracker.completeuserjobs.maximum", 0);
+      conf.set(JobHistoryServer.MAPRED_HISTORY_SERVER_HTTP_ADDRESS,
+          "localhost:0");
+
+      mrCluster = new MiniMRCluster(1, conf.get("fs.default.name"), 1,
+          null, null, conf);
+      String historyAddress = JobHistoryServer.getHistoryUrlPrefix(mrCluster.
+          getJobTrackerRunner().getJobTracker().conf);
+      LOG.info("******** History Address: " + historyAddress);
+
+      conf = mrCluster.createJobConf();
+      createInputFile(conf, "/tmp/input");
+
+      RunningJob job = runJob(conf);
+      LOG.info("Job details: " + job);
+
+      String redirectUrl = getRedirectUrl(job.getTrackingURL());
+      Assert.assertEquals(redirectUrl.contains(historyAddress), true);
+
+    } catch (IOException e) {
+      LOG.error("Failure running test", e);
+      Assert.fail(e.getMessage());
+    } finally {
+      if (mrCluster != null) mrCluster.shutdown();
+    }
+  }
+
+  public void testHistoryServerStandalone() {
+
+    MiniMRCluster mrCluster = null;
+    JobConf conf = new JobConf();
+    JobHistoryServer server = null;
+    try {
+      conf.setLong("mapred.job.tracker.retiredjobs.cache.size", 1);
+      conf.setLong("mapred.jobtracker.retirejob.interval", 0);
+      conf.setLong("mapred.jobtracker.retirejob.check", 0);
+      conf.setLong("mapred.jobtracker.completeuserjobs.maximum", 0);
+      conf.set(JobHistoryServer.MAPRED_HISTORY_SERVER_HTTP_ADDRESS,
+          "localhost:8090");
+      conf.setBoolean(JobHistoryServer.MAPRED_HISTORY_SERVER_EMBEDDED, false);
+
+      mrCluster = new MiniMRCluster(1, conf.get("fs.default.name"), 1,
+          null, null, conf);
+      server = new JobHistoryServer(conf);
+      server.start();
+
+      String historyAddress = JobHistoryServer.getHistoryUrlPrefix(conf);
+      LOG.info("******** History Address: " + historyAddress);
+
+      conf = mrCluster.createJobConf();
+      createInputFile(conf, "/tmp/input");
+
+      RunningJob job = runJob(conf);
+      LOG.info("Job details: " + job);
+
+      String redirectUrl = getRedirectUrl(job.getTrackingURL());
+      Assert.assertEquals(redirectUrl.contains(historyAddress), true);
+
+    } catch (IOException e) {
+      LOG.error("Failure running test", e);
+      Assert.fail(e.getMessage());
+    } finally {
+      if (mrCluster != null) mrCluster.shutdown();
+      try {
+        if (server != null) server.shutdown();
+      } catch (Exception ignore) { }
+    }
+  }
+
+  private void createInputFile(Configuration conf, String path)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    FSDataOutputStream out = fs.create(new Path(path));
+    try {
+      out.write("hello world".getBytes());
+    } finally {
+      out.close();
+    }
+  }
+
+  private synchronized RunningJob runJob(JobConf conf) throws IOException {
+    conf.setJobName("History");
+
+    conf.setInputFormat(TextInputFormat.class);
+
+    conf.setMapOutputKeyClass(LongWritable.class);
+    conf.setMapOutputValueClass(Text.class);
+
+    conf.setOutputFormat(NullOutputFormat.class);
+    conf.setOutputKeyClass(LongWritable.class);
+    conf.setOutputValueClass(Text.class);
+
+    conf.setMapperClass(org.apache.hadoop.mapred.lib.IdentityMapper.class);
+    conf.setReducerClass(org.apache.hadoop.mapred.lib.IdentityReducer.class);
+
+    FileInputFormat.setInputPaths(conf, "/tmp/input");
+
+    return JobClient.runJob(conf);
+  }
+
+  private String getRedirectUrl(String jobUrl) throws IOException {
+    HttpClient client = new HttpClient();
+    GetMethod method = new GetMethod(jobUrl);
+    method.setFollowRedirects(false);
+    try {
+      int status = client.executeMethod(method);
+      Assert.assertEquals(status, HttpURLConnection.HTTP_MOVED_TEMP);
+
+      LOG.info("Location: " + method.getResponseHeader("Location"));
+      return method.getResponseHeader("Location").getValue();
+    } finally {
+      method.releaseConnection();
+    }
+  }
+
+}

+ 32 - 10
src/test/org/apache/hadoop/mapred/TestWebUIAuthorization.java

@@ -293,9 +293,23 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     MyGroupsProvider.mapping.put(mrAdminUser, Arrays.asList("group8"));
     MyGroupsProvider.mapping.put(mrAdminUser, Arrays.asList("group8"));
   }
   }
 
 
-  public void testAuthorizationForJobHistoryPages() throws Exception {    
-    setupGroupsProvider();
+  public void testAuthorizationForJobHistoryPages() throws Exception {
+    checkAuthorizationForJobHistoryPages(new Properties());
+  }
+
+  public void testAuthorizationForJobHistoryPagesStandalone() throws Exception {
     Properties props = new Properties();
     Properties props = new Properties();
+    props.setProperty(JobHistoryServer.MAPRED_HISTORY_SERVER_HTTP_ADDRESS,
+        "localhost:8090");
+    props.setProperty(JobHistoryServer.MAPRED_HISTORY_SERVER_EMBEDDED,
+        "false");
+
+    checkAuthorizationForJobHistoryPages(props);
+  }
+
+  private void checkAuthorizationForJobHistoryPages(
+      Properties props) throws Exception {
+    setupGroupsProvider();
     props.setProperty("hadoop.http.filter.initializers",
     props.setProperty("hadoop.http.filter.initializers",
         DummyFilterInitializer.class.getName());
         DummyFilterInitializer.class.getName());
 
 
@@ -321,6 +335,14 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
     JobConf conf = new JobConf(cluster.createJobConf());
     JobConf conf = new JobConf(cluster.createJobConf());
     conf.set(JobContext.JOB_ACL_VIEW_JOB, viewColleague + " group3");
     conf.set(JobContext.JOB_ACL_VIEW_JOB, viewColleague + " group3");
 
 
+    //Initialize history server, if need to be started in standalone mode
+    if ("false".equals(props.getProperty(
+        JobHistoryServer.MAPRED_HISTORY_SERVER_EMBEDDED, "true"))) {
+      JobHistoryServer historyServer = new JobHistoryServer(cluster.
+          getJobTrackerRunner().getJobTracker().conf);
+      historyServer.start();
+    }
+
     // Let us add group1 and group3 to modify-job-acl. So modifyColleague and
     // Let us add group1 and group3 to modify-job-acl. So modifyColleague and
     // viewAndModifyColleague will be able to modify the job
     // viewAndModifyColleague will be able to modify the job
     conf.set(JobContext.JOB_ACL_MODIFY_JOB, " group1,group3");
     conf.set(JobContext.JOB_ACL_MODIFY_JOB, " group1,group3");
@@ -359,16 +381,16 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
         JobHistory.JobInfo.getDoneJobHistoryFileName(finalJobConf, jobid));
         JobHistory.JobInfo.getDoneJobHistoryFileName(finalJobConf, jobid));
 
 
     String urlEncodedHistoryFileName = URLEncoder.encode(historyFilePath.toString());
     String urlEncodedHistoryFileName = URLEncoder.encode(historyFilePath.toString());
-    String jtURL = "http://localhost:" + infoPort;
+    String jobHistoryUrl = JobHistoryServer.getHistoryUrlPrefix(jobTracker.conf);
 
 
     // validate access of jobdetails_history.jsp
     // validate access of jobdetails_history.jsp
-    String jobDetailsJSP =
-        jtURL + "/jobdetailshistory.jsp?logFile=" + urlEncodedHistoryFileName;
+    String jobDetailsJSP = jobHistoryUrl +
+            "/jobdetailshistory.jsp?logFile=" + urlEncodedHistoryFileName;
     validateViewJob(jobDetailsJSP, "GET");
     validateViewJob(jobDetailsJSP, "GET");
 
 
     // validate accesses of jobtaskshistory.jsp
     // validate accesses of jobtaskshistory.jsp
     String jobTasksJSP =
     String jobTasksJSP =
-        jtURL + "/jobtaskshistory.jsp?logFile=" + urlEncodedHistoryFileName;
+        jobHistoryUrl + "/jobtaskshistory.jsp?logFile=" + urlEncodedHistoryFileName;
     String[] taskTypes =
     String[] taskTypes =
         new String[] { "JOb_SETUP", "MAP", "REDUCE", "JOB_CLEANUP" };
         new String[] { "JOb_SETUP", "MAP", "REDUCE", "JOB_CLEANUP" };
     String[] states =
     String[] states =
@@ -392,7 +414,7 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
 
 
     for (String tip : tipsMap.keySet()) {
     for (String tip : tipsMap.keySet()) {
       // validate access of taskdetailshistory.jsp
       // validate access of taskdetailshistory.jsp
-      validateViewJob(jtURL + "/taskdetailshistory.jsp?logFile="
+      validateViewJob(jobHistoryUrl + "/taskdetailshistory.jsp?logFile="
           + urlEncodedHistoryFileName + "&tipid=" + tip.toString(), "GET");
           + urlEncodedHistoryFileName + "&tipid=" + tip.toString(), "GET");
 
 
       Map<String, TaskAttempt> attemptsMap =
       Map<String, TaskAttempt> attemptsMap =
@@ -400,7 +422,7 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
       for (String attempt : attemptsMap.keySet()) {
       for (String attempt : attemptsMap.keySet()) {
 
 
         // validate access to taskstatshistory.jsp
         // validate access to taskstatshistory.jsp
-        validateViewJob(jtURL + "/taskstatshistory.jsp?attemptid="
+        validateViewJob(jobHistoryUrl + "/taskstatshistory.jsp?attemptid="
             + attempt.toString() + "&logFile=" + urlEncodedHistoryFileName, "GET");
             + attempt.toString() + "&logFile=" + urlEncodedHistoryFileName, "GET");
 
 
         // validate access to tasklogs
         // validate access to tasklogs
@@ -447,12 +469,12 @@ public class TestWebUIAuthorization extends ClusterMapReduceTestCase {
 
 
     // validate access to analysejobhistory.jsp
     // validate access to analysejobhistory.jsp
     String analyseJobHistoryJSP =
     String analyseJobHistoryJSP =
-        jtURL + "/analysejobhistory.jsp?logFile=" + urlEncodedHistoryFileName;
+        jobHistoryUrl + "/analysejobhistory.jsp?logFile=" + urlEncodedHistoryFileName;
     validateViewJob(analyseJobHistoryJSP, "GET");
     validateViewJob(analyseJobHistoryJSP, "GET");
 
 
     // validate access of jobconf_history.jsp
     // validate access of jobconf_history.jsp
     String jobConfJSP =
     String jobConfJSP =
-        jtURL + "/jobconf_history.jsp?logFile=" + urlEncodedHistoryFileName;
+        jobHistoryUrl + "/jobconf_history.jsp?logFile=" + urlEncodedHistoryFileName;
     validateViewJob(jobConfJSP, "GET");
     validateViewJob(jobConfJSP, "GET");
   }
   }
 
 

+ 3 - 2
src/webapps/job/analysejobhistory.jsp → src/webapps/history/analysejobhistory.jsp

@@ -31,9 +31,10 @@
     showTasks = Integer.parseInt(numTasks);  
     showTasks = Integer.parseInt(numTasks);  
   }
   }
   FileSystem fs = (FileSystem) application.getAttribute("fileSys");
   FileSystem fs = (FileSystem) application.getAttribute("fileSys");
-  JobTracker jobTracker = (JobTracker) application.getAttribute("job.tracker");
+  JobConf jobConf = (JobConf) application.getAttribute("jobConf");
+  ACLsManager aclsManager = (ACLsManager) application.getAttribute("aclManager");
   JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
   JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
-      response, jobTracker, fs, new Path(logFile));
+      response, jobConf, aclsManager, fs, new Path(logFile));
   if (job == null) {
   if (job == null) {
     return;
     return;
   }%>
   }%>

+ 52 - 0
src/webapps/history/job_authorization_error.jsp

@@ -0,0 +1,52 @@
+<%
+/*
+ * 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.
+ */
+%>
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="javax.servlet.*"
+  import="javax.servlet.http.*"
+  import="java.io.*"
+  import="java.net.URL"
+  import="org.apache.hadoop.util.*"
+%>
+<%!	private static final long serialVersionUID = 1L;
+%>
+
+<html>
+<head>
+<title>Error: User cannot access this Job</title>
+</head>
+<body>
+<h2>Error: User cannot do this operation on this Job</h2><br>
+
+<%
+  String errorMsg = (String) request.getAttribute("error.msg");
+%>
+
+<font size="5"> 
+<%
+  out.println(errorMsg);
+%>
+</font>
+
+<hr>
+
+<%
+out.println(ServletUtil.htmlFooter());
+%>

+ 3 - 4
src/webapps/job/jobconf_history.jsp → src/webapps/history/jobconf_history.jsp

@@ -16,8 +16,6 @@
 %>
 %>
 
 
 <%
 <%
-  JobTracker tracker = (JobTracker) application.getAttribute("job.tracker");
-
   String logFileString = request.getParameter("logFile");
   String logFileString = request.getParameter("logFile");
   if (logFileString == null) {
   if (logFileString == null) {
     out.println("<h2>Missing 'logFile' for fetching job configuration!</h2>");
     out.println("<h2>Missing 'logFile' for fetching job configuration!</h2>");
@@ -43,10 +41,11 @@
   try {
   try {
     jobFile = fs.open(jobFilePath);
     jobFile = fs.open(jobFilePath);
     JobConf jobConf = new JobConf(jobFilePath);
     JobConf jobConf = new JobConf(jobFilePath);
-    JobTracker jobTracker = (JobTracker) application.getAttribute("job.tracker");
+    JobConf clusterConf = (JobConf) application.getAttribute("jobConf");
+    ACLsManager aclsManager = (ACLsManager) application.getAttribute("aclManager");
 
 
     JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
     JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
-        response, jobTracker, fs, logFile);
+        response, clusterConf, aclsManager, fs, logFile);
     if (job == null) {
     if (job == null) {
       return;
       return;
     }
     }

+ 5 - 4
src/webapps/job/jobdetailshistory.jsp → src/webapps/history/jobdetailshistory.jsp

@@ -24,9 +24,10 @@
     String jobid = JSPUtil.getJobID(new Path(logFile).getName());
     String jobid = JSPUtil.getJobID(new Path(logFile).getName());
 	
 	
     FileSystem fs = (FileSystem) application.getAttribute("fileSys");
     FileSystem fs = (FileSystem) application.getAttribute("fileSys");
-    JobTracker jobTracker = (JobTracker) application.getAttribute("job.tracker");
+    JobConf jobConf = (JobConf) application.getAttribute("jobConf");
+    ACLsManager aclsManager = (ACLsManager) application.getAttribute("aclManager");
     JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
     JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
-        response, jobTracker, fs, new Path(logFile));
+        response, jobConf, aclsManager, fs, new Path(logFile));
     if (job == null) {
     if (job == null) {
       return;
       return;
     }
     }
@@ -41,7 +42,7 @@
 </head>
 </head>
 <body>
 <body>
 
 
-<h2>Hadoop Job <%=jobid %> on <a href="jobhistory.jsp">History Viewer</a></h2>
+<h2>Hadoop Job <%=jobid %> on <a href="jobhistoryhome.jsp">History Viewer</a></h2>
 
 
 <b>User: </b> <%=HtmlQuoting.quoteHtmlChars(job.get(Keys.USER)) %><br/> 
 <b>User: </b> <%=HtmlQuoting.quoteHtmlChars(job.get(Keys.USER)) %><br/> 
 <b>JobName: </b> <%=HtmlQuoting.quoteHtmlChars(job.get(Keys.JOBNAME)) %><br/>  
 <b>JobName: </b> <%=HtmlQuoting.quoteHtmlChars(job.get(Keys.JOBNAME)) %><br/>  
@@ -49,7 +50,7 @@
                  <%=job.get(Keys.JOBCONF) %></a><br/> 
                  <%=job.get(Keys.JOBCONF) %></a><br/> 
 <%         
 <%         
   Map<JobACL, AccessControlList> jobAcls = job.getJobACLs();
   Map<JobACL, AccessControlList> jobAcls = job.getJobACLs();
-  JSPUtil.printJobACLs(jobTracker, jobAcls, out);
+  JSPUtil.printJobACLs(jobConf, jobAcls, out);
 %> 
 %> 
 <b>Submitted At: </b> <%=StringUtils.getFormattedTimeWithDiff(dateFormat, job.getLong(Keys.SUBMIT_TIME), 0 )  %><br/> 
 <b>Submitted At: </b> <%=StringUtils.getFormattedTimeWithDiff(dateFormat, job.getLong(Keys.SUBMIT_TIME), 0 )  %><br/> 
 <b>Launched At: </b> <%=StringUtils.getFormattedTimeWithDiff(dateFormat, job.getLong(Keys.LAUNCH_TIME), job.getLong(Keys.SUBMIT_TIME)) %><br/>
 <b>Launched At: </b> <%=StringUtils.getFormattedTimeWithDiff(dateFormat, job.getLong(Keys.LAUNCH_TIME), job.getLong(Keys.SUBMIT_TIME)) %><br/>

+ 573 - 0
src/webapps/history/jobhistoryhome.jsp

@@ -0,0 +1,573 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="java.io.*"
+  import="java.net.URLEncoder"
+  import="java.util.*"
+  import="java.util.regex.Pattern"
+  import="java.util.regex.Matcher"
+  import="java.util.concurrent.atomic.AtomicBoolean"
+  import="org.apache.hadoop.mapred.*"
+  import="org.apache.hadoop.util.*"
+  import="org.apache.hadoop.fs.*"
+  import="javax.servlet.jsp.*"
+  import="java.text.SimpleDateFormat"
+  import="org.apache.hadoop.http.HtmlQuoting"
+%>
+<%	
+  JobConf jobConf = (JobConf) application.getAttribute("jobConf");
+  String trackerAddress = jobConf.get("mapred.job.tracker.http.address");
+  String trackerName =
+           StringUtils.simpleHostname(trackerAddress);
+%>
+<%!	
+  private static SimpleDateFormat dateFormat = 
+                                    new SimpleDateFormat("d/MM HH:mm:ss");
+%>
+<%!	private static final long serialVersionUID = 1L;
+%>
+<html>
+<head>
+<script type="text/JavaScript">
+<!--
+function showUserHistory(search)
+{
+var url
+if (search == null || "".equals(search)) {
+  url="jobhistoryhome.jsp";
+} else {
+  url="jobhistoryhome.jsp?pageno=1&search=" + search;
+}
+window.location.href = url;
+}
+//-->
+</script>
+<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
+<title><%= trackerName %> Hadoop Map/Reduce History Viewer</title>
+<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
+</head>
+<body>
+<h1> <a href="http://<%=trackerAddress%>/jobtracker.jsp"><%= trackerName %></a> Hadoop Map/Reduce
+     <a href="jobhistoryhome.jsp">History Viewer</a></h1>
+<hr>
+<%
+  //{ // these braces are here to make indentation work and 
+  //  {// must be removed.
+
+    final int JOB_ID_START = 0;
+
+    final int FILENAME_JOBID_END = JOB_ID_START + 3;
+
+    final int FILENAME_SUBMIT_TIMESTAMP_PART = FILENAME_JOBID_END;
+    
+    final int FILENAME_USER_PART = FILENAME_JOBID_END + 1;
+
+    final int FILENAME_JOBNAME_PART = FILENAME_JOBID_END + 2;
+
+    final int[] SCAN_SIZES = { 20, 50, 200 };
+
+    final int FILES_PER_SCAN = 1000;
+
+    final int DEFAULT_PAGE_SIZE = 100;
+
+    final String DEFAULT_DATE_GLOB_COMPONENT = "*/*/*";
+
+    final String SERIAL_NUMBER_GLOB_COMPONENT = "/*";
+
+    final String search = (request.getParameter("search") == null)
+                          ? ""
+                          : request.getParameter("search");
+
+    final String dateSplit[] = search.split(";");
+
+    final String soughtDate = dateSplit.length > 1 ? dateSplit[1] : "";
+
+    final String parts[] = dateSplit[0].split(":");
+
+    final String rawUser = (parts.length >= 1)
+                            ? parts[0].toLowerCase()
+                            : "";
+
+    final String userInFname
+      = escapeUnderscores(JobHistory.JobInfo.encodeJobHistoryFileName(
+            HtmlQuoting.unquoteHtmlChars(rawUser))).toLowerCase();
+
+    final int currentScanSizeIndex
+      = (request.getParameter("scansize") == null)
+           ? 0 : Integer.parseInt(request.getParameter("scansize"));
+
+    final String SEARCH_PARSE_REGEX
+      = "([0-1]?[0-9])/([0-3]?[0-9])/((?:2[0-9])[0-9][0-9])";
+
+    final Pattern dateSearchParse = Pattern.compile(SEARCH_PARSE_REGEX);
+
+    final String rawJobname = (parts.length >= 2)
+                               ? parts[1].toLowerCase()
+                               : "";
+
+    final String jobnameKeywordInFname
+      = escapeUnderscores(JobHistory.JobInfo.encodeJobHistoryFileName(
+            HtmlQuoting.unquoteHtmlChars(rawJobname))).toLowerCase();
+
+    PathFilter jobLogFileFilter = new PathFilter() {
+      private boolean matchUser(String fileName) {
+        // return true if 
+        //  - user is not specified
+        //  - user matches
+        return "".equals(userInFname)
+           || userInFname.equals(fileName.split("_")[FILENAME_USER_PART]
+                .toLowerCase());
+      }
+
+      private boolean matchJobName(String fileName) {
+        // return true if 
+        //  - jobname is not specified
+        //  - jobname contains the keyword
+        return "".equals(jobnameKeywordInFname) 
+                 || fileName.split("_")[FILENAME_JOBNAME_PART].toLowerCase()
+                       .contains(jobnameKeywordInFname);
+      }
+
+      public boolean accept(Path path) {
+        String name = path.getName();
+
+        return !(name.endsWith(".xml")) && matchUser(name) && matchJobName(name);
+      }
+    };
+    
+    FileSystem fs = (FileSystem) application.getAttribute("fileSys");
+    String historyLogDir = (String) application.getAttribute("historyLogDir");
+    if (fs == null) {
+      out.println("Null file system. May be namenode is in safemode!");
+      return;
+    }
+
+    Comparator<Path> lastPathFirst
+      = new Comparator<Path>() {
+          public int compare(Path path1, Path path2) {
+            // these are backwards because we want the lexically lesser names
+            // to occur later in the sort.
+            return path2.getName().compareTo(path1.getName());
+          }
+    };
+
+    Comparator<Path> latestFirstCreationTimeComparator
+      = new Comparator<Path>() {
+          public int compare(Path p1, Path p2) {
+            String dp1 = null;
+            String dp2 = null;
+        
+            try {
+              dp1 = JobHistory.JobInfo.decodeJobHistoryFileName(p1.getName());
+              dp2 = JobHistory.JobInfo.decodeJobHistoryFileName(p2.getName());
+            } catch (IOException ioe) {
+              throw new RuntimeException(ioe);
+            }
+                
+            String[] split1 = dp1.split("_");
+            String[] split2 = dp2.split("_");
+        
+            // compare job tracker start time
+            // reverse the sense, because we want the newest records first
+            int res = new Date(Long.parseLong(split2[1]))
+               .compareTo(new Date(Long.parseLong(split1[1])));
+            // compare the submit times next
+            // again, reverse the sense
+            if (res == 0) {
+              res = new Date(Long.parseLong(split2[3]))
+                .compareTo(new Date(Long.parseLong(split1[3])));
+            }
+            // lastly, compare the serial numbers [a certain tiebreaker]
+            // again, reverse the sense
+            if (res == 0) {
+              Long l1 = Long.parseLong(split2[2]);
+              res = l1.compareTo(Long.parseLong(split1[2]));
+            }
+            return res;
+      }
+    };
+
+    String versionComponent = JobHistory.DONE_DIRECTORY_FORMAT_DIRNAME;
+
+    String trackerComponent = "*";
+
+    // build the glob
+    // first find the date component
+    String dateComponent = DEFAULT_DATE_GLOB_COMPONENT;
+
+    Matcher dateMatcher = dateSearchParse.matcher(soughtDate);
+
+    // burst the sought date: must be [m]m/[d]d/[2y]yy
+    if (dateMatcher.matches()) {
+      String year = dateMatcher.group(3);
+      if (year.length() == 2) {
+        year = "20" + year;
+      }
+
+      String month = dateMatcher.group(1);
+      if (month.length() == 1) {
+        month = "0" + month;
+      }
+
+      String date = dateMatcher.group(2);
+      if (date.length() == 1) {
+        date = "0" + date;
+      }
+
+      dateComponent = year + "/" + month + "/" + date;
+    }
+
+    // now we find all of the serial numbers.  This looks up all the serial
+    // number directories, but not the individual files.
+    Path historyPath = new Path(historyLogDir);
+
+    String leadGlob = (versionComponent
+            + "/" + trackerComponent
+            + "/" + dateComponent);
+
+    // Atomicity is unimportant here.
+    // I would have used MutableBoxedBoolean if such had been provided.
+    AtomicBoolean hasLegacyFiles = new AtomicBoolean(false);
+
+    Path[] snPaths
+      = FileUtil.stat2Paths(JobHistory.localGlobber
+                            (fs, historyPath, "/" + leadGlob, null, hasLegacyFiles));
+
+    Arrays.sort(snPaths, lastPathFirst);
+
+    int arrayLimit = 0;
+    int tranchesSeen = 0;
+
+    Path lastPath = null;
+
+    while (arrayLimit < snPaths.length
+           && tranchesSeen <= SCAN_SIZES[currentScanSizeIndex]) {
+      if (lastPath == null
+          || lastPathFirst.compare(lastPath, snPaths[arrayLimit]) != 0) {
+        ++tranchesSeen;
+        lastPath = snPaths[arrayLimit];
+      }
+
+      ++arrayLimit;
+    }
+
+    if (tranchesSeen > SCAN_SIZES[currentScanSizeIndex]) {
+      --arrayLimit;
+    }
+
+    // arrayLimit points to the first element [which could be element 0] that 
+    // we shouldn't consider
+
+    int numHistoryFiles = 0;
+
+    Path[] jobFiles = null;
+
+    {
+      Path[][] pathVectorVector = new Path[arrayLimit][];
+
+      for (int i = 0; i < arrayLimit; ++i) {
+        pathVectorVector[i]
+          = FileUtil.stat2Paths(fs.listStatus(snPaths[i], jobLogFileFilter));
+        numHistoryFiles += pathVectorVector[i].length;
+      }
+
+      jobFiles = new Path[numHistoryFiles];
+
+      int pathsCursor = 0;
+
+      for (int i = 0; i < arrayLimit; ++i) {
+        System.arraycopy(pathVectorVector[i], 0, jobFiles, pathsCursor,
+                         pathVectorVector[i].length);
+        pathsCursor += pathVectorVector[i].length;
+      }
+    }
+
+    boolean sizeIsExact = arrayLimit == snPaths.length;
+
+    // sizeIsExact will be true if arrayLimit is zero.
+    long lengthEstimate
+      = sizeIsExact ? numHistoryFiles
+                    : (long) numHistoryFiles * snPaths.length / arrayLimit;
+
+    if (hasLegacyFiles.get()) {
+      out.println("<h2>This history has some legacy files.  "
+                  + "<a href=\"legacyjobhistory.jsp\">go to Legacy History Viewer</a>"
+                  + "</h2>");
+    }
+
+    out.println("<!--  user : " + rawUser +
+        ", jobname : " + rawJobname + "-->");
+    if (null == jobFiles || jobFiles.length == 0)  {
+      out.println("No files found!"); 
+      return ; 
+    }
+
+    // get the pageno
+    int pageno = request.getParameter("pageno") == null
+                ? 1
+                : Integer.parseInt(request.getParameter("pageno"));
+
+    // get the total number of files to display
+    int size = DEFAULT_PAGE_SIZE;
+
+    // if show-all is requested or jobfiles < size(100)
+    if (pageno == -1 || size > jobFiles.length) {
+      size = jobFiles.length;
+    }
+
+    if (pageno == -1) { // special case 'show all'
+      pageno = 1;
+    }
+
+    int maxPageNo = (jobFiles.length + size - 1) / size;
+    // int maxPageNo = (int)Math.ceil((float)jobFiles.length / size);
+
+    // check and fix pageno
+    if (pageno < 1 || pageno > maxPageNo) {
+      out.println("Invalid page index");
+      return ;
+    }
+
+    int length = size ; // determine the length of job history files to be displayed
+    if (pageno == maxPageNo) {
+      // find the number of files to be shown on the last page
+      int startOnLast = ((pageno - 1) * size) + 1;
+      length = jobFiles.length - startOnLast + 1;
+    }
+
+    // Display the search box
+    out.println("<form name=search><b> Filter (username:jobname) </b>"); // heading
+    out.println("<input type=text name=search size=\"20\" "
+                + "value=\"" + search + "\">"); // search box
+    out.println("<input type=submit value=\"Filter!\" onClick=\"showUserHistory"
+                + "(document.getElementById('search').value)\"></form>");
+    out.println("<p><span class=\"small\">Specify [user][:jobname keyword(s)]"
+                + "[;MM/DD/YYYY] .  Each of the three components is "
+                + "optional.  Filter components are conjunctive.</span></p>");
+    out.println("<p><span class=\"small\">Example: 'smith' will display jobs"
+                + " submitted by user 'smith'. 'smith:sort' will display "
+                + "jobs from user 'smith' having a 'sort' keyword in the jobname."
+                + " ';07/04/2010' restricts to July 4, 2010</span></p>"); // example
+    out.println("<hr>");
+
+    //Show the status
+    int start = (pageno - 1) * size + 1;
+
+    // DEBUG
+    out.println("<!-- pageno : " + pageno + ", size : " + size + ", length : "
+                + length + ", start : " + start + ", maxpg : "
+                + maxPageNo + "-->");
+
+    out.println("<font size=5><b>Available Jobs in History </b></font>");
+    // display the number of jobs, start index, end index
+    out.println("(<i> <span class=\"small\">Displaying <b>" + length
+                + "</b> jobs from <b>" + start + "</b> to <b>"
+                + (start + length - 1) + "</b> out of "
+                + (sizeIsExact
+                   ? "" : "approximately ") + "<b>"
+                + lengthEstimate + "</b> jobs"
+                + (sizeIsExact
+                   ? ""
+                   : ", <b>" + numHistoryFiles + "</b> gotten"));
+    if (!"".equals(rawUser)) {
+      // show the user if present
+      out.println(" for user <b>" + rawUser + "</b>");
+    }
+    if (!"".equals(rawJobname)) {
+      out.println(" with jobname having the keyword <b>" +
+          rawJobname + "</b> in it.");
+      // show the jobname keyword if present
+    }
+    if (!DEFAULT_DATE_GLOB_COMPONENT.equals(dateComponent)) {
+      out.println(" for the date <b>" + soughtDate + "</b>");
+    }
+    out.print("</span></i>)");
+
+    final String searchPart = "&search=" + search;
+
+    final String scansizePart = "&scansize=" + currentScanSizeIndex;
+
+    final String searchPlusScan = searchPart + scansizePart;
+
+    // show the expand scope link, if we're restricted
+    if (sizeIsExact || currentScanSizeIndex == SCAN_SIZES.length - 1) {
+      out.println("[<span class=\"small\">get more results</span>]");
+    } else {
+      out.println(" [<span class=\"small\"><a href=\"jobhistoryhome.jsp?pageno=1"
+                  + searchPart + "&scansize=" + (currentScanSizeIndex + 1)
+                  + "\">get more results</a></span>]");
+    }
+
+    // show the 'show-all' link
+    out.println(" [<span class=\"small\"><a href=\"jobhistoryhome.jsp?pageno=-1"
+                + searchPlusScan + "\">show in one page</a></span>]");
+
+    // show the 'first-page' link
+    if (pageno > 1) {
+      out.println(" [<span class=\"small\"><a href=\"jobhistoryhome.jsp?pageno=1"
+                  + searchPlusScan + "\">first page</a></span>]");
+    } else {
+      out.println("[<span class=\"small\">first page]</span>");
+    }
+
+    // show the 'last-page' link
+    if (pageno < maxPageNo) {
+      out.println(" [<span class=\"small\"><a href=\"jobhistoryhome.jsp?pageno="
+                  + maxPageNo + searchPlusScan + "\">last page</a></span>]");
+    } else {
+      out.println("<span class=\"small\">[last page]</span>");
+    }
+
+    // sort the files on creation time.
+    Arrays.sort(jobFiles, latestFirstCreationTimeComparator);
+
+    out.println("<br><br>");
+
+    // print the navigation info (top)
+    printNavigationTool(pageno, size, maxPageNo, searchPlusScan, out);
+
+    out.print("<table align=center border=2 cellpadding=\"5\" cellspacing=\"2\">");
+    out.print("<tr>");
+    out.print("<td>Job submit time</td>" +
+              "<td>Job Id</td><td>Name</td><td>User</td>") ; 
+    out.print("</tr>"); 
+    
+    Set<String> displayedJobs = new HashSet<String>();
+    for (int i = start - 1; i < start + length - 1; ++i) {
+      Path jobFile = jobFiles[i];
+
+      String fname = jobFile.getName();
+      String marker = JobHistory.nonOccursString(fname);
+      String reescapedFname = JobHistory.replaceStringInstances(fname,
+                  JobHistory.UNDERSCORE_ESCAPE, marker);
+      
+      String decodedJobFileName = 
+          JobHistory.JobInfo.decodeJobHistoryFileName(reescapedFname);
+
+      String[] jobDetails = decodedJobFileName.split("_");
+      String trackerStartTime = jobDetails[1];
+      String jobId = (jobDetails[JOB_ID_START]
+                      + "_" + jobDetails[JOB_ID_START + 1]
+                      + "_" + jobDetails[JOB_ID_START + 2]);
+      String submitTimestamp = jobDetails[FILENAME_SUBMIT_TIMESTAMP_PART];
+      String userName = JobHistory.replaceStringInstances(jobDetails[FILENAME_USER_PART],
+                  marker, JobHistory.UNDERSCORE_ESCAPE);
+      String jobName = JobHistory.replaceStringInstances(jobDetails[FILENAME_JOBNAME_PART],
+                  marker, JobHistory.UNDERSCORE_ESCAPE);
+      
+      // Check if the job is already displayed. There can be multiple job 
+      // history files for jobs that have restarted
+      if (displayedJobs.contains(jobId)) {
+        continue;
+      } else {
+        displayedJobs.add(jobId);
+      }
+      
+      // Encode the logfile name again to cancel the decoding done by the browser
+      String preEncodedJobFileName = 
+          JobHistory.JobInfo.encodeJobHistoryFileName(jobFile.getName());
+
+      String encodedJobFileName = 
+          JobHistory.replaceStringInstances(preEncodedJobFileName, "%5F", "%255F");
+%>
+<center>
+<%	
+      printJob(submitTimestamp, jobId,
+               jobName, userName, new Path(jobFile.getParent(), encodedJobFileName), 
+               out) ; 
+%>
+</center> 
+<%
+    } // end while trackers 
+    out.print("</table>");
+
+    // show the navigation info (bottom)
+    printNavigationTool(pageno, size, maxPageNo, searchPlusScan, out);
+%>
+<%!
+    private void printJob(String timestamp,
+                          String jobId, String jobName,
+                          String user, Path logFile, JspWriter out)
+    throws IOException {
+      out.print("<tr>"); 
+      out.print("<td>" + new Date(Long.parseLong(timestamp)) + "</td>"); 
+      out.print("<td>" + "<a href=\"jobdetailshistory.jsp?logFile=" 
+          + logFile.toString() + "\">" + jobId + "</a></td>");
+      out.print("<td>"
+                + HtmlQuoting.quoteHtmlChars(unescapeUnderscores(jobName))
+                + "</td>"); 
+      out.print("<td>"
+                + HtmlQuoting.quoteHtmlChars(unescapeUnderscores(user))
+                + "</td>"); 
+      out.print("</tr>");
+    }
+
+    private String escapeUnderscores(String rawString) {
+      return convertStrings(rawString, "_", "%5F");
+    }
+
+    private String unescapeUnderscores(String rawString) {
+      return convertStrings(rawString, "%5F", "_");
+    }
+
+    // inefficient if there are a lot of underscores
+    private String convertStrings(String escapedString, String from, String to) {
+      int firstEscape = escapedString.indexOf(from);
+
+      if (firstEscape < 0) {
+        return escapedString;
+      }
+
+      return escapedString.substring(0, firstEscape)
+            + to
+            + unescapeUnderscores(escapedString.substring
+                                    (firstEscape + from.length()));
+    }
+
+    private void printNavigationTool(int pageno, int size, int max,
+                                     String searchPlusScan, JspWriter out)
+         throws IOException {
+      
+      final int NUMBER_INDICES_TO_SHOW = 5;
+
+      int numIndexToShow = NUMBER_INDICES_TO_SHOW; // num indexes to show on either side
+
+      //TODO check this on boundary cases
+      out.print("<center> <");
+
+      // show previous link
+      if (pageno > 1) {
+        out.println("<a href=\"jobhistoryhome.jsp?pageno=" + (pageno - 1)
+                    + searchPlusScan + "\">Previous</a>");
+      }
+
+      // display the numbered index 1 2 3 4
+      int firstPage = pageno - numIndexToShow;
+      if (firstPage < 1) {
+        firstPage = 1; // boundary condition
+      }
+
+      int lastPage = pageno + numIndexToShow;
+      if (lastPage > max) {
+        lastPage = max; // boundary condition
+      }
+
+      // debug
+      out.println("<!--DEBUG : firstPage : " + firstPage + ", lastPage : " + lastPage + " -->");
+
+      for (int i = firstPage; i <= lastPage; ++i) {
+        if (i != pageno) {// needs hyperlink
+          out.println(" <a href=\"jobhistoryhome.jsp?pageno=" + i
+                      + searchPlusScan + "\">" + i + "</a> ");
+        } else { // current page
+          out.println(i);
+        }
+      }
+
+      // show the next link
+      if (pageno < max) {
+        out.println("<a href=\"jobhistoryhome.jsp?pageno=" + (pageno + 1) + searchPlusScan + "\">Next</a>");
+      }
+      out.print("></center>");
+    }
+%> 
+</body></html>

+ 3 - 2
src/webapps/job/jobtaskshistory.jsp → src/webapps/history/jobtaskshistory.jsp

@@ -26,9 +26,10 @@
   String taskType = request.getParameter("taskType"); 
   String taskType = request.getParameter("taskType"); 
   
   
   FileSystem fs = (FileSystem) application.getAttribute("fileSys");
   FileSystem fs = (FileSystem) application.getAttribute("fileSys");
-  JobTracker jobTracker = (JobTracker) application.getAttribute("job.tracker");
+  JobConf jobConf = (JobConf) application.getAttribute("jobConf");
+  ACLsManager aclsManager = (ACLsManager) application.getAttribute("aclManager");
   JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
   JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
-      response, jobTracker, fs, new Path(logFile));
+      response, jobConf, aclsManager, fs, new Path(logFile));
   if (job == null) {
   if (job == null) {
     return;
     return;
   }
   }

+ 13 - 15
src/webapps/job/legacyjobhistory.jsp → src/webapps/history/legacyjobhistory.jsp

@@ -1,7 +1,6 @@
 <%@ page
 <%@ page
   contentType="text/html; charset=UTF-8"
   contentType="text/html; charset=UTF-8"
   import="java.io.*"
   import="java.io.*"
-  import="java.net.URLEncoder"
   import="java.util.*"
   import="java.util.*"
   import="org.apache.hadoop.mapred.*"
   import="org.apache.hadoop.mapred.*"
   import="org.apache.hadoop.util.*"
   import="org.apache.hadoop.util.*"
@@ -10,12 +9,11 @@
   import="java.text.SimpleDateFormat"
   import="java.text.SimpleDateFormat"
   import="org.apache.hadoop.http.HtmlQuoting"
   import="org.apache.hadoop.http.HtmlQuoting"
   import="org.apache.hadoop.mapred.*"
   import="org.apache.hadoop.mapred.*"
-  import="org.apache.hadoop.mapred.JobHistory.*"
 %>
 %>
 <%	
 <%	
-  JobTracker tracker = (JobTracker) application.getAttribute("job.tracker");
-  String trackerName =
-           StringUtils.simpleHostname(tracker.getJobTrackerMachine());
+    JobConf jobConf = (JobConf)application.getAttribute("jobConf");
+    String trackerAddress = jobConf.get("mapred.job.tracker.http.address");
+    String trackerName = StringUtils.simpleHostname(trackerAddress);
 %>
 %>
 <%!	
 <%!	
   private static SimpleDateFormat dateFormat = 
   private static SimpleDateFormat dateFormat = 
@@ -31,9 +29,9 @@ function showUserHistory(search)
 {
 {
 var url
 var url
 if (search == null || "".equals(search)) {
 if (search == null || "".equals(search)) {
-  url="jobhistory.jsp";
+  url="legacyjobhistory.jsp";
 } else {
 } else {
-  url="jobhistory.jsp?pageno=1&search=" + search;
+  url="legacyjobhistory.jsp?pageno=1&search=" + search;
 }
 }
 window.location.href = url;
 window.location.href = url;
 }
 }
@@ -44,8 +42,8 @@ window.location.href = url;
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">
 </head>
 </head>
 <body>
 <body>
-<h1> <a href="jobtracker.jsp"><%= trackerName %></a> Hadoop Map/Reduce 
-     <a href="jobhistory.jsp">History Viewer</a></h1>
+<h1> <a href="http://<%=trackerAddress%>/jobtracker.jsp"><%= trackerName %></a> Hadoop Map/Reduce
+     <a href="jobhistoryhome.jsp">History Viewer</a></h1>
 <hr>
 <hr>
 <%
 <%
     final String search = (request.getParameter("search") == null)
     final String search = (request.getParameter("search") == null)
@@ -163,18 +161,18 @@ window.location.href = url;
     out.print("</span></i>)");
     out.print("</span></i>)");
 
 
     // show the 'show-all' link
     // show the 'show-all' link
-    out.println(" [<span class=\"small\"><a href=\"jobhistory.jsp?pageno=-1&search=" + search + "\">show all</a></span>]");
+    out.println(" [<span class=\"small\"><a href=\"legacyjobhistory.jsp?pageno=-1&search=" + search + "\">show all</a></span>]");
 
 
     // show the 'first-page' link
     // show the 'first-page' link
     if (pageno > 1) {
     if (pageno > 1) {
-      out.println(" [<span class=\"small\"><a href=\"jobhistory.jsp?pageno=1&search=" + search + "\">first page</a></span>]");
+      out.println(" [<span class=\"small\"><a href=\"legacyjobhistory.jsp?pageno=1&search=" + search + "\">first page</a></span>]");
     } else {
     } else {
       out.println("[<span class=\"small\">first page]</span>");
       out.println("[<span class=\"small\">first page]</span>");
     }
     }
 
 
     // show the 'last-page' link
     // show the 'last-page' link
     if (pageno < maxPageNo) {
     if (pageno < maxPageNo) {
-      out.println(" [<span class=\"small\"><a href=\"jobhistory.jsp?pageno=" + maxPageNo + "&search=" + search + "\">last page</a></span>]");
+      out.println(" [<span class=\"small\"><a href=\"legacyjobhistory.jsp?pageno=" + maxPageNo + "&search=" + search + "\">last page</a></span>]");
     } else {
     } else {
       out.println("<span class=\"small\">[last page]</span>");
       out.println("<span class=\"small\">[last page]</span>");
     }
     }
@@ -286,7 +284,7 @@ window.location.href = url;
 
 
       // show previous link
       // show previous link
       if (pageno > 1) {
       if (pageno > 1) {
-        out.println("<a href=\"jobhistory.jsp?pageno=" + (pageno - 1) +
+        out.println("<a href=\"legacyjobhistory.jsp?pageno=" + (pageno - 1) +
             "&search=" + search + "\">Previous</a>");
             "&search=" + search + "\">Previous</a>");
       }
       }
 
 
@@ -306,7 +304,7 @@ window.location.href = url;
 
 
       for (int i = firstPage; i <= lastPage; ++i) {
       for (int i = firstPage; i <= lastPage; ++i) {
         if (i != pageno) {// needs hyperlink
         if (i != pageno) {// needs hyperlink
-          out.println(" <a href=\"jobhistory.jsp?pageno=" + i + "&search=" +
+          out.println(" <a href=\"legacyjobhistory.jsp?pageno=" + i + "&search=" +
               search + "\">" + i + "</a> ");
               search + "\">" + i + "</a> ");
         } else { // current page
         } else { // current page
           out.println(i);
           out.println(i);
@@ -315,7 +313,7 @@ window.location.href = url;
 
 
       // show the next link
       // show the next link
       if (pageno < max) {
       if (pageno < max) {
-        out.println("<a href=\"jobhistory.jsp?pageno=" + (pageno + 1) + "&search=" + search + "\">Next</a>");
+        out.println("<a href=\"legacyjobhistory.jsp?pageno=" + (pageno + 1) + "&search=" + search + "\">Next</a>");
       }
       }
       out.print("></center>");
       out.print("></center>");
     }
     }

+ 1 - 6
src/webapps/job/loadhistory.jsp → src/webapps/history/loadhistory.jsp

@@ -1,12 +1,7 @@
 <%@ page
 <%@ page
   contentType="text/html; charset=UTF-8"
   contentType="text/html; charset=UTF-8"
-  import="java.io.*"
-  import="java.util.*"
   import="org.apache.hadoop.mapred.*"
   import="org.apache.hadoop.mapred.*"
   import="org.apache.hadoop.fs.*"
   import="org.apache.hadoop.fs.*"
-  import="org.apache.hadoop.util.*"
-  import="javax.servlet.jsp.*"
-  import="java.text.SimpleDateFormat"  
   import="org.apache.hadoop.mapred.JobHistory.*"
   import="org.apache.hadoop.mapred.JobHistory.*"
 %>
 %>
 <%!	private static final long serialVersionUID = 1L;
 <%!	private static final long serialVersionUID = 1L;
@@ -17,7 +12,7 @@
         return !(path.getName().endsWith(".xml"));
         return !(path.getName().endsWith(".xml"));
       }
       }
     };
     };
-    
+
     FileSystem fs = (FileSystem) application.getAttribute("fileSys");
     FileSystem fs = (FileSystem) application.getAttribute("fileSys");
     String jobId = request.getParameter("jobid");
     String jobId = request.getParameter("jobid");
     JobHistory.JobInfo job = (JobHistory.JobInfo)
     JobHistory.JobInfo job = (JobHistory.JobInfo)

+ 4 - 3
src/webapps/job/taskdetailshistory.jsp → src/webapps/history/taskdetailshistory.jsp

@@ -25,9 +25,10 @@
   String encodedLogFileName = JobHistory.JobInfo.encodeJobHistoryFilePath(logFile);
   String encodedLogFileName = JobHistory.JobInfo.encodeJobHistoryFilePath(logFile);
   String jobid = JSPUtil.getJobID(new Path(encodedLogFileName).getName());
   String jobid = JSPUtil.getJobID(new Path(encodedLogFileName).getName());
   FileSystem fs = (FileSystem) application.getAttribute("fileSys");
   FileSystem fs = (FileSystem) application.getAttribute("fileSys");
-  JobTracker jobTracker = (JobTracker) application.getAttribute("job.tracker");
+  JobConf jobConf = (JobConf) application.getAttribute("jobConf");
+  ACLsManager aclsManager = (ACLsManager) application.getAttribute("aclManager");
   JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
   JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
-      response, jobTracker, fs, new Path(logFile));
+      response, jobConf, aclsManager, fs, new Path(logFile));
   if (job == null) {
   if (job == null) {
     return;
     return;
   }
   }
@@ -120,7 +121,7 @@
       TaskID tipid = attemptId.getTaskID();
       TaskID tipid = attemptId.getTaskID();
       org.apache.hadoop.mapreduce.JobID jobId = tipid.getJobID();
       org.apache.hadoop.mapreduce.JobID jobId = tipid.getJobID();
       out.print("<td>" 
       out.print("<td>" 
-       + "<a href=\"/taskstatshistory.jsp?attemptid=" + attemptId
+       + "<a href=\"taskstatshistory.jsp?attemptid=" + attemptId
            + "&logFile=" + logFile + "\">"
            + "&logFile=" + logFile + "\">"
            + counters.size() + "</a></td>");
            + counters.size() + "</a></td>");
     } else {
     } else {

+ 4 - 3
src/webapps/job/taskstatshistory.jsp → src/webapps/history/taskstatshistory.jsp

@@ -47,9 +47,10 @@
   Format decimal = new DecimalFormat();
   Format decimal = new DecimalFormat();
 
 
   FileSystem fs = (FileSystem) application.getAttribute("fileSys");
   FileSystem fs = (FileSystem) application.getAttribute("fileSys");
-  JobTracker jobTracker = (JobTracker) application.getAttribute("job.tracker");
+  JobConf jobConf = (JobConf) application.getAttribute("jobConf");
+  ACLsManager aclsManager = (ACLsManager) application.getAttribute("aclManager");
   JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
   JobHistory.JobInfo job = JSPUtil.checkAccessAndGetJobInfo(request,
-      response, jobTracker, fs, new Path(logFile));
+      response, jobConf, aclsManager, fs, new Path(logFile));
   if (job == null) {
   if (job == null) {
     return;
     return;
   }
   }
@@ -110,7 +111,7 @@
 
 
 <hr>
 <hr>
 <a href="jobdetailshistory.jsp?logFile=<%=encodedLogFileName%>">Go back to the job</a><br>
 <a href="jobdetailshistory.jsp?logFile=<%=encodedLogFileName%>">Go back to the job</a><br>
-<a href="jobtracker.jsp">Go back to JobTracker</a><br>
+<a href="jobhistoryhome.jsp">Go back to Job History Viewer</a><br>
 <%
 <%
 out.println(ServletUtil.htmlFooter());
 out.println(ServletUtil.htmlFooter());
 %>
 %>

+ 2 - 1
src/webapps/job/jobdetails.jsp

@@ -268,7 +268,8 @@
         out.println("<h2>Job " + jobId + " not known!</h2>");
         out.println("<h2>Job " + jobId + " not known!</h2>");
         return;
         return;
       }
       }
-      String historyUrl = "/jobdetailshistory.jsp?logFile=" + 
+      String historyUrl = JobHistoryServer.getHistoryUrlPrefix(tracker.conf) +
+          "/jobdetailshistory.jsp?logFile=" +
           JobHistory.JobInfo.encodeJobHistoryFilePath(historyFile);
           JobHistory.JobInfo.encodeJobHistoryFilePath(historyFile);
       response.sendRedirect(response.encodeRedirectURL(historyUrl));
       response.sendRedirect(response.encodeRedirectURL(historyUrl));
       return;
       return;

+ 28 - 569
src/webapps/job/jobhistory.jsp

@@ -1,574 +1,33 @@
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="java.io.*"
-  import="java.net.URLEncoder"
-  import="java.util.*"
-  import="java.util.regex.Pattern"
-  import="java.util.regex.Matcher"
-  import="java.util.concurrent.atomic.AtomicBoolean"
-  import="org.apache.hadoop.mapred.*"
-  import="org.apache.hadoop.util.*"
-  import="org.apache.hadoop.fs.*"
-  import="javax.servlet.jsp.*"
-  import="java.text.SimpleDateFormat"
-  import="org.apache.hadoop.http.HtmlQuoting"
-  import="org.apache.hadoop.mapred.*"
-  import="org.apache.hadoop.mapred.JobHistory.*"
-%>
-<%	
-  JobTracker tracker = (JobTracker) application.getAttribute("job.tracker");
-  String trackerName =
-           StringUtils.simpleHostname(tracker.getJobTrackerMachine());
-%>
-<%!	
-  private static SimpleDateFormat dateFormat = 
-                                    new SimpleDateFormat("d/MM HH:mm:ss");
-%>
-<%!	private static final long serialVersionUID = 1L;
-%>
-<html>
-<head>
-<script type="text/JavaScript">
-<!--
-function showUserHistory(search)
-{
-var url
-if (search == null || "".equals(search)) {
-  url="jobhistory.jsp";
-} else {
-  url="jobhistory.jsp?pageno=1&search=" + search;
-}
-window.location.href = url;
-}
-//-->
-</script>
-<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
-<title><%= trackerName %> Hadoop Map/Reduce History Viewer</title>
-<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
-</head>
-<body>
-<h1> <a href="jobtracker.jsp"><%= trackerName %></a> Hadoop Map/Reduce 
-     <a href="jobhistory.jsp">History Viewer</a></h1>
-<hr>
 <%
 <%
-  //{ // these braces are here to make indentation work and 
-  //  {// must be removed.
-
-    final int JOB_ID_START = 0;
-
-    final int FILENAME_JOBID_END = JOB_ID_START + 3;
-
-    final int FILENAME_SUBMIT_TIMESTAMP_PART = FILENAME_JOBID_END;
-    
-    final int FILENAME_USER_PART = FILENAME_JOBID_END + 1;
-
-    final int FILENAME_JOBNAME_PART = FILENAME_JOBID_END + 2;
-
-    final int[] SCAN_SIZES = { 20, 50, 200 };
-
-    final int FILES_PER_SCAN = 1000;
-
-    final int DEFAULT_PAGE_SIZE = 100;
-
-    final String DEFAULT_DATE_GLOB_COMPONENT = "*/*/*";
-
-    final String SERIAL_NUMBER_GLOB_COMPONENT = "/*";
-
-    final String search = (request.getParameter("search") == null)
-                          ? ""
-                          : request.getParameter("search");
-
-    final String dateSplit[] = search.split(";");
-
-    final String soughtDate = dateSplit.length > 1 ? dateSplit[1] : "";
-
-    final String parts[] = dateSplit[0].split(":");
-
-    final String rawUser = (parts.length >= 1)
-                            ? parts[0].toLowerCase()
-                            : "";
-
-    final String userInFname
-      = escapeUnderscores(JobHistory.JobInfo.encodeJobHistoryFileName(
-            HtmlQuoting.unquoteHtmlChars(rawUser))).toLowerCase();
-
-    final int currentScanSizeIndex
-      = (request.getParameter("scansize") == null)
-           ? 0 : Integer.parseInt(request.getParameter("scansize"));
-
-    final String SEARCH_PARSE_REGEX
-      = "([0-1]?[0-9])/([0-3]?[0-9])/((?:2[0-9])[0-9][0-9])";
-
-    final Pattern dateSearchParse = Pattern.compile(SEARCH_PARSE_REGEX);
-
-    final String rawJobname = (parts.length >= 2)
-                               ? parts[1].toLowerCase()
-                               : "";
-
-    final String jobnameKeywordInFname
-      = escapeUnderscores(JobHistory.JobInfo.encodeJobHistoryFileName(
-            HtmlQuoting.unquoteHtmlChars(rawJobname))).toLowerCase();
-
-    PathFilter jobLogFileFilter = new PathFilter() {
-      private boolean matchUser(String fileName) {
-        // return true if 
-        //  - user is not specified
-        //  - user matches
-        return "".equals(userInFname)
-           || userInFname.equals(fileName.split("_")[FILENAME_USER_PART]
-                .toLowerCase());
-      }
-
-      private boolean matchJobName(String fileName) {
-        // return true if 
-        //  - jobname is not specified
-        //  - jobname contains the keyword
-        return "".equals(jobnameKeywordInFname) 
-                 || fileName.split("_")[FILENAME_JOBNAME_PART].toLowerCase()
-                       .contains(jobnameKeywordInFname);
-      }
-
-      public boolean accept(Path path) {
-        String name = path.getName();
-
-        return !(name.endsWith(".xml")) && matchUser(name) && matchJobName(name);
-      }
-    };
-    
-    FileSystem fs = (FileSystem) application.getAttribute("fileSys");
-    String historyLogDir = (String) application.getAttribute("historyLogDir");
-    if (fs == null) {
-      out.println("Null file system. May be namenode is in safemode!");
-      return;
-    }
-
-    Comparator<Path> lastPathFirst
-      = new Comparator<Path>() {
-          public int compare(Path path1, Path path2) {
-            // these are backwards because we want the lexically lesser names
-            // to occur later in the sort.
-            return path2.getName().compareTo(path1.getName());
-          }
-    };
-
-    Comparator<Path> latestFirstCreationTimeComparator
-      = new Comparator<Path>() {
-          public int compare(Path p1, Path p2) {
-            String dp1 = null;
-            String dp2 = null;
-        
-            try {
-              dp1 = JobHistory.JobInfo.decodeJobHistoryFileName(p1.getName());
-              dp2 = JobHistory.JobInfo.decodeJobHistoryFileName(p2.getName());
-            } catch (IOException ioe) {
-              throw new RuntimeException(ioe);
-            }
-                
-            String[] split1 = dp1.split("_");
-            String[] split2 = dp2.split("_");
-        
-            // compare job tracker start time
-            // reverse the sense, because we want the newest records first
-            int res = new Date(Long.parseLong(split2[1]))
-               .compareTo(new Date(Long.parseLong(split1[1])));
-            // compare the submit times next
-            // again, reverse the sense
-            if (res == 0) {
-              res = new Date(Long.parseLong(split2[3]))
-                .compareTo(new Date(Long.parseLong(split1[3])));
-            }
-            // lastly, compare the serial numbers [a certain tiebreaker]
-            // again, reverse the sense
-            if (res == 0) {
-              Long l1 = Long.parseLong(split2[2]);
-              res = l1.compareTo(Long.parseLong(split1[2]));
-            }
-            return res;
-      }
-    };
-
-    String versionComponent = JobHistory.DONE_DIRECTORY_FORMAT_DIRNAME;
-
-    String trackerComponent = "*";
-
-    // build the glob
-    // first find the date component
-    String dateComponent = DEFAULT_DATE_GLOB_COMPONENT;
-
-    Matcher dateMatcher = dateSearchParse.matcher(soughtDate);
-
-    // burst the sought date: must be [m]m/[d]d/[2y]yy
-    if (dateMatcher.matches()) {
-      String year = dateMatcher.group(3);
-      if (year.length() == 2) {
-        year = "20" + year;
-      }
-
-      String month = dateMatcher.group(1);
-      if (month.length() == 1) {
-        month = "0" + month;
-      }
-
-      String date = dateMatcher.group(2);
-      if (date.length() == 1) {
-        date = "0" + date;
-      }
-
-      dateComponent = year + "/" + month + "/" + date;
-    }
-
-    // now we find all of the serial numbers.  This looks up all the serial
-    // number directories, but not the individual files.
-    Path historyPath = new Path(historyLogDir);
-
-    String leadGlob = (versionComponent
-            + "/" + trackerComponent
-            + "/" + dateComponent);
-
-    // Atomicity is unimportant here.
-    // I would have used MutableBoxedBoolean if such had been provided.
-    AtomicBoolean hasLegacyFiles = new AtomicBoolean(false);
-
-    Path[] snPaths
-      = FileUtil.stat2Paths(JobHistory.localGlobber
-                            (fs, historyPath, "/" + leadGlob, null, hasLegacyFiles));
-
-    Arrays.sort(snPaths, lastPathFirst);
-
-    int arrayLimit = 0;
-    int tranchesSeen = 0;
-
-    Path lastPath = null;
-
-    while (arrayLimit < snPaths.length
-           && tranchesSeen <= SCAN_SIZES[currentScanSizeIndex]) {
-      if (lastPath == null
-          || lastPathFirst.compare(lastPath, snPaths[arrayLimit]) != 0) {
-        ++tranchesSeen;
-        lastPath = snPaths[arrayLimit];
-      }
-
-      ++arrayLimit;
-    }
-
-    if (tranchesSeen > SCAN_SIZES[currentScanSizeIndex]) {
-      --arrayLimit;
-    }
-
-    // arrayLimit points to the first element [which could be element 0] that 
-    // we shouldn't consider
-
-    int numHistoryFiles = 0;
-
-    Path[] jobFiles = null;
-
-    {
-      Path[][] pathVectorVector = new Path[arrayLimit][];
-
-      for (int i = 0; i < arrayLimit; ++i) {
-        pathVectorVector[i]
-          = FileUtil.stat2Paths(fs.listStatus(snPaths[i], jobLogFileFilter));
-        numHistoryFiles += pathVectorVector[i].length;
-      }
-
-      jobFiles = new Path[numHistoryFiles];
-
-      int pathsCursor = 0;
-
-      for (int i = 0; i < arrayLimit; ++i) {
-        System.arraycopy(pathVectorVector[i], 0, jobFiles, pathsCursor,
-                         pathVectorVector[i].length);
-        pathsCursor += pathVectorVector[i].length;
-      }
-    }
-
-    boolean sizeIsExact = arrayLimit == snPaths.length;
-
-    // sizeIsExact will be true if arrayLimit is zero.
-    long lengthEstimate
-      = sizeIsExact ? numHistoryFiles
-                    : (long) numHistoryFiles * snPaths.length / arrayLimit;
-
-    if (hasLegacyFiles.get()) {
-      out.println("<h2>This history has some legacy files.  "
-                  + "<a href=\"legacyjobhistory.jsp\">go to Legacy History Viewer</a>"
-                  + "</h2>");
-    }
-
-    out.println("<!--  user : " + rawUser +
-        ", jobname : " + rawJobname + "-->");
-    if (null == jobFiles || jobFiles.length == 0)  {
-      out.println("No files found!"); 
-      return ; 
-    }
-
-    // get the pageno
-    int pageno = request.getParameter("pageno") == null
-                ? 1
-                : Integer.parseInt(request.getParameter("pageno"));
-
-    // get the total number of files to display
-    int size = DEFAULT_PAGE_SIZE;
-
-    // if show-all is requested or jobfiles < size(100)
-    if (pageno == -1 || size > jobFiles.length) {
-      size = jobFiles.length;
-    }
-
-    if (pageno == -1) { // special case 'show all'
-      pageno = 1;
-    }
-
-    int maxPageNo = (jobFiles.length + size - 1) / size;
-    // int maxPageNo = (int)Math.ceil((float)jobFiles.length / size);
-
-    // check and fix pageno
-    if (pageno < 1 || pageno > maxPageNo) {
-      out.println("Invalid page index");
-      return ;
-    }
-
-    int length = size ; // determine the length of job history files to be displayed
-    if (pageno == maxPageNo) {
-      // find the number of files to be shown on the last page
-      int startOnLast = ((pageno - 1) * size) + 1;
-      length = jobFiles.length - startOnLast + 1;
-    }
-
-    // Display the search box
-    out.println("<form name=search><b> Filter (username:jobname) </b>"); // heading
-    out.println("<input type=text name=search size=\"20\" "
-                + "value=\"" + search + "\">"); // search box
-    out.println("<input type=submit value=\"Filter!\" onClick=\"showUserHistory"
-                + "(document.getElementById('search').value)\"></form>");
-    out.println("<p><span class=\"small\">Specify [user][:jobname keyword(s)]"
-                + "[;MM/DD/YYYY] .  Each of the three components is "
-                + "optional.  Filter components are conjunctive.</span></p>");
-    out.println("<p><span class=\"small\">Example: 'smith' will display jobs"
-                + " submitted by user 'smith'. 'smith:sort' will display "
-                + "jobs from user 'smith' having a 'sort' keyword in the jobname."
-                + " ';07/04/2010' restricts to July 4, 2010</span></p>"); // example
-    out.println("<hr>");
-
-    //Show the status
-    int start = (pageno - 1) * size + 1;
-
-    // DEBUG
-    out.println("<!-- pageno : " + pageno + ", size : " + size + ", length : "
-                + length + ", start : " + start + ", maxpg : "
-                + maxPageNo + "-->");
-
-    out.println("<font size=5><b>Available Jobs in History </b></font>");
-    // display the number of jobs, start index, end index
-    out.println("(<i> <span class=\"small\">Displaying <b>" + length
-                + "</b> jobs from <b>" + start + "</b> to <b>"
-                + (start + length - 1) + "</b> out of "
-                + (sizeIsExact
-                   ? "" : "approximately ") + "<b>"
-                + lengthEstimate + "</b> jobs"
-                + (sizeIsExact
-                   ? ""
-                   : ", <b>" + numHistoryFiles + "</b> gotten"));
-    if (!"".equals(rawUser)) {
-      // show the user if present
-      out.println(" for user <b>" + rawUser + "</b>");
-    }
-    if (!"".equals(rawJobname)) {
-      out.println(" with jobname having the keyword <b>" +
-          rawJobname + "</b> in it.");
-      // show the jobname keyword if present
-    }
-    if (!DEFAULT_DATE_GLOB_COMPONENT.equals(dateComponent)) {
-      out.println(" for the date <b>" + soughtDate + "</b>");
-    }
-    out.print("</span></i>)");
-
-    final String searchPart = "&search=" + search;
-
-    final String scansizePart = "&scansize=" + currentScanSizeIndex;
-
-    final String searchPlusScan = searchPart + scansizePart;
-
-    // show the expand scope link, if we're restricted
-    if (sizeIsExact || currentScanSizeIndex == SCAN_SIZES.length - 1) {
-      out.println("[<span class=\"small\">get more results</span>]");
-    } else {
-      out.println(" [<span class=\"small\"><a href=\"jobhistory.jsp?pageno=1"
-                  + searchPart + "&scansize=" + (currentScanSizeIndex + 1)
-                  + "\">get more results</a></span>]");
-    }
-
-    // show the 'show-all' link
-    out.println(" [<span class=\"small\"><a href=\"jobhistory.jsp?pageno=-1"
-                + searchPlusScan + "\">show in one page</a></span>]");
-
-    // show the 'first-page' link
-    if (pageno > 1) {
-      out.println(" [<span class=\"small\"><a href=\"jobhistory.jsp?pageno=1"
-                  + searchPlusScan + "\">first page</a></span>]");
-    } else {
-      out.println("[<span class=\"small\">first page]</span>");
-    }
-
-    // show the 'last-page' link
-    if (pageno < maxPageNo) {
-      out.println(" [<span class=\"small\"><a href=\"jobhistory.jsp?pageno="
-                  + maxPageNo + searchPlusScan + "\">last page</a></span>]");
-    } else {
-      out.println("<span class=\"small\">[last page]</span>");
-    }
-
-    // sort the files on creation time.
-    Arrays.sort(jobFiles, latestFirstCreationTimeComparator);
-
-    out.println("<br><br>");
-
-    // print the navigation info (top)
-    printNavigationTool(pageno, size, maxPageNo, searchPlusScan, out);
-
-    out.print("<table align=center border=2 cellpadding=\"5\" cellspacing=\"2\">");
-    out.print("<tr>");
-    out.print("<td>Job submit time</td>" +
-              "<td>Job Id</td><td>Name</td><td>User</td>") ; 
-    out.print("</tr>"); 
-    
-    Set<String> displayedJobs = new HashSet<String>();
-    for (int i = start - 1; i < start + length - 1; ++i) {
-      Path jobFile = jobFiles[i];
-
-      String fname = jobFile.getName();
-      String marker = JobHistory.nonOccursString(fname);
-      String reescapedFname = JobHistory.replaceStringInstances(fname,
-                  JobHistory.UNDERSCORE_ESCAPE, marker);
-      
-      String decodedJobFileName = 
-          JobHistory.JobInfo.decodeJobHistoryFileName(reescapedFname);
-
-      String[] jobDetails = decodedJobFileName.split("_");
-      String trackerStartTime = jobDetails[1];
-      String jobId = (jobDetails[JOB_ID_START]
-                      + "_" + jobDetails[JOB_ID_START + 1]
-                      + "_" + jobDetails[JOB_ID_START + 2]);
-      String submitTimestamp = jobDetails[FILENAME_SUBMIT_TIMESTAMP_PART];
-      String userName = JobHistory.replaceStringInstances(jobDetails[FILENAME_USER_PART],
-                  marker, JobHistory.UNDERSCORE_ESCAPE);
-      String jobName = JobHistory.replaceStringInstances(jobDetails[FILENAME_JOBNAME_PART],
-                  marker, JobHistory.UNDERSCORE_ESCAPE);
-      
-      // Check if the job is already displayed. There can be multiple job 
-      // history files for jobs that have restarted
-      if (displayedJobs.contains(jobId)) {
-        continue;
-      } else {
-        displayedJobs.add(jobId);
-      }
-      
-      // Encode the logfile name again to cancel the decoding done by the browser
-      String preEncodedJobFileName = 
-          JobHistory.JobInfo.encodeJobHistoryFileName(jobFile.getName());
-
-      String encodedJobFileName = 
-          JobHistory.replaceStringInstances(preEncodedJobFileName, "%5F", "%255F");
-%>
-<center>
-<%	
-      printJob(submitTimestamp, jobId,
-               jobName, userName, new Path(jobFile.getParent(), encodedJobFileName), 
-               out) ; 
+/*
+ * 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.
+ */
 %>
 %>
-</center> 
-<%
-    } // end while trackers 
-    out.print("</table>");
 
 
-    // show the navigation info (bottom)
-    printNavigationTool(pageno, size, maxPageNo, searchPlusScan, out);
+<%@ page
+    contentType="text/html; charset=UTF-8"
+    import="org.apache.hadoop.mapred.JobHistoryServer"
+    import="org.apache.hadoop.mapred.JobConf" 
 %>
 %>
-<%!
-    private void printJob(String timestamp,
-                          String jobId, String jobName,
-                          String user, Path logFile, JspWriter out)
-    throws IOException {
-      out.print("<tr>"); 
-      out.print("<td>" + new Date(Long.parseLong(timestamp)) + "</td>"); 
-      out.print("<td>" + "<a href=\"jobdetailshistory.jsp?logFile=" 
-          + logFile.toString() + "\">" + jobId + "</a></td>");
-      out.print("<td>"
-                + HtmlQuoting.quoteHtmlChars(unescapeUnderscores(jobName))
-                + "</td>"); 
-      out.print("<td>"
-                + HtmlQuoting.quoteHtmlChars(unescapeUnderscores(user))
-                + "</td>"); 
-      out.print("</tr>");
-    }
-
-    private String escapeUnderscores(String rawString) {
-      return convertStrings(rawString, "_", "%5F");
-    }
-
-    private String unescapeUnderscores(String rawString) {
-      return convertStrings(rawString, "%5F", "_");
-    }
 
 
-    // inefficient if there are a lot of underscores
-    private String convertStrings(String escapedString, String from, String to) {
-      int firstEscape = escapedString.indexOf(from);
-
-      if (firstEscape < 0) {
-        return escapedString;
-      }
-
-      return escapedString.substring(0, firstEscape)
-            + to
-            + unescapeUnderscores(escapedString.substring
-                                    (firstEscape + from.length()));
-    }
-
-    private void printNavigationTool(int pageno, int size, int max,
-                                     String searchPlusScan, JspWriter out)
-         throws IOException {
-      
-      final int NUMBER_INDICES_TO_SHOW = 5;
-
-      int numIndexToShow = NUMBER_INDICES_TO_SHOW; // num indexes to show on either side
-
-      //TODO check this on boundary cases
-      out.print("<center> <");
-
-      // show previous link
-      if (pageno > 1) {
-        out.println("<a href=\"jobhistory.jsp?pageno=" + (pageno - 1)
-                    + searchPlusScan + "\">Previous</a>");
-      }
-
-      // display the numbered index 1 2 3 4
-      int firstPage = pageno - numIndexToShow;
-      if (firstPage < 1) {
-        firstPage = 1; // boundary condition
-      }
-
-      int lastPage = pageno + numIndexToShow;
-      if (lastPage > max) {
-        lastPage = max; // boundary condition
-      }
-
-      // debug
-      out.println("<!--DEBUG : firstPage : " + firstPage + ", lastPage : " + lastPage + " -->");
-
-      for (int i = firstPage; i <= lastPage; ++i) {
-        if (i != pageno) {// needs hyperlink
-          out.println(" <a href=\"jobhistory.jsp?pageno=" + i
-                      + searchPlusScan + "\">" + i + "</a> ");
-        } else { // current page
-          out.println(i);
-        }
-      }
-
-      // show the next link
-      if (pageno < max) {
-        out.println("<a href=\"jobhistory.jsp?pageno=" + (pageno + 1) + searchPlusScan + "\">Next</a>");
-      }
-      out.print("></center>");
-    }
-%> 
-</body></html>
+<%
+    JobConf clusterConf = (JobConf) application.getAttribute("jobConf");
+    String historyUrl = JobHistoryServer.getHistoryUrlPrefix(clusterConf) +
+        "/jobhistoryhome.jsp";
+    response.sendRedirect(response.encodeRedirectURL(historyUrl));
+    return;
+%>

+ 2 - 1
src/webapps/job/jobtracker.jsp

@@ -174,7 +174,8 @@ if (failedJobs.size() > 0) {
 <hr>
 <hr>
 
 
 <h2 id="local_logs">Local Logs</h2>
 <h2 id="local_logs">Local Logs</h2>
-<a href="logs/">Log</a> directory, <a href="jobhistory.jsp">
+<a href="logs/">Log</a> directory,
+<a href="<%=JobHistoryServer.getHistoryUrlPrefix(tracker.conf)%>/jobhistoryhome.jsp">
 Job Tracker History</a>
 Job Tracker History</a>
 
 
 <%
 <%