You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2006/05/20 01:45:18 UTC

svn commit: r407930 - in /lucene/hadoop/trunk: ./ src/examples/org/apache/hadoop/examples/ src/java/org/apache/hadoop/mapred/ src/webapps/ src/webapps/job/ src/webapps/mapred/ src/webapps/static/ src/webapps/task/

Author: cutting
Date: Fri May 19 16:45:16 2006
New Revision: 407930

URL: http://svn.apache.org/viewvc?rev=407930&view=rev
Log:
HADOOP-233.  Add web server to tasktracker that shows running tasks and logs.  Contributed by Owen.

Added:
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java
    lucene/hadoop/trunk/src/webapps/job/
      - copied from r407929, lucene/hadoop/trunk/src/webapps/mapred/
    lucene/hadoop/trunk/src/webapps/job/index.html
      - copied, changed from r407929, lucene/hadoop/trunk/src/webapps/index.html
    lucene/hadoop/trunk/src/webapps/static/
    lucene/hadoop/trunk/src/webapps/static/hadoop-logo.jpg   (with props)
    lucene/hadoop/trunk/src/webapps/task/
    lucene/hadoop/trunk/src/webapps/task/index.html
    lucene/hadoop/trunk/src/webapps/task/tasktracker.jsp
Removed:
    lucene/hadoop/trunk/src/webapps/index.html
    lucene/hadoop/trunk/src/webapps/mapred/
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/build.xml
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/RandomWriter.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java
    lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp
    lucene/hadoop/trunk/src/webapps/job/machines.jsp

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=407930&r1=407929&r2=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Fri May 19 16:45:16 2006
@@ -48,6 +48,9 @@
 12. HADOOP-216. Add task progress to task status page.
     (Bryan Pendelton via cutting)
 
+13. HADOOP-233.  Add web server to task tracker that shows running
+    tasks and logs.  Also add log access to job tracker web interface.
+    (omalley via cutting)
 
 Release 0.2.1 - 2006-05-12
 

Modified: lucene/hadoop/trunk/build.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/build.xml?rev=407930&r1=407929&r2=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/build.xml (original)
+++ lucene/hadoop/trunk/build.xml Fri May 19 16:45:16 2006
@@ -81,7 +81,8 @@
     <mkdir dir="${build.dir}"/>
     <mkdir dir="${build.classes}"/>
     <mkdir dir="${build.src}"/>
-    <mkdir dir="${build.webapps}/WEB-INF"/>
+    <mkdir dir="${build.webapps}/task/WEB-INF"/>
+    <mkdir dir="${build.webapps}/job/WEB-INF"/>
     <mkdir dir="${build.examples}"/>
 
     <mkdir dir="${test.build.dir}"/>
@@ -91,8 +92,11 @@
       <fileset dir="${conf.dir}" includes="**/*.template"/>
     </touch>
 
-    <copy todir="${build.webapps}/">
-      <fileset file="${src.webapps}/index.html"/>
+    <!-- copy all of the jsp and static files -->
+    <copy todir="${build.webapps}">
+      <fileset dir="${src.webapps}">
+        <exclude name="**/*.jsp" />
+      </fileset>
     </copy>
 
     <copy todir="${conf.dir}" verbose="true">
@@ -119,10 +123,17 @@
   <target name="compile" depends="init, record-parser">
 
     <jsp-compile
-     uriroot="${src.webapps}/mapred"
+     uriroot="${src.webapps}/task"
      outputdir="${build.src}"
      package="org.apache.hadoop.mapred"
-     webxml="${build.webapps}/WEB-INF/web.xml">
+     webxml="${build.webapps}/task/WEB-INF/web.xml">
+    </jsp-compile>
+
+    <jsp-compile
+     uriroot="${src.webapps}/job"
+     outputdir="${build.src}"
+     package="org.apache.hadoop.mapred"
+     webxml="${build.webapps}/job/WEB-INF/web.xml">
     </jsp-compile>
 
     <javac 

