You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by om...@apache.org on 2009/10/27 16:44:06 UTC

svn commit: r830230 [9/9] - in /hadoop/mapreduce/branches/HDFS-641: ./ .eclipse.templates/ conf/ ivy/ lib/ src/c++/ src/contrib/ src/contrib/capacity-scheduler/ src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/ src/contrib/capacity-sche...

Modified: hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java?rev=830230&r1=830229&r2=830230&view=diff
==============================================================================
--- hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java (original)
+++ hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java Tue Oct 27 15:43:58 2009
@@ -19,6 +19,10 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.codehaus.jackson.annotate.JsonAnySetter;
 
 /**
  * A {@link LoggedTask} represents a [hadoop] task that is part of a hadoop job.
@@ -41,11 +45,26 @@
   Pre21JobHistoryConstants.Values taskStatus;
   List<LoggedTaskAttempt> attempts = new ArrayList<LoggedTaskAttempt>();
 
-  ArrayList<LoggedLocation> preferredLocations = new ArrayList<LoggedLocation>();
+  ArrayList<LoggedLocation> preferredLocations =
+      new ArrayList<LoggedLocation>();
 
   int numberMaps = -1;
   int numberReduces = -1;
 
+  static private Set<String> alreadySeenAnySetterAttributes =
+      new TreeSet<String>();
+
+  @SuppressWarnings("unused")
+  // for input parameter ignored.
+  @JsonAnySetter
+  public void setUnknownAttribute(String attributeName, Object ignored) {
+    if (!alreadySeenAnySetterAttributes.contains(attributeName)) {
+      alreadySeenAnySetterAttributes.add(attributeName);
+      System.err.println("In LoggedJob, we saw the unknown attribute "
+          + attributeName + ".");
+    }
+  }
+
   LoggedTask() {
     super();
   }
@@ -173,8 +192,9 @@
     }
   }
 
-  private void compare1(Pre21JobHistoryConstants.Values c1, Pre21JobHistoryConstants.Values c2,
-      TreePath loc, String eltname) throws DeepInequalityException {
+  private void compare1(Pre21JobHistoryConstants.Values c1,
+      Pre21JobHistoryConstants.Values c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
     if (c1 == null && c2 == null) {
       return;
     }

Modified: hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java?rev=830230&r1=830229&r2=830230&view=diff
==============================================================================
--- hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java (original)
+++ hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java Tue Oct 27 15:43:58 2009
@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.tools.rumen;
 
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.codehaus.jackson.annotate.JsonAnySetter;
+
 // HACK ALERT!!!  This "should" have have two subclasses, which might be called
 //                LoggedMapTaskAttempt and LoggedReduceTaskAttempt, but 
 //                the Jackson implementation of JSON doesn't handle a 
@@ -63,6 +68,20 @@
     super();
   }
 
+  static private Set<String> alreadySeenAnySetterAttributes =
+      new TreeSet<String>();
+
+  @SuppressWarnings("unused")
+  // for input parameter ignored.
+  @JsonAnySetter
+  public void setUnknownAttribute(String attributeName, Object ignored) {
+    if (!alreadySeenAnySetterAttributes.contains(attributeName)) {
+      alreadySeenAnySetterAttributes.add(attributeName);
+      System.err.println("In LoggedJob, we saw the unknown attribute "
+          + attributeName + ".");
+    }
+  }
+
   public long getShuffleFinished() {
     return shuffleFinished;
   }
@@ -259,8 +278,9 @@
     }
   }
 
-  private void compare1(Pre21JobHistoryConstants.Values c1, Pre21JobHistoryConstants.Values c2,
-      TreePath loc, String eltname) throws DeepInequalityException {
+  private void compare1(Pre21JobHistoryConstants.Values c1,
+      Pre21JobHistoryConstants.Values c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
     if (c1 != c2) {
       throw new DeepInequalityException(eltname + " miscompared", new TreePath(
           loc, eltname));

Modified: hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java?rev=830230&r1=830229&r2=830230&view=diff
==============================================================================
--- hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java (original)
+++ hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java Tue Oct 27 15:43:58 2009
@@ -56,6 +56,12 @@
   private final ClusterStory cluster;
   private JobConf jobConf;
 
+  private long seed;
+  private boolean hasRandomSeed = false;
+
+  private Map<LoggedDiscreteCDF, CDFRandomGenerator> interpolatorMap =
+      new HashMap<LoggedDiscreteCDF, CDFRandomGenerator>();
+
   // TODO: Fix ZombieJob to initialize this correctly from observed data
   double rackLocalOverNodeLocal = 1.5;
   double rackRemoteOverNodeLocal = 3.0;
@@ -80,6 +86,8 @@
     this.job = job;
     this.cluster = cluster;
     random = new Random(seed);
+    this.seed = seed;
+    hasRandomSeed = true;
   }
 
   /**
@@ -128,6 +136,13 @@
       Path emptyPath = new Path("/");
       int totalHosts = 0; // use to determine avg # of hosts per split.
       for (LoggedTask mapTask : job.getMapTasks()) {
+        Pre21JobHistoryConstants.Values taskType = mapTask.getTaskType();
+        if (taskType != Pre21JobHistoryConstants.Values.MAP) {
+          LOG.warn("TaskType for a MapTask is not Map. task="
+              + mapTask.getTaskID() + " type="
+              + ((taskType == null) ? "null" : taskType.toString()));
+          continue;
+        }
         List<LoggedLocation> locations = mapTask.getPreferredLocations();
         List<String> hostList = new ArrayList<String>();
         if (locations != null) {
@@ -166,20 +181,23 @@
       }
 
       int avgHostPerSplit;
-      if (splitsList.size() == 0) avgHostPerSplit = 3;
-      else {
+      if (splitsList.size() == 0) {
+        avgHostPerSplit = 3;
+      } else {
         avgHostPerSplit = totalHosts / splitsList.size();
-        if (avgHostPerSplit == 0) avgHostPerSplit = 3;
+        if (avgHostPerSplit == 0) {
+          avgHostPerSplit = 3;
+        }
       }
-      
+
       for (int i = splitsList.size(); i < totalMaps; i++) {
         if (cluster == null) {
           splitsList.add(new FileSplit(emptyPath, 0, 0, new String[0]));
         } else {
           MachineNode[] mNodes = cluster.getRandomMachines(avgHostPerSplit);
           String[] hosts = new String[mNodes.length];
-          for (int j=0; j<hosts.length; ++j) {
-            hosts[j]=mNodes[j].getName();
+          for (int j = 0; j < hosts.length; ++j) {
+            hosts[j] = mNodes[j].getName();
           }
           // TODO set size of a split to 0 now.
           splitsList.add(new FileSplit(emptyPath, 0, 0, hosts));
@@ -235,6 +253,23 @@
   }
 
   /**
+   * Getting the number of map tasks that are actually logged in the trace.
+   * @return The number of map tasks that are actually logged in the trace.
+   */
+  public int getNumLoggedMaps() {
+    return job.getMapTasks().size();
+  }
+
+
+  /**
+   * Getting the number of reduce tasks that are actually logged in the trace.
+   * @return The number of map tasks that are actually logged in the trace.
+   */
+  public int getNumLoggedReduces() {
+    return job.getReduceTasks().size();
+  }
+  
+  /**
    * Mask the job ID part in a {@link TaskID}.
    * 
    * @param taskId
@@ -262,9 +297,10 @@
         taskId.getId(), attemptId.getId());
   }
 
-
   private LoggedTask sanitizeLoggedTask(LoggedTask task) {
-    if (task == null) return null;
+    if (task == null) {
+      return null;
+    }
     if (task.getTaskType() == null) {
       LOG.warn("Task " + task.getTaskID() + " has nulll TaskType");
       return null;
@@ -275,17 +311,19 @@
     }
     return task;
   }
-  
+
   private LoggedTaskAttempt sanitizeLoggedTaskAttempt(LoggedTaskAttempt attempt) {
-    if (attempt == null) return null;
+    if (attempt == null) {
+      return null;
+    }
     if (attempt.getResult() == null) {
       LOG.warn("TaskAttempt " + attempt.getResult() + " has nulll Result");
       return null;
     }
-    
+
     return attempt;
   }
-  
+
   /**
    * Build task mapping and task attempt mapping, to be later used to find
    * information of a particular {@link TaskID} or {@link TaskAttemptID}.
@@ -414,7 +452,6 @@
       return makeUpTaskAttemptInfo(taskType, taskInfo, taskAttemptNumber,
           taskNumber, locality);
     }
-
     LoggedTaskAttempt loggedAttempt = getLoggedTaskAttempt(taskType,
         taskNumber, taskAttemptNumber);
     if (loggedAttempt == null) {
@@ -489,9 +526,13 @@
   private int getLocality(LoggedTask loggedTask, LoggedTaskAttempt loggedAttempt) {
     int distance = cluster.getMaximumDistance();
     String rackHostName = loggedAttempt.getHostName();
-    if (rackHostName == null) return distance;
+    if (rackHostName == null) {
+      return distance;
+    }
     MachineNode mn = getMachineNode(rackHostName);
-    if (mn == null) return distance;
+    if (mn == null) {
+      return distance;
+    }
     List<LoggedLocation> locations = loggedTask.getPreferredLocations();
     if (locations != null) {
       for (LoggedLocation location : locations) {
@@ -578,7 +619,8 @@
     Values type = loggedTask.getTaskType();
     if ((type != Values.MAP) && (type != Values.REDUCE)) {
       throw new IllegalArgumentException(
-          "getTaskInfo only supports MAP or REDUCE tasks: " + type.toString());
+          "getTaskInfo only supports MAP or REDUCE tasks: " + type.toString() +
+          " for task = " + loggedTask.getTaskID());
     }
 
     for (LoggedTaskAttempt attempt : attempts) {
@@ -672,40 +714,33 @@
             "state is neither SUCCEEDED nor FAILED: " + state);
       }
       return reduceTime;
-    } catch (IllegalArgumentException e) {
-      if (e.getMessage().startsWith("no value to use to make up runtime")) {
-        return 0;
-      }
-      throw e;
+    } catch (NoValueToMakeUpRuntime e) {
+      return 0;
     }
   }
 
   private long makeUpMapRuntime(State state, int locality) {
     long runtime;
     // make up runtime
-    if (state == State.SUCCEEDED) {
-      /**
-       * MapCDFs is a ArrayList of 4 possible groups: distance=0, 1, 2, and
-       * the last group is "distance cannot be determined". All pig jobs would
-       * have only the 4th group, and pig tasks usually do not have any
-       * locality, so this group should count as "distance=2". However,
-       * setup/cleanup tasks are also counted in the 4th group. These tasks do
-       * not make sense.
-       */
-      runtime = makeUpRuntime(job.getSuccessfulMapAttemptCDFs().get(locality));
-      if (runtime < 0) {
-        runtime = makeUpRuntime(job.getSuccessfulMapAttemptCDFs());
-      }
-    } else if (state == State.FAILED) {
-      runtime = makeUpRuntime(job.getFailedMapAttemptCDFs().get(locality));
-      if (runtime < 0) {
-        runtime = makeUpRuntime(job.getFailedMapAttemptCDFs());
-      }  
+    if (state == State.SUCCEEDED || state == State.FAILED) {
+      List<LoggedDiscreteCDF> cdfList =
+          state == State.SUCCEEDED ? job.getSuccessfulMapAttemptCDFs() : job
+              .getFailedMapAttemptCDFs();
+      // XXX MapCDFs is a ArrayList of 4 possible groups: distance=0, 1, 2, and
+      // the last group is "distance cannot be determined". All pig jobs
+      // would have only the 4th group, and pig tasks usually do not have
+      // any locality, so this group should count as "distance=2".
+      // However, setup/cleanup tasks are also counted in the 4th group.
+      // These tasks do not make sense.
+      try {
+        runtime = makeUpRuntime(cdfList.get(locality));
+      } catch (NoValueToMakeUpRuntime e) {
+        runtime = makeUpRuntime(cdfList);
+      }
     } else {
       throw new IllegalArgumentException(
           "state is neither SUCCEEDED nor FAILED: " + state);
     }
-    
     return runtime;
   }
 
