Browse Source

HADOOP-3923. Remove org.apache.hadoop.mapred.StatusHttpServer. (szetszwo)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@713222 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 16 years ago
parent
commit
63d9a6e205

+ 2 - 0
CHANGES.txt

@@ -88,6 +88,8 @@ Trunk (unreleased changes)
     HADOOP-4583. Several code optimizations in HDFS.  (Suresh Srinivas via
     szetszwo)
 
+    HADOOP-3923. Remove org.apache.hadoop.mapred.StatusHttpServer.  (szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 2 - 1
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java

@@ -34,6 +34,7 @@ import java.util.Map.Entry;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.util.ReflectionUtils;
 
@@ -145,7 +146,7 @@ public class FairScheduler extends TaskScheduler {
       // Register servlet with JobTracker's Jetty server
       if (taskTrackerManager instanceof JobTracker) {
         JobTracker jobTracker = (JobTracker) taskTrackerManager;
-        StatusHttpServer infoServer = jobTracker.infoServer;
+        HttpServer infoServer = jobTracker.infoServer;
         infoServer.setAttribute("scheduler", this);
         infoServer.addServlet("scheduler", "/scheduler",
             FairSchedulerServlet.class);

+ 4 - 2
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
@@ -1036,7 +1037,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
                                    );
 
   // Used to provide an HTML view on Job, Task, and TaskTracker structures
-  StatusHttpServer infoServer;
+  final HttpServer infoServer;
   int infoPort;
 
   Server interTrackerServer;
@@ -1103,7 +1104,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     String infoBindAddress = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();
     this.startTime = System.currentTimeMillis();
-    infoServer = new StatusHttpServer("job", infoBindAddress, tmpInfoPort, 
+    infoServer = new HttpServer("job", infoBindAddress, tmpInfoPort, 
         tmpInfoPort == 0, conf);
     infoServer.setAttribute("job.tracker", this);
     // initialize history parameters.
@@ -1117,6 +1118,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
       historyFS = new Path(historyLogDir).getFileSystem(conf);
       infoServer.setAttribute("fileSys", historyFS);
     }
+    infoServer.addServlet("reducegraph", "/taskgraph", TaskGraphServlet.class);
     infoServer.start();
     
     trackerIdentifier = getDateFormat().format(new Date());

+ 0 - 266
src/mapred/org/apache/hadoop/mapred/StatusHttpServer.java

@@ -1,266 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.mapred;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * A mapred http server.  
- */
-public class StatusHttpServer extends org.apache.hadoop.http.HttpServer {
-  /**
-   * Create a status server on the given port.
-   * The jsp scripts are taken from src/webapps/<name>.
-   * @param name The name of the server
-   * @param port The port to use on the server
-   * @param findPort whether the server should start at the given port and 
-   *        increment by 1 until it finds a free port.
-   */
-  StatusHttpServer(String name, String bindAddress, int port, boolean findPort,
-      Configuration conf) throws IOException {
-    super(name, bindAddress, port, findPort, conf);
-    addServlet("reducegraph", "/taskgraph", TaskGraphServlet.class);
-  }
-
-  /** The servlet that outputs svg graphics for map / reduce task 
-   *  statuses
-   */
-  public static class TaskGraphServlet extends HttpServlet {
-
-    private static final long serialVersionUID = -1365683739392460020L;
-
-    /**height of the graph w/o margins*/ 
-    public static final int width = 600;
-    
-    /**height of the graph w/o margins*/ 
-    public static final int height = 200;
-    
-    /**margin space on y axis */
-    public static final int ymargin = 20;
-
-    /**margin space on x axis */
-    public static final int xmargin = 80;
-    
-    private static final float oneThird = 1f / 3f;
-    
-    @Override
-    public void doGet(HttpServletRequest request, HttpServletResponse response)
-      throws ServletException, IOException {
-
-      response.setContentType("image/svg+xml");
-
-      JobTracker tracker = 
-        (JobTracker) getServletContext().getAttribute("job.tracker");
-      
-      String jobIdStr = request.getParameter("jobid");
-      if(jobIdStr == null)
-        return;
-      JobID jobId = JobID.forName(jobIdStr);
-      String typeStr = request.getParameter("type");
-      boolean isMap = false;
-      if("map".equalsIgnoreCase(typeStr)) {
-        isMap = true;
-      }
-      
-      PrintWriter out = response.getWriter();
-      TaskReport[] reports = null;
-      
-      reports = isMap ? tracker.getMapTaskReports(jobId) 
-                      : tracker.getReduceTaskReports(jobId);
-      
-      int numTasks = reports.length;
-      if(numTasks <= 0) {
-        return;
-      }
-       
-      int tasksPerBar = (int)Math.ceil(numTasks / 600d);
-      int numBars = (int) Math.ceil((double)numTasks / tasksPerBar);
-      int w = Math.max(600, numBars);
-      int barWidth = Math.min(10,  w / numBars); //min 1px, max 10px
-      int barsPerNotch = (int)Math.ceil(10d / barWidth);
-      w = w + numBars / barsPerNotch;
-      int totalWidth = w + 2 * xmargin;
-      
-      //draw a white rectangle
-      out.print("<?xml version=\"1.0\" standalone=\"no\"?>\n" + 
-        "<!DOCTYPE svg PUBLIC \"-//W3C//DTD SVG 1.1//EN\" \n" + 
-        "\"http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd\">\n" +
-        "<?xml-stylesheet type=\"text/css\" href=\"/static/hadoop.css\"?>\n\n"+
-        "<svg width=\"");out.print(totalWidth);
-      out.print("\" height=\"");out.print(height + 2 * ymargin);
-      out.print("\" version=\"1.1\"\n" + 
-        "xmlns=\"http://www.w3.org/2000/svg\">\n\n"); 
-      
-      //axes
-      printLine(out, xmargin - 1, xmargin - 1, height + ymargin + 1
-          , ymargin - 1, "black" );
-      printLine(out, xmargin - 1, w + xmargin + 1 ,height + ymargin + 1 
-          , height + ymargin + 1, "black" );
-      
-      //borderlines
-      printLine(out, w + xmargin + 1 , w + xmargin +1
-          , height + ymargin + 1,ymargin - 1, "#CCCCCC" );
-      printLine(out, xmargin - 1, w + xmargin + 1
-          , ymargin - 1 , ymargin - 1, "#CCCCCC" );
-      
-      String[]  colors = new String[] {"#00DD00", "#E50000", "#AAAAFF"};
-      
-      //determine the notch interval using the number of digits for numTasks
-      int xNotchInterval = (int)(Math.ceil( numTasks / 10d));
-      
-      int xOffset = -1; 
-      int xNotchCount = 0;
-      //task bar graph
-      if(reports != null) {
-        for(int i=0, barCnt=0; ;i+=tasksPerBar, barCnt++) {
-          if(barCnt % barsPerNotch == 0) {
-            xOffset++;
-          }
-          int x = barCnt * barWidth + xmargin + xOffset;
-          //x axis notches
-          if(i >= xNotchInterval * xNotchCount) {
-            printLine(out, x, x, height + ymargin + 3 
-                , height + ymargin - 2, "black");
-            printText(out, x, height + ymargin + 15 
-                , String.valueOf(xNotchInterval * xNotchCount++ ), "middle");
-          }
-          if(i >= reports.length) break;
-          
-          if(isMap) {
-            float progress = getMapAvarageProgress(tasksPerBar, i, reports);
-            int barHeight = (int)Math.ceil(height * progress);
-            int y = height - barHeight + ymargin;
-            printRect(out, barWidth, barHeight,x , y , colors[2]);
-          }
-          else {
-            float[] progresses 
-              = getReduceAvarageProgresses(tasksPerBar, i, reports);
-            //draw three bars stacked, for copy, sort, reduce
-            
-            int prevHeight =0;
-            for(int j=0; j < 3 ; j++) {
-              int barHeight = (int)((height / 3) * progresses[j]);
-              if(barHeight > height/ 3 - 3)//fix rounding error
-                barHeight = height / 3 + 1;
-              
-              int y = height - barHeight + ymargin - prevHeight;
-              prevHeight += barHeight;
-              printRect(out, barWidth, barHeight, x, y, colors[j] );
-            }
-          }
-        }
-      }
-      
-      //y axis notches
-      for(int i=0;i<=10;i++) {
-        printLine(out, xmargin-3 , xmargin+2 , ymargin + (i * height) / 10
-            , ymargin + (i * height) / 10 , "black");
-        printText(out, xmargin - 10 , ymargin + 4 + (i * height) / 10 
-            , String.valueOf(100 - i * 10), "end");
-      }
-      
-      if(!isMap) {
-        //print color codes for copy, sort, reduce
-        printRect(out, 14, 14, xmargin + w + 4, ymargin + 20, colors[0]);
-        printText(out, xmargin + w + 24, ymargin + 30, "copy", "start");
-        printRect(out, 14, 14, xmargin + w + 4, ymargin + 50, colors[1]);
-        printText(out, xmargin + w + 24, ymargin + 60, "sort", "start");
-        printRect(out, 14, 14, xmargin + w + 4, ymargin + 80, colors[2]);
-        printText(out, xmargin + w + 24, ymargin + 90, "reduce", "start");
-      }
-      
-      
-      //firefox curently does not support vertical text
-      //out.print("<text x=\"");out.print(6);
-      //out.print("\" y=\""); out.print(ymargin + height / 2); 
-      //out.print("\" style=\"text-anchor:middle;writing-mode:tb\">"
-      //+"Percent</text>\n");
-      
-      out.print("</svg>");
-    }
-  
-    /**Computes average progress per bar*/
-    private float getMapAvarageProgress(int tasksPerBar, int index
-        , TaskReport[] reports ) {
-      float progress = 0f;
-      int k=0;
-      for(;k < tasksPerBar && index + k < reports.length; k++) { 
-        progress += reports[index + k].getProgress();
-      }
-      progress /= k;
-      return progress;
-    }
-  
-    /**Computes average progresses per bar*/
-    private float[] getReduceAvarageProgresses(int tasksPerBar, int index
-        , TaskReport[] reports ) {
-      float[] progresses = new float[] {0,0,0};
-      int k=0;
-      for(;k < tasksPerBar && index + k < reports.length; k++) {
-        float progress = reports[index+k].getProgress();
-        for(int j=0; progress > 0 ; j++, progress -= oneThird) {
-          if(progress > oneThird)
-            progresses[j] += 1f;
-          else 
-            progresses[j] += progress * 3 ;
-        }
-      }
-      for(int j=0; j<3; j++) { progresses[j] /= k;}
-      
-      return progresses;
-    }
-    
-    private void printRect(PrintWriter out, int width, int height
-        , int x, int y, String color) throws IOException {
-      if(height > 0) {
-        out.print("<rect width=\"");out.print(width);
-        out.print("\" height=\"");  out.print(height);
-        out.print("\" x=\""); out.print(x);
-        out.print("\" y=\""); out.print(y);
-        out.print("\" style=\"fill:"); out.print(color);out.print("\"/>\n");
-      }
-    }
-    private void printLine(PrintWriter out, int x1, int x2
-        , int y1, int y2, String color) throws IOException {
-      out.print("<line x1=\"");out.print(x1);
-      out.print("\" x2=\"");out.print(x2);
-      out.print("\" y1=\"");out.print(y1);
-      out.print("\" y2=\""); out.print(y2);
-      out.print("\" class=\"taskgraphline\" style=\"stroke:"); 
-      out.print(color); out.print("\"/>\n"); 
-    }
-    private void printText(PrintWriter out, int x, int y, String text
-        , String anchor) throws IOException {
-      out.print("<text x=\"");out.print(String.valueOf(x));
-      out.print("\" y=\""); out.print(String.valueOf(y));
-      out.print("\" style=\"fill:black;font-family:sans-serif;" 
-          + "text-anchor:");out.print(anchor); out.print("\">");
-      out.print(text); out.print("</text>\n");
-    }
-  }
-  
-}
-

+ 235 - 0
src/mapred/org/apache/hadoop/mapred/TaskGraphServlet.java

@@ -0,0 +1,235 @@
+/**
+ * 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 java.io.IOException;
+import java.io.PrintWriter;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+/** The servlet that outputs svg graphics for map / reduce task
+ *  statuses
+ */
+public class TaskGraphServlet extends HttpServlet {
+
+  private static final long serialVersionUID = -1365683739392460020L;
+
+  /**height of the graph w/o margins*/ 
+  public static final int width = 600;
+  
+  /**height of the graph w/o margins*/ 
+  public static final int height = 200;
+  
+  /**margin space on y axis */
+  public static final int ymargin = 20;
+
+  /**margin space on x axis */
+  public static final int xmargin = 80;
+  
+  private static final float oneThird = 1f / 3f;
+  
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+    throws ServletException, IOException {
+
+    response.setContentType("image/svg+xml");
+
+    JobTracker tracker = 
+      (JobTracker) getServletContext().getAttribute("job.tracker");
+    
+    String jobIdStr = request.getParameter("jobid");
+    if(jobIdStr == null)
+      return;
+    JobID jobId = JobID.forName(jobIdStr);
+
+    final boolean isMap = "map".equalsIgnoreCase(request.getParameter("type"));
+    final TaskReport[] reports = isMap? tracker.getMapTaskReports(jobId) 
+                                      : tracker.getReduceTaskReports(jobId);
+    if(reports == null || reports.length == 0) {
+      return;
+    }
+
+    final int numTasks = reports.length;     
+    int tasksPerBar = (int)Math.ceil(numTasks / 600d);
+    int numBars = (int) Math.ceil((double)numTasks / tasksPerBar);
+    int w = Math.max(600, numBars);
+    int barWidth = Math.min(10,  w / numBars); //min 1px, max 10px
+    int barsPerNotch = (int)Math.ceil(10d / barWidth);
+    w = w + numBars / barsPerNotch;
+    int totalWidth = w + 2 * xmargin;
+    
+    //draw a white rectangle
+    final PrintWriter out = response.getWriter();
+    out.print("<?xml version=\"1.0\" standalone=\"no\"?>\n" + 
+      "<!DOCTYPE svg PUBLIC \"-//W3C//DTD SVG 1.1//EN\" \n" + 
+      "\"http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd\">\n" +
+      "<?xml-stylesheet type=\"text/css\" href=\"/static/hadoop.css\"?>\n\n"+
+      "<svg width=\"");out.print(totalWidth);
+    out.print("\" height=\"");out.print(height + 2 * ymargin);
+    out.print("\" version=\"1.1\"\n" + 
+      "xmlns=\"http://www.w3.org/2000/svg\">\n\n"); 
+    
+    //axes
+    printLine(out, xmargin - 1, xmargin - 1, height + ymargin + 1
+        , ymargin - 1, "black" );
+    printLine(out, xmargin - 1, w + xmargin + 1 ,height + ymargin + 1 
+        , height + ymargin + 1, "black" );
+    
+    //borderlines
+    printLine(out, w + xmargin + 1 , w + xmargin +1
+        , height + ymargin + 1,ymargin - 1, "#CCCCCC" );
+    printLine(out, xmargin - 1, w + xmargin + 1
+        , ymargin - 1 , ymargin - 1, "#CCCCCC" );
+    
+    String[]  colors = new String[] {"#00DD00", "#E50000", "#AAAAFF"};
+    
+    //determine the notch interval using the number of digits for numTasks
+    int xNotchInterval = (int)(Math.ceil( numTasks / 10d));
+    
+    int xOffset = -1; 
+    int xNotchCount = 0;
+    //task bar graph
+    for(int i=0, barCnt=0; ;i+=tasksPerBar, barCnt++) {
+      if(barCnt % barsPerNotch == 0) {
+        xOffset++;
+      }
+      int x = barCnt * barWidth + xmargin + xOffset;
+      //x axis notches
+      if(i >= xNotchInterval * xNotchCount) {
+        printLine(out, x, x, height + ymargin + 3 
+            , height + ymargin - 2, "black");
+        printText(out, x, height + ymargin + 15 
+            , String.valueOf(xNotchInterval * xNotchCount++ ), "middle");
+      }
+      if(i >= reports.length) break;
+      
+      if(isMap) {
+        float progress = getMapAvarageProgress(tasksPerBar, i, reports);
+        int barHeight = (int)Math.ceil(height * progress);
+        int y = height - barHeight + ymargin;
+        printRect(out, barWidth, barHeight,x , y , colors[2]);
+      }
+      else {
+        float[] progresses 
+          = getReduceAvarageProgresses(tasksPerBar, i, reports);
+        //draw three bars stacked, for copy, sort, reduce
+        
+        int prevHeight =0;
+        for(int j=0; j < 3 ; j++) {
+          int barHeight = (int)((height / 3) * progresses[j]);
+          if(barHeight > height/ 3 - 3)//fix rounding error
+            barHeight = height / 3 + 1;
+          
+          int y = height - barHeight + ymargin - prevHeight;
+          prevHeight += barHeight;
+          printRect(out, barWidth, barHeight, x, y, colors[j] );
+        }
+      }
+    }
+    
+    //y axis notches
+    for(int i=0;i<=10;i++) {
+      printLine(out, xmargin-3 , xmargin+2 , ymargin + (i * height) / 10
+          , ymargin + (i * height) / 10 , "black");
+      printText(out, xmargin - 10 , ymargin + 4 + (i * height) / 10 
+          , String.valueOf(100 - i * 10), "end");
+    }
+    
+    if(!isMap) {
+      //print color codes for copy, sort, reduce
+      printRect(out, 14, 14, xmargin + w + 4, ymargin + 20, colors[0]);
+      printText(out, xmargin + w + 24, ymargin + 30, "copy", "start");
+      printRect(out, 14, 14, xmargin + w + 4, ymargin + 50, colors[1]);
+      printText(out, xmargin + w + 24, ymargin + 60, "sort", "start");
+      printRect(out, 14, 14, xmargin + w + 4, ymargin + 80, colors[2]);
+      printText(out, xmargin + w + 24, ymargin + 90, "reduce", "start");
+    }
+    
+    
+    //firefox curently does not support vertical text
+    //out.print("<text x=\"");out.print(6);
+    //out.print("\" y=\""); out.print(ymargin + height / 2); 
+    //out.print("\" style=\"text-anchor:middle;writing-mode:tb\">"
+    //+"Percent</text>\n");
+    
+    out.print("</svg>");
+  }
+
+  /**Computes average progress per bar*/
+  private float getMapAvarageProgress(int tasksPerBar, int index
+      , TaskReport[] reports ) {
+    float progress = 0f;
+    int k=0;
+    for(;k < tasksPerBar && index + k < reports.length; k++) { 
+      progress += reports[index + k].getProgress();
+    }
+    progress /= k;
+    return progress;
+  }
+
+  /**Computes average progresses per bar*/
+  private float[] getReduceAvarageProgresses(int tasksPerBar, int index
+      , TaskReport[] reports ) {
+    float[] progresses = new float[] {0,0,0};
+    int k=0;
+    for(;k < tasksPerBar && index + k < reports.length; k++) {
+      float progress = reports[index+k].getProgress();
+      for(int j=0; progress > 0 ; j++, progress -= oneThird) {
+        if(progress > oneThird)
+          progresses[j] += 1f;
+        else 
+          progresses[j] += progress * 3 ;
+      }
+    }
+    for(int j=0; j<3; j++) { progresses[j] /= k;}
+    
+    return progresses;
+  }
+  
+  private void printRect(PrintWriter out, int width, int height
+      , int x, int y, String color) throws IOException {
+    if(height > 0) {
+      out.print("<rect width=\"");out.print(width);
+      out.print("\" height=\"");  out.print(height);
+      out.print("\" x=\""); out.print(x);
+      out.print("\" y=\""); out.print(y);
+      out.print("\" style=\"fill:"); out.print(color);out.print("\"/>\n");
+    }
+  }
+  private void printLine(PrintWriter out, int x1, int x2
+      , int y1, int y2, String color) throws IOException {
+    out.print("<line x1=\"");out.print(x1);
+    out.print("\" x2=\"");out.print(x2);
+    out.print("\" y1=\"");out.print(y1);
+    out.print("\" y2=\""); out.print(y2);
+    out.print("\" class=\"taskgraphline\" style=\"stroke:"); 
+    out.print(color); out.print("\"/>\n"); 
+  }
+  private void printText(PrintWriter out, int x, int y, String text
+      , String anchor) throws IOException {
+    out.print("<text x=\"");out.print(String.valueOf(x));
+    out.print("\" y=\""); out.print(String.valueOf(y));
+    out.print("\" style=\"fill:black;font-family:sans-serif;" 
+        + "text-anchor:");out.print(anchor); out.print("\">");
+    out.print(text); out.print("</text>\n");
+  }
+}
+

+ 4 - 15
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -17,15 +17,10 @@
  */
  package org.apache.hadoop.mapred;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.io.PrintStream;
 import java.io.RandomAccessFile;
 import java.net.InetSocketAddress;
 import java.net.URI;
@@ -35,13 +30,13 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeMap;
-import java.util.LinkedHashMap;
 import java.util.Vector;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -59,20 +54,18 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSError;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.mapred.JobClient.TaskStatusFilter;
 import org.apache.hadoop.mapred.TaskStatus.Phase;
 import org.apache.hadoop.mapred.pipes.Submitter;
 import org.apache.hadoop.metrics.MetricsContext;
@@ -80,7 +73,6 @@ import org.apache.hadoop.metrics.MetricsException;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.Updater;
-import org.apache.hadoop.metrics.jvm.JvmMetrics;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.DiskChecker;
@@ -91,7 +83,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
-import org.apache.log4j.LogManager;
 
 /*******************************************************
  * TaskTracker is a process that starts and tracks MR Tasks
@@ -149,7 +140,7 @@ public class TaskTracker
   // The filesystem where job files are stored
   FileSystem systemFS = null;
   
-  StatusHttpServer server = null;
+  private final HttpServer server;
     
   volatile boolean shuttingDown = false;
     
@@ -583,7 +574,6 @@ public class TaskTracker
           }
           synchronized (waitingOn) {
             try {
-              int waitTime;
               if (!fetchAgain) {
                 waitingOn.wait(heartbeatInterval);
               }
@@ -623,7 +613,6 @@ public class TaskTracker
       // Note that the sync is first on fromEventId and then on allMapEvents
       synchronized (fromEventId) {
         synchronized (allMapEvents) {
-          int index = 0;
           if (allMapEvents.size() > lastKnownIndex) {
             fromEventId.set(lastKnownIndex);
             allMapEvents = allMapEvents.subList(0, lastKnownIndex);
@@ -874,7 +863,7 @@ public class TaskTracker
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     String httpBindAddress = infoSocAddr.getHostName();
     int httpPort = infoSocAddr.getPort();
-    this.server = new StatusHttpServer("task", httpBindAddress, httpPort,
+    this.server = new HttpServer("task", httpBindAddress, httpPort,
         httpPort == 0, conf);
     workerThreads = conf.getInt("tasktracker.http.threads", 40);
     this.shuffleServerMetrics = new ShuffleServerMetrics(conf);

+ 5 - 5
src/webapps/job/jobdetails.jsp

@@ -179,7 +179,7 @@
     }
 %>
 
-<%@page import="org.apache.hadoop.mapred.StatusHttpServer.TaskGraphServlet"%>
+<%@page import="org.apache.hadoop.mapred.TaskGraphServlet"%>
 <html>
 <head>
   <% 
@@ -328,8 +328,8 @@ if("off".equals(session.getAttribute("map.graph"))) { %>
 <%} else { %> 
 <a href="/jobdetails.jsp?jobid=<%=jobId%>&refresh=<%=refresh%>&map.graph=off" > close </a>
 <br><embed src="/taskgraph?type=map&jobid=<%=jobId%>" 
-       width="<%=StatusHttpServer.TaskGraphServlet.width + 2 * StatusHttpServer.TaskGraphServlet.xmargin%>" 
-       height="<%=StatusHttpServer.TaskGraphServlet.height + 3 * StatusHttpServer.TaskGraphServlet.ymargin%>"
+       width="<%=TaskGraphServlet.width + 2 * TaskGraphServlet.xmargin%>" 
+       height="<%=TaskGraphServlet.height + 3 * TaskGraphServlet.ymargin%>"
        style="width:100%" type="image/svg+xml" pluginspage="http://www.adobe.com/svg/viewer/install/" />
 <%}%>
 
@@ -341,8 +341,8 @@ if("off".equals(session.getAttribute("map.graph"))) { %>
 <a href="/jobdetails.jsp?jobid=<%=jobId%>&refresh=<%=refresh%>&reduce.graph=off" > close </a>
  
  <br><embed src="/taskgraph?type=reduce&jobid=<%=jobId%>" 
-       width="<%=StatusHttpServer.TaskGraphServlet.width + 2 * StatusHttpServer.TaskGraphServlet.xmargin%>" 
-       height="<%=StatusHttpServer.TaskGraphServlet.height + 3 * StatusHttpServer.TaskGraphServlet.ymargin%>" 
+       width="<%=TaskGraphServlet.width + 2 * TaskGraphServlet.xmargin%>" 
+       height="<%=TaskGraphServlet.height + 3 * TaskGraphServlet.ymargin%>" 
        style="width:100%" type="image/svg+xml" pluginspage="http://www.adobe.com/svg/viewer/install/" />
 <%} }%>