Modified: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/RandomWriter.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/RandomWriter.java?rev=407930&r1=407929&r2=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/RandomWriter.java (original)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/RandomWriter.java Fri May 19 16:45:16 2006
@@ -79,10 +79,12 @@
                                 BytesWritable.class, BytesWritable.class);
       int itemCount = 0;
       while (numBytesToWrite > 0) {
-        int keyLength = random.nextInt(keySizeRange) + minKeySize;
+        int keyLength = minKeySize + 
+           (keySizeRange != 0 ? random.nextInt(keySizeRange) : 0);
         randomKey.setSize(keyLength);
         randomizeBytes(randomKey.get(), 0, randomKey.getSize());
-        int valueLength = random.nextInt(valueSizeRange) + minValueSize;
+        int valueLength = minValueSize +
+           (valueSizeRange != 0 ? random.nextInt(valueSizeRange) : 0);
         randomValue.setSize(valueLength);
         randomizeBytes(randomValue.get(), 0, randomValue.getSize());
         writer.append(randomKey, randomValue);

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java?rev=407930&r1=407929&r2=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java Fri May 19 16:45:16 2006
@@ -382,7 +382,7 @@
     });
 
     // Used to provide an HTML view on Job, Task, and TaskTracker structures
-    JobTrackerInfoServer infoServer;
+    StatusHttpServer infoServer;
     int infoPort;
 
     Server interTrackerServer;
@@ -434,7 +434,7 @@
         }
 
         this.infoPort = conf.getInt("mapred.job.tracker.info.port", 50030);
-        this.infoServer = new JobTrackerInfoServer(this, infoPort);
+        this.infoServer = new StatusHttpServer("job", infoPort);
         this.infoServer.start();
 
         this.startTime = System.currentTimeMillis();
@@ -799,7 +799,7 @@
              result.add(new MapOutputLocation(status.getTaskId(), 
                                               mapTasksNeeded[i],
                                               tracker.getHost(), 
-                                              tracker.getPort()));
+                                              tracker.getHttpPort()));
           }
         }
         return (MapOutputLocation[]) 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java?rev=407930&r1=407929&r2=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java Fri May 19 16:45:16 2006
