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 yh...@apache.org on 2009/07/31 11:15:35 UTC

svn commit: r799551 - in /hadoop/mapreduce/trunk: ./ src/java/org/apache/hadoop/mapred/ src/test/mapred/org/apache/hadoop/mapred/ src/webapps/job/

Author: yhemanth
Date: Fri Jul 31 09:15:34 2009
New Revision: 799551

URL: http://svn.apache.org/viewvc?rev=799551&view=rev
Log:
MAPREDUCE-766. Enhanced list-blacklisted-trackers to display reasons for blacklisting a node. Contributed by Sreekanth Ramakrishnan.

Modified:
    hadoop/mapreduce/trunk/CHANGES.txt
    hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/ClusterStatus.java
    hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobClient.java
    hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java
    hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java
    hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerBlacklisting.java
    hadoop/mapreduce/trunk/src/webapps/job/machines.jsp

Modified: hadoop/mapreduce/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/CHANGES.txt?rev=799551&r1=799550&r2=799551&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/CHANGES.txt (original)
+++ hadoop/mapreduce/trunk/CHANGES.txt Fri Jul 31 09:15:34 2009
@@ -14,6 +14,9 @@
     MAPREDUCE-245. Change Job and jobcontrol classes to use the List interface
     rather than ArrayList in APIs. (Tom White via cdouglas)
 
+    MAPREDUCE-766. Enhanced list-blacklisted-trackers to display reasons
+    for blacklisting a node. (Sreekanth Ramakrishnan via yhemanth)
+
   NEW FEATURES
 
     MAPREDUCE-546. Provide sample fair scheduler config file in conf/ and use

Modified: hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/ClusterStatus.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/ClusterStatus.java?rev=799551&r1=799550&r2=799551&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/ClusterStatus.java (original)
+++ hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/ClusterStatus.java Fri Jul 31 09:15:34 2009
@@ -48,6 +48,9 @@
  *   <li>
  *   State of the <code>JobTracker</code>.
  *   </li>
+ *   <li>
+ *   Details regarding black listed trackers.
+ *   </li>
  * </ol></p>
  * 
  * <p>Clients can query for the latest <code>ClusterStatus</code>, via 
@@ -56,10 +59,116 @@
  * @see JobClient
  */
 public class ClusterStatus implements Writable {
+  /**
+   * Class which encapsulates information about a blacklisted tasktracker.
+   *  
+   * The information includes the tasktracker's name and reasons for
+   * getting blacklisted. The toString method of the class will print
+   * the information in a whitespace separated fashion to enable parsing.
+   */
+  public static class BlackListInfo implements Writable {
+
+    private String trackerName;
+
+    private String reasonForBlackListing;
+    
+    private String blackListReport;
+    
+    BlackListInfo() {
+    }
+    
+
+    /**
+     * Gets the blacklisted tasktracker's name.
+     * 
+     * @return tracker's name.
+     */
+    public String getTrackerName() {
+      return trackerName;
+    }
 
+    /**
+     * Gets the reason for which the tasktracker was blacklisted.
+     * 
+     * @return reason which tracker was blacklisted
+     */
+    public String getReasonForBlackListing() {
+      return reasonForBlackListing;
+    }
+
+    /**
+     * Sets the blacklisted tasktracker's name.
+     * 
+     * @param trackerName of the tracker.
+     */
+    void setTrackerName(String trackerName) {
+      this.trackerName = trackerName;
+    }
+
+    /**
+     * Sets the reason for which the tasktracker was blacklisted.
+     * 
+     * @param reasonForBlackListing
+     */
+    void setReasonForBlackListing(String reasonForBlackListing) {
+      this.reasonForBlackListing = reasonForBlackListing;
+    }
+
+    /**
+     * Gets a descriptive report about why the tasktracker was blacklisted.
+     * 
+     * @return report describing why the tasktracker was blacklisted.
+     */
+    public String getBlackListReport() {
+      return blackListReport;
+    }
+
+    /**
+     * Sets a descriptive report about why the tasktracker was blacklisted.
+     * @param blackListReport report describing why the tasktracker 
+     *                        was blacklisted.
+     */
+    void setBlackListReport(String blackListReport) {
+      this.blackListReport = blackListReport;
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      trackerName = Text.readString(in);
+      reasonForBlackListing = Text.readString(in);
+      blackListReport = Text.readString(in);
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      Text.writeString(out, trackerName);
+      Text.writeString(out, reasonForBlackListing);
+      Text.writeString(out, blackListReport);
+    }
+
+    @Override
+    /**
+     * Print information related to the blacklisted tasktracker in a
+     * whitespace separated fashion.
+     * 
+     * The method changes any newlines in the report describing why
+     * the tasktracker was blacklisted to a ':' for enabling better
+     * parsing.
+     */
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append(trackerName);
+      sb.append("\t");
+      sb.append(reasonForBlackListing);
+      sb.append("\t");
+      sb.append(blackListReport.replace("\n", ":"));
+      return sb.toString();
+    }
+    
+  }
+  
   private int numActiveTrackers;
   private Collection<String> activeTrackers = new ArrayList<String>();
