Browse Source

commit 6939f6854b330a01cc4427f4c657df0c3c4d53ab
Author: Arun C Murthy <acmurthy@apache.org>
Date: Fri Jul 23 15:39:49 2010 -0700

MAPREDUCE-1966. Change blacklisting of tasktrackers on task failures to be a simple graylist to fingerpoint bad tasktrackers. Contributed by Greg Roelofs.

+++ b/YAHOO-CHANGES.txt
+ MAPREDUCE-1966. Change blacklisting of tasktrackers on task failures to be
+ a simple graylist to fingerpoint bad tasktrackers. (Greg Roelofs via
+ acmurthy)
+


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

Owen O'Malley 14 years ago
parent
commit
9fc6dca454

+ 35 - 5
src/mapred/mapred-default.xml

@@ -678,11 +678,41 @@
 <property>
   <name>mapred.max.tracker.blacklists</name>
   <value>4</value>
-  <description>The number of blacklists for a taskTracker by various jobs
-               after which the task tracker could be blacklisted across
-               all jobs. The tracker will be given a tasks later
-               (after a day). The tracker will become a healthy
-               tracker after a restart.
+  <description>The number of blacklists for a tasktracker by various jobs
+               after which the tasktracker will be marked as potentially
+               faulty and is a candidate for graylisting across all jobs.
+               (Unlike blacklisting, this is advisory; the tracker remains
+               active.  However, it is reported as graylisted in the web UI,
+               with the expectation that chronically graylisted trackers
+               will be manually decommissioned.)  This value is tied to
+               mapred.jobtracker.blacklist.fault-timeout-window; faults
+               older than the window width are forgiven, so the tracker
+               will recover from transient problems.  It will also become
+               healthy after a restart.
+  </description>
+</property> 
+
+<property>
+  <name>mapred.jobtracker.blacklist.fault-timeout-window</name>
+  <value>180</value>
+  <description>The timeout (in minutes) after which per-job tasktracker
+               faults are forgiven.  The window is logically a circular
+               buffer of time-interval buckets whose width is defined by
+               mapred.jobtracker.blacklist.fault-bucket-width; when the
+               "now" pointer moves across a bucket boundary, the previous
+               contents (faults) of the new bucket are cleared.  In other
+               words, the timeout's granularity is determined by the bucket
+               width.
+  </description>
+</property> 
+
+<property>
+  <name>mapred.jobtracker.blacklist.fault-bucket-width</name>
+  <value>15</value>
+  <description>The width (in minutes) of each bucket in the tasktracker
+               fault timeout window.  Each bucket is reused in a circular
+               manner after a full timeout-window interval (defined by
+               mapred.jobtracker.blacklist.fault-timeout-window).
   </description>
 </property> 
 

+ 87 - 36
src/mapred/org/apache/hadoop/mapred/ClusterStatus.java