@@ -1,117 +0,0 @@
-/**
- * Copyright 2005 The Apache Software Foundation
- *
- * Licensed 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.mortbay.http.*;
-import org.mortbay.http.handler.*;
-import org.mortbay.jetty.servlet.*;
-
-import java.io.*;
-import java.net.*;
-
-/*******************************************************
- * JobTrackerInfoServer provides stats about the JobTracker
- * via HTTP.  It's useful for clients that want to track
- * their jobs' progress.
- *
- * @author Mike Cafarella
- *******************************************************/
-class JobTrackerInfoServer {
-
-    public static class RedirectHandler extends AbstractHttpHandler {
-        public void handle(String pathInContext, String pathParams, HttpRequest request, HttpResponse response) throws HttpException, IOException {
-            response.sendRedirect("/jobtracker");
-            request.setHandled(true);
-        }
-    }
-
-    /////////////////////////////////////
-    // The actual JobTrackerInfoServer
-    /////////////////////////////////////
-    static JobTracker jobTracker;
-    org.mortbay.jetty.Server server;
-
-    /**
-     * We need the jobTracker to grab stats, and the port to 
-     * know where to listen.
-     */
-    private static final boolean WINDOWS = System.getProperty("os.name").startsWith("Windows");
-    public JobTrackerInfoServer(JobTracker jobTracker, int port) throws IOException {
-        this.jobTracker = jobTracker;
-        this.server = new org.mortbay.jetty.Server();
-	URL url = JobTrackerInfoServer.class.getClassLoader().getResource("webapps");
-	String path = url.getPath();
-	if (WINDOWS && path.startsWith("/")) {
-	    path = path.substring(1);
-	    try {
-		path = URLDecoder.decode(path, "UTF-8");
-	    } catch (UnsupportedEncodingException e) {
-	    }
-	}
-        WebApplicationContext context =
-          server.addWebApplication(null,"/",new File(path).getCanonicalPath());
-
-        SocketListener socketListener = new SocketListener();
-        socketListener.setPort(port);
-        this.server.addListener(socketListener);
-
-        //
-        // REMIND - mjc - I can't figure out how to get request redirect to work.
-        // I've tried adding an additional default handler to the context, but
-        // it doesn't seem to work.  The handler has its handle() function called
-        // even when the JSP is processed correctly!  I just want to add a redirect
-        // page, when the URL is incorrectly typed.
-        //
-        // context.addHandler(new LocalNotFoundHandler());
-    }
-
-    /**
-     * The thread class we need to kick off the HTTP server async-style.
-     */
-    class HTTPStarter implements Runnable {
-        public void run() {
-            try {
-                server.start();
-            } catch (Exception me) {
-                me.printStackTrace();
-            }
-        }
-    }
-
-    /**
-     * Launch the HTTP server
-     */
-    public void start() throws IOException {
-        new Thread(new HTTPStarter()).start();
-        try {
-            Thread.sleep(1000);
-        } catch (InterruptedException ie) {
-        }
-        if (! server.isStarted()) {
-            throw new IOException("Could not start HTTP server");
-        }
-    }
-
-    /**
-     * Stop the HTTP server
-     */
-    public void stop() {
-        try {
-            this.server.stop();
-        } catch (InterruptedException ie) {
-        }
-    }
-}

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java?rev=407930&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/StatusHttpServer.java Fri May 19 16:45:16 2006
@@ -0,0 +1,110 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed 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 java.io.File;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URL;
+import java.net.URLDecoder;
+
+import org.mortbay.http.HttpContext;
+import org.mortbay.http.handler.ResourceHandler;
+import org.mortbay.http.SocketListener;
+
+/**
+ * Create a Jetty embedded server to answer http requests. The primary goal
+ * is to serve up status information for the server.
+ * There are three contexts:
+ *   "/logs/" -> points to the log directory
+ *   "/static/" -> points to common static files (src/webapps/static)
+ *   "/" -> the jsp server code from (src/webapps/<name>)
+ * @author Owen O'Malley
+ */
+public class StatusHttpServer {
+  private static org.mortbay.jetty.Server webServer = null;
+  private static final boolean isWindows = 
+    System.getProperty("os.name").startsWith("Windows");
+  
+  /**
+   * Create a status server on the given port.
+   * The jsp scripts are taken from src/webapps/<name>.
+   */
+  public StatusHttpServer(String name, int port) throws IOException {
+    webServer = new org.mortbay.jetty.Server();
+    SocketListener listener = new SocketListener();
+    listener.setPort(port);
+    webServer.addListener(listener);
+
+    // set up the context for "/logs/"
+    HttpContext logContext = new HttpContext();
+    logContext.setContextPath("/logs/*");
+    String logDir = System.getProperty("hadoop.log.dir");
+    logContext.setResourceBase(logDir);
+    logContext.addHandler(new ResourceHandler());
+    webServer.addContext(logContext);
+
+    // set up the context for "/static/*"
+    String appDir = getWebAppsPath();
+    HttpContext staticContext = new HttpContext();
+    staticContext.setContextPath("/static/*");
+    staticContext.setResourceBase(appDir + File.separator + "static");
+    staticContext.addHandler(new ResourceHandler());
+    webServer.addContext(staticContext);
+
+    // set up the context for "/" jsp files
+    webServer.addWebApplication("/", appDir + File.separator + name);      
+  }
+
+  /**
+   * Get the pathname to the webapps files.
+   * @return the pathname
+   */
+  private static String getWebAppsPath() throws IOException {
+    URL url = StatusHttpServer.class.getClassLoader().getResource("webapps");
+    String path = url.getPath();
+    if (isWindows && path.startsWith("/")) {
+      path = path.substring(1);
+      try {
+        path = URLDecoder.decode(path, "UTF-8");
+      } catch (UnsupportedEncodingException e) {
+      }
+    }
+    return new File(path).getCanonicalPath();
+  }
+  
+  /**
+   * Start the server. Does not wait for the server to start.
+   */
+  public void start() throws IOException {
+    try {
+      webServer.start();
+    } catch (IOException ie) {
+      throw ie;
+    } catch (Exception e) {
+      IOException ie = new IOException("Problem starting http server");
+      ie.initCause(e);
+      throw ie;
+    }
+  }
+  
+  /**
+   * stop the server
+   */
+  public void stop() throws InterruptedException {
+    webServer.stop();
+  }
+}

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java?rev=407930&r1=407929&r2=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java Fri May 19 16:45:16 2006
@@ -16,7 +16,6 @@
  package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.util.*;
 