-  private Collection<String> blacklistedTrackers = new ArrayList<String>();
   private int numBlacklistedTrackers;
   private int numExcludedNodes;
   private long ttExpiryInterval;
@@ -70,6 +179,8 @@
   private JobTracker.State state;
   private long used_memory;
   private long max_memory;
+  private Collection<BlackListInfo> blacklistedTrackersInfo =
+    new ArrayList<BlackListInfo>();
 
   ClusterStatus() {}
   
@@ -134,7 +245,7 @@
    * @param state the {@link JobTracker.State} of the <code>JobTracker</code>
    */
   ClusterStatus(Collection<String> activeTrackers, 
-      Collection<String> blacklistedTrackers,
+      Collection<BlackListInfo> blacklistedTrackers,
       long ttExpiryInterval,
       int maps, int reduces, int maxMaps, int maxReduces, 
       JobTracker.State state) {
@@ -142,11 +253,13 @@
          maxMaps, maxReduces, state, 0);
   }
 
+
   /**
    * Construct a new cluster status.
    * 
    * @param activeTrackers active tasktrackers in the cluster
-   * @param blacklistedTrackers blacklisted tasktrackers in the cluster
+   * @param blackListedTrackerInfo blacklisted tasktrackers information 
+   * in the cluster
    * @param ttExpiryInterval the tasktracker expiry interval
    * @param maps no. of currently running map-tasks in the cluster
    * @param reduces no. of currently running reduce-tasks in the cluster
@@ -155,17 +268,18 @@
    * @param state the {@link JobTracker.State} of the <code>JobTracker</code>
    * @param numDecommissionNodes number of decommission trackers
    */