@@ -717,7 +752,7 @@
    *          A list of CDFs for the distribution of runtime for the 1st, 2nd,
    *          ... map attempts for the job.
    */
-  private long makeUpRuntime(ArrayList<LoggedDiscreteCDF> mapAttemptCDFs) {
+  private long makeUpRuntime(List<LoggedDiscreteCDF> mapAttemptCDFs) {
     int total = 0;
     for (LoggedDiscreteCDF cdf : mapAttemptCDFs) {
       total += cdf.getNumberValues();
@@ -739,39 +774,72 @@
     throw new IllegalStateException("not possible to get here");
   }
 
-  // return -1 if we fail to makeup runtime with available info.
   private long makeUpRuntime(LoggedDiscreteCDF loggedDiscreteCDF) {
-    ArrayList<LoggedSingleRelativeRanking> rankings = new ArrayList<LoggedSingleRelativeRanking>(
-        loggedDiscreteCDF.getRankings());
-    if (loggedDiscreteCDF.getNumberValues() == 0) {
-      return -1;
+    /*
+     * We need this odd-looking code because if a seed exists we need to ensure
+     * that only one interpolator is generated per LoggedDiscreteCDF, but if no
+     * seed exists then the potentially lengthy process of making an
+     * interpolator can happen outside the lock. makeUpRuntimeCore only locks
+     * around the two hash map accesses.
+     */
+    if (hasRandomSeed) {
+      synchronized (interpolatorMap) {
+        return makeUpRuntimeCore(loggedDiscreteCDF);
+      }
     }
 
-    LoggedSingleRelativeRanking ranking = new LoggedSingleRelativeRanking();
-    ranking.setDatum(loggedDiscreteCDF.getMaximum());
-    ranking.setRelativeRanking(1.0);
-    rankings.add(ranking);
-
-    ranking = new LoggedSingleRelativeRanking();
-    ranking.setDatum(loggedDiscreteCDF.getMinimum());
-    ranking.setRelativeRanking(0.0);
-    rankings.add(0, ranking);
-
-    double r = random.nextDouble();
-    LoggedSingleRelativeRanking prevRanking = rankings.get(0);
-    for (LoggedSingleRelativeRanking ranking2 : rankings) {
-      double r2 = ranking2.getRelativeRanking();
-      if (r < r2) {
-        double r1 = prevRanking.getRelativeRanking();
-        double f1 = prevRanking.getDatum();
-        double f2 = ranking2.getDatum();
-        double runtime = (r - r1) / (r2 - r1) * (f2 - f1) + f1;
-        return (long) runtime;
+    return makeUpRuntimeCore(loggedDiscreteCDF);
+  }
+
+  private long makeUpRuntimeCore(LoggedDiscreteCDF loggedDiscreteCDF) {
+    CDFRandomGenerator interpolator;
+
+    synchronized (interpolatorMap) {
+      interpolator = interpolatorMap.get(loggedDiscreteCDF);
+    }
+
+    if (interpolator == null) {
+      if (loggedDiscreteCDF.getNumberValues() == 0) {
+        throw new NoValueToMakeUpRuntime("no value to use to make up runtime");
+      }
+
+      interpolator =
+          hasRandomSeed ? new CDFPiecewiseLinearRandomGenerator(
+              loggedDiscreteCDF, ++seed)
+              : new CDFPiecewiseLinearRandomGenerator(loggedDiscreteCDF);
+
+      /*
+       * It doesn't matter if we compute and store an interpolator twice because
+       * the two instances will be semantically identical and stateless, unless
+       * we're seeded, in which case we're not stateless but this code will be
+       * called synchronizedly.
+       */
+      synchronized (interpolatorMap) {
+        interpolatorMap.put(loggedDiscreteCDF, interpolator);
       }
-      prevRanking = ranking2;
     }
 
-    return rankings.get(rankings.size() - 1).getDatum();
+    return interpolator.randomValue();
+  }
+
+  static private class NoValueToMakeUpRuntime extends IllegalArgumentException {
+    static final long serialVersionUID = 1L;
+
+    NoValueToMakeUpRuntime() {
+      super();
+    }
+
+    NoValueToMakeUpRuntime(String detailMessage) {
+      super(detailMessage);
+    }
+
+    NoValueToMakeUpRuntime(String detailMessage, Throwable cause) {
+      super(detailMessage, cause);
+    }
+
+    NoValueToMakeUpRuntime(Throwable cause) {
+      super(cause);
+    }
   }
 
   private State makeUpState(int taskAttemptNumber, double[] numAttempts) {

Modified: hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/ZombieJobProducer.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/ZombieJobProducer.java?rev=830230&r1=830229&r2=830230&view=diff
==============================================================================
--- hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/ZombieJobProducer.java (original)
+++ hadoop/mapreduce/branches/HDFS-641/src/tools/org/apache/hadoop/tools/rumen/ZombieJobProducer.java Tue Oct 27 15:43:58 2009
@@ -69,7 +69,7 @@
   }
 
   @Override
-  public JobStory getNextJob() throws IOException {
+  public ZombieJob getNextJob() throws IOException {
     LoggedJob job = reader.getNext();
     return (job == null) ? null : new ZombieJob(job, cluster);
   }

Propchange: hadoop/mapreduce/branches/HDFS-641/src/webapps/job/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Tue Oct 27 15:43:58 2009
@@ -1,3 +1,3 @@
 /hadoop/core/branches/branch-0.19/mapred/src/webapps/job:713112
 /hadoop/core/trunk/src/webapps/job:776175-785643
-/hadoop/mapreduce/trunk/src/webapps/job:817879-818559
+/hadoop/mapreduce/trunk/src/webapps/job:817878-830225

Modified: hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtable.jsp
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtable.jsp?rev=830230&r1=830229&r2=830230&view=diff
==============================================================================
--- hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtable.jsp (original)
+++ hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtable.jsp Tue Oct 27 15:43:58 2009
@@ -32,7 +32,7 @@
 <%
 JobTracker tracker = (JobTracker) application.getAttribute("job.tracker");
 QueueManager qmgr = tracker.getQueueManager();
-String queue = (String)request.getParameter("queue_name");
+String queue = request.getParameter("queue_name");
 TaskScheduler scheduler = tracker.getTaskScheduler();
 JobQueueInfo queueInfo = tracker.getQueueInfo(queue);
 %>

Modified: hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtracker.jsp
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtracker.jsp?rev=830230&r1=830229&r2=830230&view=diff
==============================================================================
--- hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtracker.jsp (original)
+++ hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtracker.jsp Tue Oct 27 15:43:58 2009
@@ -25,6 +25,7 @@
   import="java.util.*"
   import="java.text.DecimalFormat"
   import="org.apache.hadoop.mapred.*"
+  import="org.apache.hadoop.mapreduce.*"
   import="org.apache.hadoop.util.*"
 %>
 <%!	private static final long serialVersionUID = 1L;
@@ -32,6 +33,7 @@
 <%
   JobTracker tracker = (JobTracker) application.getAttribute("job.tracker");
   ClusterStatus status = tracker.getClusterStatus();
+  ClusterMetrics metrics = tracker.getClusterMetrics();
   String trackerName = 
            StringUtils.simpleHostname(tracker.getJobTrackerMachine());
   JobQueueInfo[] queues = tracker.getJobQueues();
@@ -42,34 +44,41 @@
 <%!
   private static DecimalFormat percentFormat = new DecimalFormat("##0.00");
   
-  public void generateSummaryTable(JspWriter out, ClusterStatus status,
+  public void generateSummaryTable(JspWriter out, ClusterMetrics metrics,
                                    JobTracker tracker) throws IOException {
-    String tasksPerNode = status.getTaskTrackers() > 0 ?
-      percentFormat.format(((double)(status.getMaxMapTasks() +
-                      status.getMaxReduceTasks())) / status.getTaskTrackers()):
+    String tasksPerNode = metrics.getTaskTrackerCount() > 0 ?
+      percentFormat.format(((double)(metrics.getMapSlotCapacity() +
+      metrics.getReduceSlotCapacity())) / metrics.getTaskTrackerCount()):
       "-";
     out.print("<table border=\"1\" cellpadding=\"5\" cellspacing=\"0\">\n"+
               "<tr><th>Queues</th>" +
-              "<th>Maps</th><th>Reduces</th>" + 
+              "<th>Running Map Tasks</th><th>Running Reduce Tasks</th>" + 
               "<th>Total Submissions</th>" +
-              "<th>Nodes</th><th>Map Task Capacity</th>" +
-              "<th>Reduce Task Capacity</th><th>Avg. Tasks/Node</th>" + 
+              "<th>Nodes</th>" +
+              "<th>Occupied Map Slots</th><th>Occupied Reduce Slots</th>" + 
+              "<th>Reserved Map Slots</th><th>Reserved Reduce Slots</th>" + 
+              "<th>Map Slot Capacity</th>" +
+              "<th>Reduce Slot Capacity</th><th>Avg. Slots/Node</th>" + 
               "<th>Blacklisted Nodes</th>" +
               "<th>Excluded Nodes</th></tr>\n");
     out.print("<tr><td><a href=\"queueinfo.jsp\">" +
               tracker.getRootQueues().length + "</a></td><td>" + 
-              status.getMapTasks() + "</td><td>" +
-              status.getReduceTasks() + "</td><td>" + 
-              tracker.getTotalSubmissions() +
+              metrics.getRunningMaps() + "</td><td>" +
+              metrics.getRunningReduces() + "</td><td>" + 
+              metrics.getTotalJobSubmissions() +
               "</td><td><a href=\"machines.jsp?type=active\">" +
-              status.getTaskTrackers() +
-              "</a></td><td>" + status.getMaxMapTasks() +
-              "</td><td>" + status.getMaxReduceTasks() +
+              metrics.getTaskTrackerCount() + "</a></td><td>" + 
+              metrics.getOccupiedMapSlots() + "</td><td>" +
+              metrics.getOccupiedReduceSlots() + "</td><td>" + 
+              metrics.getReservedMapSlots() + "</td><td>" +
+              metrics.getReservedReduceSlots() + "</td><td>" + 
+              + metrics.getMapSlotCapacity() +
+              "</td><td>" + metrics.getReduceSlotCapacity() +
               "</td><td>" + tasksPerNode +
               "</td><td><a href=\"machines.jsp?type=blacklisted\">" +
-              status.getBlacklistedTrackers() + "</a>" +
+              metrics.getBlackListedTaskTrackerCount() + "</a>" +
               "</td><td><a href=\"machines.jsp?type=excluded\">" +
-              status.getNumExcludedNodes() + "</a>" +
+              metrics.getDecommissionedTaskTrackerCount() + "</a>" +
               "</td></tr></table>\n");
 
     out.print("<br>");
@@ -120,7 +129,7 @@
 <hr>
 <h2>Cluster Summary (Heap Size is <%= StringUtils.byteDesc(status.getUsedMemory()) %>/<%= StringUtils.byteDesc(status.getMaxMemory()) %>)</h2>
 <% 
- generateSummaryTable(out, status, tracker); 
+ generateSummaryTable(out, metrics, tracker); 
 %>
 <hr>
 <b>Filter (Jobid, Priority, User, Name)</b> <input type="text" id="filter" onkeyup="applyfilter()"> <br>

Modified: hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtracker.jspx
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtracker.jspx?rev=830230&r1=830229&r2=830230&view=diff
==============================================================================
--- hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtracker.jspx (original)
+++ hadoop/mapreduce/branches/HDFS-641/src/webapps/job/jobtracker.jspx Tue Oct 27 15:43:58 2009
@@ -27,6 +27,10 @@
   <jsp:directive.page import="org.apache.hadoop.mapreduce.*" />
   <jsp:directive.page import="org.apache.hadoop.mapred.JSPUtil" />
 
+  <jsp:declaration>
+  private static final long serialVersionUID = 1L;
+  </jsp:declaration>
+
   <jsp:scriptlet>
     response.setHeader("Pragma", "no-cache");
     response.setHeader("Cache-Control", "no-store");