@@ -33,8 +32,10 @@
  * @author Mike Cafarella
  *******************************************************/
 public class TaskTracker implements MRConstants, TaskUmbilicalProtocol, MapOutputProtocol, Runnable {
+    private static TaskTracker taskTracker = null;
     static final long WAIT_FOR_DONE = 3 * 1000;
     private long taskTimeout; 
+    private int httpPort;
 
     static final int STALE_STATE = 1;
 
@@ -111,14 +112,15 @@
     /**
      * Start with the local machine name, and the default JobTracker
      */
-    public TaskTracker(JobConf conf) throws IOException {
-      this(JobTracker.getAddress(conf), conf);
+    public TaskTracker(JobConf conf, int httpPort) throws IOException {
+      this(JobTracker.getAddress(conf), conf, httpPort);
     }
 
     /**
      * Start with the local machine name, and the addr of the target JobTracker
      */
-    public TaskTracker(InetSocketAddress jobTrackAddr, JobConf conf) throws IOException {
+    public TaskTracker(InetSocketAddress jobTrackAddr, JobConf conf,
+                       int httpPort) throws IOException {
         maxCurrentTasks = conf.getInt("mapred.tasktracker.tasks.maximum", 2);
 
         this.fConf = conf;
@@ -126,6 +128,7 @@
         this.taskTimeout = conf.getInt("mapred.task.timeout", 10* 60 * 1000);
         this.mapOutputFile = new MapOutputFile();
         this.mapOutputFile.setConf(conf);
+        this.httpPort = httpPort;
         initialize();
     }
 
@@ -285,7 +288,8 @@
             
             TaskTrackerStatus status = 
               new TaskTrackerStatus(taskTrackerName, localHostname, 
-                                    mapOutputPort, taskReports, failures); 
+                                    mapOutputPort, httpPort, taskReports, 
+                                    failures); 
             int resultCode = jobClient.emitHeartbeat(status, justStarted);
             justStarted = false;
               
@@ -906,9 +910,47 @@
     }
 
     /**
+     * Get the task tracker for use with the webapp stuff.
+     * @return The task tracker object
+     */
+    static TaskTracker getTracker() {
+      return taskTracker;
+    }
+    
+    /**
+     * Get the name for this task tracker.
+     * @return the string like "tracker_mymachine:50010"
+     */
+    String getName() {
+      return taskTrackerName;
+    }
+    
+    /**
+     * Get the list of tasks that will be reported back to the 
+     * job tracker in the next heartbeat cycle.
+     * @return a copy of the list of TaskStatus objects
+     */
+    synchronized List getRunningTaskStatuses() {
+      List result = new ArrayList(runningTasks.size());
+      Iterator itr = runningTasks.values().iterator();
+      while (itr.hasNext()) {
+        TaskInProgress tip = (TaskInProgress) itr.next();
+        result.add(tip.createStatus());
+      }
+      return result;
+    }
+    
+    /**
+     * Get the default job conf for this tracker.
+     */
+    JobConf getJobConf() {
+      return fConf;
+    }
+    
+    /**
      * Start the TaskTracker, point toward the indicated JobTracker
      */
-    public static void main(String argv[]) throws IOException {
+    public static void main(String argv[]) throws Exception {
         if (argv.length != 0) {
             System.out.println("usage: TaskTracker");
             System.exit(-1);
@@ -916,7 +958,10 @@
 
         JobConf conf=new JobConf();
         LogFormatter.initFileHandler( conf, "tasktracker" );
-        TaskTracker tt = new TaskTracker(conf);
-        tt.run();
+        int httpPort = conf.getInt("tasktracker.http.port", 50060);
+        StatusHttpServer server = new StatusHttpServer("task", httpPort);
+        server.start();
+        taskTracker = new TaskTracker(conf, httpPort);
+        taskTracker.run();
     }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java?rev=407930&r1=407929&r2=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java Fri May 19 16:45:16 2006
@@ -41,6 +41,7 @@
     String trackerName;
     String host;
     int port;
+    int httpPort;
     int failures;
     Vector taskReports;
     
@@ -54,10 +55,11 @@
     /**
      */
     public TaskTrackerStatus(String trackerName, String host, int port, 
-                             Vector taskReports, int failures) {
+                             int httpPort, Vector taskReports, int failures) {
         this.trackerName = trackerName;
         this.host = host;
         this.port = port;
+        this.httpPort = httpPort;
 
         this.taskReports = new Vector();
         this.taskReports.addAll(taskReports);
@@ -81,6 +83,14 @@
     }
 
     /**
+     * Get the port that this task tracker is serving http requests on.
+     * @return the http port
+     */
+    public int getHttpPort() {
+      return httpPort;
+    }
+    
+    /**
      * Get the number of tasks that have failed on this tracker.
      * @return The number of failed tasks
      */
@@ -136,6 +146,7 @@
         UTF8.writeString(out, trackerName);
         UTF8.writeString(out, host);
         out.writeInt(port);
+        out.writeInt(httpPort);
 
         out.writeInt(taskReports.size());
         out.writeInt(failures);
@@ -150,6 +161,7 @@
         this.trackerName = UTF8.readString(in);
         this.host = UTF8.readString(in);
         this.port = in.readInt();
+        this.httpPort = in.readInt();
 
         taskReports = new Vector();
         taskReports.clear();

Copied: lucene/hadoop/trunk/src/webapps/job/index.html (from r407929, lucene/hadoop/trunk/src/webapps/index.html)
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/index.html?p2=lucene/hadoop/trunk/src/webapps/job/index.html&p1=lucene/hadoop/trunk/src/webapps/index.html&r1=407929&r2=407930&rev=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/index.html (original)
+++ lucene/hadoop/trunk/src/webapps/job/index.html Fri May 19 16:45:16 2006
@@ -1,3 +1,4 @@
+<meta HTTP-EQUIV="REFRESH" content="0;url=jobtracker.jsp"/>
 <html>
 
 <head>

Modified: lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp?rev=407930&r1=407929&r2=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp Fri May 19 16:45:16 2006
@@ -79,6 +79,7 @@
 <h1><%= trackerLabel %> Hadoop Map/Reduce Administration</h1>
 
 This JobTracker has been up since <%= new Date(tracker.getStartTime())%>.<br>
+
 <hr>
 <h2>Cluster Summary</h2>
 <center>
@@ -98,12 +99,19 @@
 <%
     generateJobTable(out, "Completed", tracker.completedJobs());
 %>
+
 <hr>
 
 <h2>Failed Jobs</h2>
 <%
     generateJobTable(out, "Failed", tracker.failedJobs());
 %>
+
+<hr>
+
+<h2>Local logs</h2>
+<a href="/logs/">Log</a> directory
+
 <hr>
 <a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2006.<br>
 </body>

Modified: lucene/hadoop/trunk/src/webapps/job/machines.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/machines.jsp?rev=407930&r1=407929&r2=407930&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/webapps/job/machines.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/job/machines.jsp Fri May 19 16:45:16 2006
@@ -44,9 +44,10 @@
           maxFailures = numFailures;
           failureKing = tt.getTrackerName();
         }
-
-        out.print("<tr><td>" + tt.getTrackerName() + "</td><td>" + 
-                  tt.getHost() + "</td><td>" + numCurTasks +
+        out.print("<tr><td><a href=\"http://");
+        out.print(tt.getHost() + ":" + tt.getHttpPort() + "/\">");
+        out.print(tt.getTrackerName() + "</a></td><td>");
+        out.print(tt.getHost() + "</td><td>" + numCurTasks +
                   "</td><td>" + numFailures + 
                   "</td><td>" + sinceHeartbeat + "</td></tr>\n");
       }

Added: lucene/hadoop/trunk/src/webapps/static/hadoop-logo.jpg
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/static/hadoop-logo.jpg?rev=407930&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/hadoop/trunk/src/webapps/static/hadoop-logo.jpg
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/hadoop/trunk/src/webapps/task/index.html
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/task/index.html?rev=407930&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/webapps/task/index.html (added)
+++ lucene/hadoop/trunk/src/webapps/task/index.html Fri May 19 16:45:16 2006
@@ -0,0 +1 @@
+<meta HTTP-EQUIV="REFRESH" content="0;url=tasktracker.jsp"/>

Added: lucene/hadoop/trunk/src/webapps/task/tasktracker.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/task/tasktracker.jsp?rev=407930&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/webapps/task/tasktracker.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/task/tasktracker.jsp Fri May 19 16:45:16 2006
@@ -0,0 +1,69 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="javax.servlet.*"
+  import="javax.servlet.http.*"
+  import="java.io.*"
+  import="java.util.*"
+  import="java.text.DecimalFormat"
+  import="org.apache.hadoop.mapred.*"
+  import="org.apache.hadoop.util.*"
+%>
+<%!
+  private static DecimalFormat percentFormat = new DecimalFormat("##0.00");
+  
+  private String stringifyState(int state) {
+    if (state == TaskStatus.RUNNING){
+      return "RUNNING";
+    } else if (state == TaskStatus.SUCCEEDED){
+      return "SUCCEDED";
+    } else if (state == TaskStatus.FAILED){
+      return "FAILED";
+    } else if (state == TaskStatus.UNASSIGNED){
+      return "UNASSIGNED";
+    }
+    return "unknown status";
+  }
+  
+ %>
+
+<%
+  TaskTracker tracker = TaskTracker.getTracker();
+  String trackerName = tracker.getName();
+%>
+
+<html>
+
+<title><%= trackerName %> Task Tracker Status</title>
+
+<body>
+<h1><%= trackerName %> Task Tracker Status</h1>
+<img src="/static/hadoop-logo.jpg"/>
+
+<h2>Running tasks</h2>
+<center>
+<table border=2 cellpadding="5" cellspacing="2">
+<tr><td align="center">Task Attempts</td><td>Status</td>
+    <td>Progress</td><td>Errors</td></tr>
+
+  <%
+     Iterator itr = tracker.getRunningTaskStatuses().iterator();
+     while (itr.hasNext()) {
+       TaskStatus status = (TaskStatus) itr.next();
+       out.print("<tr><td>" + status.getTaskId());
+       out.print("</td><td>" + stringifyState(status.getRunState())); 
+       out.print("</td><td>" + 
+                 percentFormat.format(100.0 * status.getProgress()));
+       out.print("</td><td><pre>" + status.getDiagnosticInfo() + "</pre></td>");
+       out.print("</tr>\n");
+     }
+  %>
+</table>
+</center>
+
+<h2>Local Logs</h2>
+<a href="/logs/">Log</a> directory
+
+<hr>
+<a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2006.<br>
+</body>
+</html>