-  ClusterStatus(Collection<String> activeTrackers, 
-                Collection<String> blacklistedTrackers, long ttExpiryInterval,
-                int maps, int reduces, int maxMaps, int maxReduces, 
-                JobTracker.State state, int numDecommissionNodes) {
-    this(activeTrackers.size(), blacklistedTrackers.size(), ttExpiryInterval, 
-        maps, reduces, maxMaps, maxReduces, state, numDecommissionNodes);
+  
+  ClusterStatus(Collection<String> activeTrackers,
+      Collection<BlackListInfo> blackListedTrackerInfo, long ttExpiryInterval,
+      int maps, int reduces, int maxMaps, int maxReduces,
+      JobTracker.State state, int numDecommissionNodes) {
+    this(activeTrackers.size(), blackListedTrackerInfo.size(),
+        ttExpiryInterval, maps, reduces, maxMaps, maxReduces, state,
+        numDecommissionNodes);
     this.activeTrackers = activeTrackers;
-    this.blacklistedTrackers = blacklistedTrackers;
+    this.blacklistedTrackersInfo = blackListedTrackerInfo;
   }
 
-
   /**
    * Get the number of task trackers in the cluster.
    * 
@@ -190,6 +304,10 @@
    * @return the blacklisted task trackers in the cluster.
    */
   public Collection<String> getBlacklistedTrackerNames() {
+    ArrayList<String> blacklistedTrackers = new ArrayList<String>();
+    for(BlackListInfo bi : blacklistedTrackersInfo) {
+      blacklistedTrackers.add(bi.getTrackerName());
+    }
     return blacklistedTrackers;
   }
   
@@ -281,6 +399,16 @@
   public long getMaxMemory() {
     return max_memory;
   }
+  
+  /**
+   * Gets the list of blacklisted trackers along with reasons for blacklisting.
+   * 
+   * @return the collection of {@link BlackListInfo} objects. 
+   * 
+   */
+  public Collection<BlackListInfo> getBlackListedTrackersInfo() {
+    return blacklistedTrackersInfo;
+  }
 
   public void write(DataOutput out) throws IOException {
     if (activeTrackers.size() == 0) {
@@ -293,14 +421,14 @@
         Text.writeString(out, tracker);
       }
     }
-    if (blacklistedTrackers.size() == 0) {
+    if (blacklistedTrackersInfo.size() == 0) {
       out.writeInt(numBlacklistedTrackers);
-      out.writeInt(0);
+      out.writeInt(blacklistedTrackersInfo.size());
     } else {
-      out.writeInt(blacklistedTrackers.size());
-      out.writeInt(blacklistedTrackers.size());
-      for (String tracker : blacklistedTrackers) {
-        Text.writeString(out, tracker);
+      out.writeInt(blacklistedTrackersInfo.size());
+      out.writeInt(blacklistedTrackersInfo.size());
+      for (BlackListInfo tracker : blacklistedTrackersInfo) {
+        tracker.write(out);
       }
     }
     out.writeInt(numExcludedNodes);
@@ -324,11 +452,12 @@
       }
     }
     numBlacklistedTrackers = in.readInt();
-    numTrackerNames = in.readInt();
-    if (numTrackerNames > 0) {
-      for (int i = 0; i < numTrackerNames; i++) {
-        String name = Text.readString(in);
-        blacklistedTrackers.add(name);
+    int blackListTrackerInfoSize = in.readInt();
+    if(blackListTrackerInfoSize > 0) {
+      for (int i = 0; i < blackListTrackerInfoSize; i++) {
+        BlackListInfo info = new BlackListInfo();
+        info.readFields(in);
+        blacklistedTrackersInfo.add(info);
       }
     }
     numExcludedNodes = in.readInt();

Modified: hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobClient.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobClient.java?rev=799551&r1=799550&r2=799551&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobClient.java (original)
+++ hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobClient.java Fri Jul 31 09:15:34 2009
@@ -63,6 +63,7 @@
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.io.serializer.Serializer;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.mapred.ClusterStatus.BlackListInfo;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.Counters.Group;
 import org.apache.hadoop.net.NetUtils;
@@ -1838,9 +1839,12 @@
    */
   private void listBlacklistedTrackers() throws IOException {
     ClusterStatus c = jobSubmitClient.getClusterStatus(true);
-    Collection<String> trackers = c.getBlacklistedTrackerNames();
-    for (String trackerName : trackers) {
-      System.out.println(trackerName);
+    Collection<BlackListInfo> trackers = c.getBlackListedTrackersInfo();
+    if(trackers.size() > 0) {
+      System.out.println("BlackListedNode \t Reason \t Report");
+    }
+    for (BlackListInfo tracker : trackers) {
+      System.out.println(tracker.toString());
     }
   }
 

Modified: hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java?rev=799551&r1=799550&r2=799551&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java (original)
+++ hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java Fri Jul 31 09:15:34 2009
@@ -62,9 +62,11 @@
    * Version 22: Added method getQueueAclsForCurrentUser to get queue acls info
    *             for a user
    * Version 23: Modified the JobQueueInfo class to inlucde queue state.
-   *             Part of HADOOP-5913.            
+   *             Part of HADOOP-5913.  
+   * Version 24: Modified ClusterStatus to include BlackListInfo class which 
+   *             encapsulates reasons and report for blacklisted node.          
    */
-  public static final long versionID = 23L;
+  public static final long versionID = 24L;
 
   /**
    * Allocate a name for the job.

Modified: hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java?rev=799551&r1=799550&r2=799551&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java (original)
+++ hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java Fri Jul 31 09:15:34 2009
@@ -60,6 +60,7 @@
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
+import org.apache.hadoop.mapred.ClusterStatus.BlackListInfo;
 import org.apache.hadoop.mapred.JobHistory.Keys;
 import org.apache.hadoop.mapred.JobHistory.Listener;
 import org.apache.hadoop.mapred.JobHistory.Values;
@@ -552,6 +553,9 @@
         sb.append(reasons);
         sb.append("\n");
       }
+      if (sb.length() > 0) {
+        sb.replace(sb.length()-1, sb.length(), "");
+      }
       return sb.toString();
     }
     
@@ -3441,8 +3445,9 @@
     synchronized (taskTrackers) {
       if (detailed) {
         List<List<String>> trackerNames = taskTrackerNames();
+        Collection<BlackListInfo> blackListedTrackers = getBlackListedTrackers();
         return new ClusterStatus(trackerNames.get(0),
-            trackerNames.get(1),
+            blackListedTrackers,
             tasktrackerExpiryInterval,
             totalMaps,
             totalReduces,
@@ -4257,7 +4262,7 @@
     }
   }
   
-  String getReasonsForBlacklisting(String host) {
+  String getFaultReport(String host) {
     FaultInfo fi = faultyTrackers.getFaultInfo(host, false);
     if (fi == null) {
       return "";
@@ -4265,7 +4270,6 @@
     return fi.getTrackerFaultReport();
   }
 
-  /** Test Methods */
   Set<ReasonForBlackListing> getReasonForBlackList(String host) {
     FaultInfo fi = faultyTrackers.getFaultInfo(host, false);
     if (fi == null) {
@@ -4274,7 +4278,36 @@
     return fi.getReasonforblacklisting();
   }
   
+  Collection<BlackListInfo> getBlackListedTrackers() {
+    Collection<BlackListInfo> blackListedTrackers = 
+      new ArrayList<BlackListInfo>();
+    for(TaskTrackerStatus tracker : blacklistedTaskTrackers()) {
+      String hostName = tracker.getHost();
+      BlackListInfo bi = new BlackListInfo();
+      bi.setTrackerName(tracker.getTrackerName());
+      Set<ReasonForBlackListing> rfbs = 
+        getReasonForBlackList(hostName);
+      StringBuffer sb = new StringBuffer();
+      for(ReasonForBlackListing rfb : rfbs) {
+        sb.append(rfb.toString());
+        sb.append(",");
+      }
+      if (sb.length() > 0) {
+        sb.replace(sb.length()-1, sb.length(), "");
+      }
+      bi.setReasonForBlackListing(sb.toString());
+      bi.setBlackListReport(
+          getFaultReport(hostName));
+      blackListedTrackers.add(bi);
+    }
+    return blackListedTrackers;
+  }
+  
+  /** Test method to increment the fault*/
+  
   void incrementFaults(String hostName) {
     faultyTrackers.incrementFaults(hostName);
   }
+  
+  
 }

Modified: hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerBlacklisting.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerBlacklisting.java?rev=799551&r1=799550&r2=799551&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerBlacklisting.java (original)
+++ hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerBlacklisting.java Fri Jul 31 09:15:34 2009
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -30,8 +31,8 @@
 import junit.framework.TestCase;
 import junit.framework.TestSuite;
 
+import org.apache.hadoop.mapred.ClusterStatus.BlackListInfo;
 import org.apache.hadoop.mapred.JobTracker.ReasonForBlackListing;
-import org.apache.hadoop.mapred.TaskStatus.Phase;
 import org.apache.hadoop.mapred.TaskTrackerStatus.TaskTrackerHealthStatus;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
@@ -254,9 +255,9 @@
     checkReasonForBlackListing(hosts[2], nodeUnHealthyReasonSet);
     for (int i = 0; i < hosts.length; i++) {
       //Replace new line as we are adding new line
-      //in getReasonsForBlacklisting
+      //in getFaultReport
       assertEquals("Blacklisting reason string not correct for host " + i,
-          jobTracker.getReasonsForBlacklisting(hosts[i]).replace("\n", ""),
+          jobTracker.getFaultReport(hosts[i]).replace("\n", ""),
           error);
     }
     status.setNodeHealthy(false);
@@ -268,15 +269,15 @@
     checkReasonForBlackListing(hosts[2], nodeUnHealthyReasonSet);
     for (int i = 0; i < hosts.length; i++) {
       //Replace new line as we are adding new line
-      //in getReasonsForBlacklisting
+      //in getFaultReport
       assertEquals("Blacklisting reason string not correct for host " + i,
-          jobTracker.getReasonsForBlacklisting(hosts[i]).replace("\n", ""),
+          jobTracker.getFaultReport(hosts[i]).replace("\n", ""),
           error1);
     }
     //clear the blacklisted trackers with node health reasons.
     sendHeartBeat(null, false);
   }
-
+  
   private TaskTrackerHealthStatus getUnhealthyNodeStatus(String error) {
     TaskTrackerHealthStatus status = new TaskTrackerHealthStatus();
     status.setNodeHealthy(false);
@@ -322,9 +323,78 @@
         .getNumReservedTaskTrackersForMaps());
     assertEquals("Tracker 1 not unreserved for the job 1", 0, job
         .getNumReservedTaskTrackersForReduces());
-    //white list all trackers for health reasons
+    //white list all trackers for health reasons and failure counts
+    clock.jumpADay = true;
+    sendHeartBeat(null, false);
+  }
+  
+  /**
+   * Test case to test if the cluster status is populated with the right
+   * blacklist information, which would be used by the {@link JobClient} to
+   * display information on the Command Line interface.
+   * 
+   */
+  public void testClusterStatusBlacklistedReason() throws Exception {
+    String error = "ERROR";
+    String errorWithNewLines = "ERROR\nERROR";
+    String expectedErrorReport = "ERROR:ERROR";
+    // Create an unhealthy tracker health status.
+    Collection<BlackListInfo> blackListedTrackerInfo = jobTracker
+        .getBlackListedTrackers();
+
+    assertTrue("The blacklisted tracker nodes is not empty.",
+        blackListedTrackerInfo.isEmpty());
+
+    TaskTrackerHealthStatus status = getUnhealthyNodeStatus(errorWithNewLines);
+    // make all tracker unhealthy
+    sendHeartBeat(status, false);
+    assertEquals("All trackers not blacklisted", jobTracker
+        .getBlacklistedTrackerCount(), 3);
+    // Verify the new method .getBlackListedTracker() which is
+    // used by the ClusterStatus to set the list of blacklisted
+    // tracker.
+    blackListedTrackerInfo = jobTracker.getBlackListedTrackers();
+
+    // Check if all the black listed tracker information is obtained
+    // in new method.
+    assertEquals("Blacklist tracker info does not contain all trackers", 3,
+        blackListedTrackerInfo.size());
+    // verify all the trackers are blacklisted for health reasons.
+    // Also check the health report.
+    for (BlackListInfo bi : blackListedTrackerInfo) {
+      assertEquals("Tracker not blacklisted for health reason",
+          ReasonForBlackListing.NODE_UNHEALTHY.toString().trim(), bi
+              .getReasonForBlackListing().trim());
+      assertTrue("Tracker blacklist report does not match", 
+          bi.toString().endsWith(expectedErrorReport));
+    }
+    // reset the tracker health status back to normal.
+    sendHeartBeat(null, false);
+    runBlackListingJob(jobTracker, trackers);
+    sendHeartBeat(status, false);
+    blackListedTrackerInfo = jobTracker.getBlackListedTrackers();
+    for (BlackListInfo bi : blackListedTrackerInfo) {
+      if (bi.getTrackerName().equals(trackers[0])) {
+        assertTrue(
+            "Reason for blacklisting of tracker 1 does not contain Unhealthy reasons",
+            bi.getReasonForBlackListing().contains(
+                ReasonForBlackListing.NODE_UNHEALTHY.toString().trim()));
+        assertTrue(
+            "Reason for blacklisting of tracker 1 does not contain Unhealthy reasons",
+            bi.getReasonForBlackListing().contains(
+                ReasonForBlackListing.EXCEEDING_FAILURES.toString().trim()));
+        assertTrue("Blacklist failure does not contain failure report string",
+            bi.getBlackListReport().contains("failures on the tracker"));
+      } else {
+        assertEquals("Tracker not blacklisted for health reason",
+            ReasonForBlackListing.NODE_UNHEALTHY.toString().trim(), bi
+                .getReasonForBlackListing().trim());
+      }
+      assertTrue("Tracker blacklist report does not match", bi
+          .getBlackListReport().trim().contains(error));
+    }
+    clock.jumpADay = true;
     sendHeartBeat(null, false);
-    clock.jumpADay = false;
   }
 
   /**
@@ -333,7 +403,7 @@
    * 
    * @param jobTracker JobTracker instance
    * @param trackers array of trackers, the method would blacklist
-   * first element of the arry
+   * first element of the array
    * @return A job in progress object.
    * @throws Exception
    */

Modified: hadoop/mapreduce/trunk/src/webapps/job/machines.jsp
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/webapps/job/machines.jsp?rev=799551&r1=799550&r2=799551&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/webapps/job/machines.jsp (original)
+++ hadoop/mapreduce/trunk/src/webapps/job/machines.jsp Fri Jul 31 09:15:34 2009
@@ -117,7 +117,7 @@
                   "</td><td>" + healthString +
                   "</td><td>" + sinceHealthCheck); 
         if(type.equals("blacklisted")) {
-          out.print("</td><td>" + tracker.getReasonsForBlacklisting(tt.getHost()));
+          out.print("</td><td>" + tracker.getFaultReport(tt.getHost()));
         }
         for(StatisticsCollector.TimeWindow window : tracker.getStatistics().
           collector.DEFAULT_COLLECT_WINDOWS) {