@@ -60,7 +60,9 @@ public class ClusterStatus implements Writable {
   private int numActiveTrackers;
   private Collection<String> activeTrackers = new ArrayList<String>();
   private Collection<String> blacklistedTrackers = new ArrayList<String>();
+  private Collection<String> graylistedTrackers = new ArrayList<String>();
   private int numBlacklistedTrackers;
+  private int numGraylistedTrackers;
   private int numExcludedNodes;
   private long ttExpiryInterval;
   private int map_tasks;
@@ -78,7 +80,7 @@ public class ClusterStatus implements Writable {
   /**
    * Construct a new cluster status.
    * 
-   * @param trackers no. of tasktrackers in the cluster
+   * @param trackers no. of active tasktrackers in the cluster
    * @param maps no. of currently running map-tasks in the cluster
    * @param reduces no. of currently running reduce-tasks in the cluster
    * @param maxMaps the maximum no. of map tasks in the cluster
@@ -89,15 +91,16 @@ public class ClusterStatus implements Writable {
   @Deprecated
   ClusterStatus(int trackers, int maps, int reduces, int maxMaps,
                 int maxReduces, JobTracker.State state) {
-    this(trackers, 0, JobTracker.TASKTRACKER_EXPIRY_INTERVAL, maps, reduces,
+    this(trackers, 0, 0, JobTracker.TASKTRACKER_EXPIRY_INTERVAL, maps, reduces,
         maxMaps, maxReduces, state);
   }
   
   /**
    * Construct a new cluster status.
    * 
-   * @param trackers no. of tasktrackers in the cluster
-   * @param blacklists no of blacklisted task trackers in the cluster
+   * @param trackers no. of active tasktrackers in the cluster (includes gray)
+   * @param blacklists no. of blacklisted task trackers in the cluster
+   * @param graylists no. of graylisted task trackers 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
@@ -105,17 +108,19 @@ public class ClusterStatus implements Writable {
    * @param maxReduces the maximum no. of reduce tasks in the cluster
    * @param state the {@link JobTracker.State} of the <code>JobTracker</code>
    */
-  ClusterStatus(int trackers, int blacklists, long ttExpiryInterval, 
-                int maps, int reduces,
+  ClusterStatus(int trackers, int blacklists, int graylists,
+                long ttExpiryInterval, int maps, int reduces,
                 int maxMaps, int maxReduces, JobTracker.State state) {
-    this(trackers, blacklists, ttExpiryInterval, maps, reduces, maxMaps, 
-         maxReduces, state, 0);
+    this(trackers, blacklists, graylists, ttExpiryInterval, maps, reduces,
+         maxMaps, maxReduces, state, 0);
   }
 
   /**
    * Construct a new cluster status.
-   * @param trackers no. of tasktrackers in the cluster
-   * @param blacklists no of blacklisted task trackers in the cluster
+   * 
+   * @param trackers no. of active tasktrackers in the cluster (includes gray)
+   * @param blacklists no. of blacklisted task trackers in the cluster
+   * @param graylists no. of graylisted task trackers 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
@@ -124,10 +129,11 @@ public class ClusterStatus implements Writable {
    * @param state the {@link JobTracker.State} of the <code>JobTracker</code>
    * @param numDecommissionedNodes number of decommission trackers
    */
-  ClusterStatus(int trackers, int blacklists, long ttExpiryInterval, 
+  ClusterStatus(int trackers, int blacklists, int graylists,
+                long ttExpiryInterval, 
                 int maps, int reduces, int maxMaps, int maxReduces, 
                 JobTracker.State state, int numDecommissionedNodes) {
-    this(trackers, blacklists, ttExpiryInterval, maps, reduces, 
+    this(trackers, blacklists, graylists, ttExpiryInterval, maps, reduces, 
          maxMaps, maxReduces, state, numDecommissionedNodes, 
          UNINITIALIZED_MEMORY_VALUE, UNINITIALIZED_MEMORY_VALUE);
   }
@@ -135,8 +141,9 @@ public class ClusterStatus implements Writable {
   /**
    * Construct a new cluster status.
    * 
-   * @param activeTrackers active tasktrackers in the cluster
+   * @param activeTrackers active tasktrackers in the cluster (includes gray)
    * @param blacklistedTrackers blacklisted tasktrackers in the cluster
+   * @param graylistedTrackers graylisted tasktrackers 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
@@ -144,21 +151,23 @@ public class ClusterStatus implements Writable {
    * @param maxReduces the maximum no. of reduce tasks in the cluster
    * @param state the {@link JobTracker.State} of the <code>JobTracker</code>
    */
-  ClusterStatus(Collection<String> activeTrackers, 
-      Collection<String> blacklistedTrackers,
-      long ttExpiryInterval,
-      int maps, int reduces, int maxMaps, int maxReduces, 
-      JobTracker.State state) {
-    this(activeTrackers, blacklistedTrackers, ttExpiryInterval, maps, reduces, 
-         maxMaps, maxReduces, state, 0);
+  ClusterStatus(Collection<String> activeTrackers,
+                Collection<String> blacklistedTrackers,
+                Collection<String> graylistedTrackers,
+                long ttExpiryInterval,
+                int maps, int reduces, int maxMaps, int maxReduces,
+                JobTracker.State state) {
+    this(activeTrackers, blacklistedTrackers, graylistedTrackers,
+         ttExpiryInterval, maps, reduces, maxMaps, maxReduces, state, 0);
   }
 
-  ClusterStatus(int trackers, int blacklists, long ttExpiryInterval, 
-      int maps, int reduces, int maxMaps, int maxReduces, 
-      JobTracker.State state, int numDecommissionedNodes,
-      long used_memory, long max_memory) {
+  ClusterStatus(int trackers, int blacklists, int graylists,
+                long ttExpiryInterval, int maps, int reduces, int maxMaps,
+                int maxReduces, JobTracker.State state,
+                int numDecommissionedNodes, long used_memory, long max_memory) {
     numActiveTrackers = trackers;
     numBlacklistedTrackers = blacklists;
+    numGraylistedTrackers = graylists;
     this.numExcludedNodes = numDecommissionedNodes;
     this.ttExpiryInterval = ttExpiryInterval;
     map_tasks = maps;
@@ -172,8 +181,9 @@ public class ClusterStatus implements Writable {
   
   /**
    * Construct a new cluster status. 
-   * @param activeTrackers active tasktrackers in the cluster
+   * @param activeTrackers active tasktrackers in the cluster (includes gray)
    * @param blacklistedTrackers blacklisted tasktrackers in the cluster
+   * @param graylistedTrackers graylisted tasktrackers 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
@@ -183,28 +193,33 @@ public class ClusterStatus implements Writable {
    * @param numDecommissionNodes number of decommission trackers
    */
   ClusterStatus(Collection<String> activeTrackers, 
-                Collection<String> blacklistedTrackers, long ttExpiryInterval,
+                Collection<String> blacklistedTrackers,
+                Collection<String> graylistedTrackers, 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, 
+    this(activeTrackers.size(), blacklistedTrackers.size(),
+        graylistedTrackers.size(), ttExpiryInterval, maps, reduces,
+        maxMaps, maxReduces, state, numDecommissionNodes,
         Runtime.getRuntime().totalMemory(), Runtime.getRuntime().maxMemory());
     this.activeTrackers = activeTrackers;
     this.blacklistedTrackers = blacklistedTrackers;
+    this.graylistedTrackers = graylistedTrackers;
   }
 
   /**
-   * Get the number of task trackers in the cluster.
-   * 
-   * @return the number of task trackers in the cluster.
+   * Get the number of active task trackers in the cluster.  Includes
+   * graylisted but not blacklisted trackers.
+   *
+   * @return the number of active task trackers in the cluster.
    */
   public int getTaskTrackers() {
     return numActiveTrackers;
   }
   
   /**
-   * Get the names of task trackers in the cluster.
-   * 
+   * Get the names of active task trackers in the cluster.  Includes
+   * graylisted but not blacklisted trackers.
+   *
    * @return the active task trackers in the cluster.
    */
   public Collection<String> getActiveTrackerNames() {
@@ -212,14 +227,14 @@ public class ClusterStatus implements Writable {
   }
 
   /**
-   * Get the names of task trackers in the cluster.
+   * Get the names of blacklisted task trackers in the cluster.
    * 
    * @return the blacklisted task trackers in the cluster.
    */
   public Collection<String> getBlacklistedTrackerNames() {
     return blacklistedTrackers;
   }
-  
+
   /**
    * Get the number of blacklisted task trackers in the cluster.
    * 
@@ -228,7 +243,25 @@ public class ClusterStatus implements Writable {
   public int getBlacklistedTrackers() {
     return numBlacklistedTrackers;
   }
-  
+
+  /**
+   * Get the names of graylisted task trackers in the cluster.
+   * 
+   * @return the graylisted task trackers in the cluster.
+   */
+  public Collection<String> getGraylistedTrackerNames() {
+    return graylistedTrackers;
+  }
+
+  /**
+   * Get the number of graylisted task trackers in the cluster.
+   * 
+   * @return the number of graylisted task trackers in the cluster.
+   */
+  public int getGraylistedTrackers() {
+    return numGraylistedTrackers;
+  }
+
   /**
    * Get the number of excluded hosts in the cluster.
    * @return the number of excluded hosts in the cluster.
@@ -330,6 +363,16 @@ public class ClusterStatus implements Writable {
         Text.writeString(out, tracker);
       }
     }
+    if (graylistedTrackers.size() == 0) {
+      out.writeInt(numGraylistedTrackers);
+      out.writeInt(0);
+    } else {
+      out.writeInt(graylistedTrackers.size());
+      out.writeInt(graylistedTrackers.size());
+      for (String tracker : graylistedTrackers) {
+        Text.writeString(out, tracker);
+      }
+    }
     out.writeInt(numExcludedNodes);
     out.writeLong(ttExpiryInterval);
     out.writeInt(map_tasks);
@@ -358,6 +401,14 @@ public class ClusterStatus implements Writable {
         blacklistedTrackers.add(name);
       }
     }
+    numGraylistedTrackers = in.readInt();
+    numTrackerNames = in.readInt();
+    if (numTrackerNames > 0) {
+      for (int i = 0; i < numTrackerNames; i++) {
+        String name = Text.readString(in);
+        graylistedTrackers.add(name);
+      }
+    }
     numExcludedNodes = in.readInt();
     ttExpiryInterval = in.readLong();
     map_tasks = in.readInt();

File diff suppressed because it is too large
+ 466 - 261
src/mapred/org/apache/hadoop/mapred/JobTracker.java


+ 6 - 0
src/mapred/org/apache/hadoop/mapred/JobTrackerInstrumentation.java

@@ -157,6 +157,12 @@ class JobTrackerInstrumentation {
   public void decBlackListedTrackers(int trackers)
   { }
 
+  public void addGrayListedTrackers(int trackers)
+  { }
+
+  public void decGrayListedTrackers(int trackers)
+  { }
+
   public void setDecommissionedTrackers(int trackers)
   { }   
 

+ 13 - 0
src/mapred/org/apache/hadoop/mapred/JobTrackerMetricsInst.java

@@ -62,6 +62,7 @@ class JobTrackerMetricsInst extends JobTrackerInstrumentation implements Updater
 
   private int numTrackers = 0;
   private int numTrackersBlackListed = 0;
+  private int numTrackersGrayListed = 0;
   private int numTrackersDecommissioned = 0;
 
   // long, because 2^31 could well be only about a month's worth of
@@ -402,6 +403,18 @@ class JobTrackerMetricsInst extends JobTrackerInstrumentation implements Updater
     numTrackersBlackListed -= trackers;
   }
 
+  @Override
+  public synchronized void addGrayListedTrackers(int trackers)
+  {
+    numTrackersGrayListed += trackers;
+  }
+
+  @Override
+  public synchronized void decGrayListedTrackers(int trackers)
+  {
+    numTrackersGrayListed -= trackers;
+  }
+
   @Override
   public synchronized void setDecommissionedTrackers(int trackers)
   {

+ 1 - 1
src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -474,7 +474,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
   }
   
   public ClusterStatus getClusterStatus(boolean detailed) {
-    return new ClusterStatus(1, 0, 0, map_tasks, reduce_tasks, 1, 1, 
+    return new ClusterStatus(1, 0, 0, 0, map_tasks, reduce_tasks, 1, 1, 
                              JobTracker.State.RUNNING);
   }
 

+ 12 - 1
src/mapred/org/apache/hadoop/mapreduce/ClusterMetrics.java

@@ -61,6 +61,7 @@ public class ClusterMetrics implements Writable {
   private int totalJobSubmissions;
   private int numTrackers;
   private int numBlacklistedTrackers;
+  private int numGraylistedTrackers;
   private int numDecommissionedTrackers;
 
   public ClusterMetrics() {
@@ -71,7 +72,7 @@ public class ClusterMetrics implements Writable {
       int reservedMapSlots, int reservedReduceSlots,
       int mapSlots, int reduceSlots, 
       int totalJobSubmissions,
-      int numTrackers, int numBlacklistedTrackers,
+      int numTrackers, int numBlacklistedTrackers, int numGraylistedTrackers,
       int numDecommissionedNodes) {
     this.runningMaps = runningMaps;
     this.runningReduces = runningReduces;
@@ -84,6 +85,7 @@ public class ClusterMetrics implements Writable {
     this.totalJobSubmissions = totalJobSubmissions;
     this.numTrackers = numTrackers;
     this.numBlacklistedTrackers = numBlacklistedTrackers;
+    this.numGraylistedTrackers = numGraylistedTrackers;
     this.numDecommissionedTrackers = numDecommissionedNodes;
   }
 
@@ -186,6 +188,15 @@ public class ClusterMetrics implements Writable {
     return numBlacklistedTrackers;
   }
   
+  /**
+   * Get the number of graylisted trackers in the cluster.
+   * 
+   * @return graylisted tracker count
+   */
+  public int getGrayListedTaskTrackerCount() {
+    return numGraylistedTrackers;
+  }
+  
   /**
    * Get the number of decommissioned trackers in the cluster.
    * 

+ 2 - 2
src/test/org/apache/hadoop/mapred/FakeObjectUtilities.java

@@ -58,8 +58,8 @@ public class FakeObjectUtilities {
     }
     @Override
     public ClusterStatus getClusterStatus(boolean detailed) {
-      return new ClusterStatus(trackers.length,
-          0, 0, 0, 0, totalSlots/2, totalSlots/2, JobTracker.State.RUNNING, 0);
+      return new ClusterStatus(trackers.length, 0, 0,
+          0, 0, 0, totalSlots/2, totalSlots/2, JobTracker.State.RUNNING, 0);
     }
 
     public void setNumSlots(int totalSlots) {

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestJobQueueTaskScheduler.java

@@ -144,7 +144,7 @@ public class TestJobQueueTaskScheduler extends TestCase {
     @Override
     public ClusterStatus getClusterStatus() {
       int numTrackers = trackers.size();
-      return new ClusterStatus(numTrackers, 0, 
+      return new ClusterStatus(numTrackers, 0, 0,
                                JobTracker.TASKTRACKER_EXPIRY_INTERVAL,
                                maps, reduces,
                                numTrackers * maxMapTasksPerTracker,

+ 10 - 9
src/test/org/apache/hadoop/mapred/TestNodeBlacklisting.java

@@ -99,16 +99,17 @@ public class TestNodeBlacklisting extends TestCase {
       job.waitForCompletion();
 
       // validate the total tracker count
-      assertEquals("Active tracker count mismatch", 
-                   1, jt.getClusterStatus(false).getTaskTrackers());
-      // validate blacklisted count
-      assertEquals("Blacklisted tracker count mismatch", 
-                   1, jt.getClusterStatus(false).getBlacklistedTrackers());
+      // (graylisted trackers remain active, unlike blacklisted ones)
+      assertEquals("Active tracker count mismatch",
+                   2, jt.getClusterStatus(false).getTaskTrackers());
+      // validate graylisted count
+      assertEquals("Graylisted tracker count mismatch",
+                   1, jt.getClusterStatus(false).getGraylistedTrackers());
 
-      // find the blacklisted tracker
+      // find the graylisted tracker
       String trackerName = null;
       for (TaskTrackerStatus status : jt.taskTrackers()) {
-        if (jt.isBlacklisted(status.getTrackerName())) {
+        if (jt.isGraylisted(status.getTrackerName())) {
           trackerName = status.getTrackerName();
           break;
         }
@@ -127,9 +128,9 @@ public class TestNodeBlacklisting extends TestCase {
       // check the cluster status and tracker size
       assertEquals("Tracker is not lost upon host decommissioning", 
                    1, jt.getClusterStatus(false).getTaskTrackers());
-      assertEquals("Blacklisted tracker count incorrect in cluster status "
+      assertEquals("Graylisted tracker count incorrect in cluster status "
                    + "after decommissioning", 
-                   0, jt.getClusterStatus(false).getBlacklistedTrackers());
+                   0, jt.getClusterStatus(false).getGraylistedTrackers());
       assertEquals("Tracker is not lost upon host decommissioning", 
                    1, jt.taskTrackers().size());
     } finally {

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestParallelInitialization.java

@@ -97,7 +97,7 @@ public class TestParallelInitialization extends TestCase {
     
     public ClusterStatus getClusterStatus() {
       int numTrackers = trackers.size();
-      return new ClusterStatus(numTrackers, 0, 
+      return new ClusterStatus(numTrackers, 0, 0,
                                JobTracker.TASKTRACKER_EXPIRY_INTERVAL,
                                maps, reduces,
                                numTrackers * maxMapTasksPerTracker,

+ 3 - 0
src/webapps/job/jobtracker.jsp

@@ -41,6 +41,7 @@
               "<th>Map Task Capacity</th>" +
               "<th>Reduce Task Capacity</th><th>Avg. Tasks/Node</th>" + 
               "<th>Blacklisted Nodes</th>" +
+              "<th>Graylisted Nodes</th>" +
               "<th>Excluded Nodes</th></tr>\n");
     out.print("<tr><td>" + metrics.getRunningMaps() + "</td><td>" +
               metrics.getRunningReduces() + "</td><td>" + 
@@ -56,6 +57,8 @@
               "</td><td>" + tasksPerNode +
               "</td><td><a href=\"machines.jsp?type=blacklisted\">" +
               metrics.getBlackListedTaskTrackerCount() + "</a>" +
+              "</td><td><a href=\"machines.jsp?type=graylisted\">" +
+              metrics.getGrayListedTaskTrackerCount() + "</a>" +
               "</td><td><a href=\"machines.jsp?type=excluded\">" +
               metrics.getDecommissionedTaskTrackerCount() + "</a>" +
               "</td></tr></table>\n");

+ 13 - 6
src/webapps/job/machines.jsp

@@ -24,6 +24,9 @@
     if (("blacklisted").equals(type)) {
       out.println("<h2>Blacklisted Task Trackers</h2>");
       c = tracker.blacklistedTaskTrackers();
+    } else if (("graylisted").equals(type)) {
+      out.println("<h2>Graylisted Task Trackers</h2>");
+      c = tracker.graylistedTaskTrackers();
     } else if (("active").equals(type)) {
       out.println("<h2>Active Task Trackers</h2>");
       c = tracker.activeTaskTrackers();
@@ -33,7 +36,7 @@
     }
     int noCols = 9 + 
       (2 * tracker.getStatistics().collector.DEFAULT_COLLECT_WINDOWS.length);
-    if(type.equals("blacklisted")) {
+    if (type.equals("blacklisted") || type.equals("graylisted")) {
       noCols = noCols + 1;
     }
     if (c.size() == 0) {
@@ -49,10 +52,12 @@
                 "<td><b>Failures</b></td>" +
                 "<td><b>Node Health Status</b></td>" +
                 "<td><b>Seconds Since Node Last Healthy</b></td>");
-      if(type.equals("blacklisted")) {
-      	out.print("<td><b>Reason For blacklisting</b></td>");
+      if (type.equals("blacklisted")) {
+        out.print("<td><b>Reason for Blacklisting</b></td>");
+      } else if (type.equals("graylisted")) {
+        out.print("<td><b>Reason for Graylisting</b></td>");
       }
-      for(StatisticsCollector.TimeWindow window : tracker.getStatistics().
+      for (StatisticsCollector.TimeWindow window : tracker.getStatistics().
            collector.DEFAULT_COLLECT_WINDOWS) {
          out.println("<td><b>Total Tasks "+window.name+"</b></td>");
          out.println("<td><b>Succeeded Tasks "+window.name+"</b></td>");
@@ -97,10 +102,12 @@
                   "</td><td>" + numFailures +
                   "</td><td>" + healthString +
                   "</td><td>" + sinceHealthCheck); 
-        if(type.equals("blacklisted")) {
+        if (type.equals("blacklisted")) {
           out.print("</td><td>" + tracker.getReasonsForBlacklisting(tt.getHost()));
+        } else if (type.equals("graylisted")) {
+          out.print("</td><td>" + tracker.getReasonsForGraylisting(tt.getHost()));
         }
-        for(StatisticsCollector.TimeWindow window : tracker.getStatistics().
+        for (StatisticsCollector.TimeWindow window : tracker.getStatistics().
           collector.DEFAULT_COLLECT_WINDOWS) {
           JobTrackerStatistics.TaskTrackerStat ttStat = tracker.getStatistics().
              getTaskTrackerStat(tt.getTrackerName());

Some files were not shown because too many files changed in this diff