You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by om...@apache.org on 2011/03/04 04:25:11 UTC

svn commit: r1076949 [3/3] - in /hadoop/common/branches/branch-0.20-security-patches/src: contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/ contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/ contrib/fairscheduler/src/java/org/apa...

Added: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapreduce/server/jobtracker/TaskTracker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapreduce/server/jobtracker/TaskTracker.java?rev=1076949&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapreduce/server/jobtracker/TaskTracker.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapreduce/server/jobtracker/TaskTracker.java Fri Mar  4 03:25:10 2011
@@ -0,0 +1,198 @@
+/**
+ * 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.mapreduce.server.jobtracker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.JobInProgress;
+import org.apache.hadoop.mapred.JobTracker;
+import org.apache.hadoop.mapred.TaskTrackerStatus;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskType;
+
+/**
+ * The representation of a single <code>TaskTracker</code> as seen by 
+ * the {@link JobTracker}.
+ */
+public class TaskTracker {
+  static final Log LOG = LogFactory.getLog(TaskTracker.class);
+  
+  final private String trackerName;
+  private TaskTrackerStatus status;
+
+  private JobInProgress jobForFallowMapSlot;
+  private JobInProgress jobForFallowReduceSlot;
+  
+  /**
+   * Create a new {@link TaskTracker}.
+   * @param trackerName Unique identifier for the <code>TaskTracker</code>
+   */
+  public TaskTracker(String trackerName) {
+    this.trackerName = trackerName;
+  }
+
+  /**
+   * Get the unique identifier for the {@link TaskTracker}
+   * @return the unique identifier for the <code>TaskTracker</code>
+   */
+  public String getTrackerName() {
+    return trackerName;
+  }
+
+  /**
+   * Get the current {@link TaskTrackerStatus} of the <code>TaskTracker</code>.
+   * @return the current <code>TaskTrackerStatus</code> of the 
+   *         <code>TaskTracker</code>
+   */
+  public TaskTrackerStatus getStatus() {
+    return status;
+  }
+
+  /**
+   * Set the current {@link TaskTrackerStatus} of the <code>TaskTracker</code>.
+   * @param status the current <code>TaskTrackerStatus</code> of the 
+   *               <code>TaskTracker</code>
+   */
+  public void setStatus(TaskTrackerStatus status) {
+    this.status = status;
+  }
+
+  /**
+   * Get the number of currently available slots on this tasktracker for the 
+   * given type of the task.
+   * @param taskType the {@link TaskType} to check for number of available slots 
+   * @return the number of currently available slots for the given 
+   *         <code>taskType</code>
+   */
+  public int getAvailableSlots(TaskType taskType) {
+    int availableSlots = 0;
+    if (taskType == TaskType.MAP) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(trackerName + " getAvailSlots:" +
+        		     " max(m)=" + status.getMaxMapSlots() + 
+        		     " occupied(m)=" + status.countOccupiedMapSlots());
+      }
+      availableSlots = status.getAvailableMapSlots();
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(trackerName + " getAvailSlots:" +
+                  " max(r)=" + status.getMaxReduceSlots() + 
+                  " occupied(r)=" + status.countOccupiedReduceSlots());
+      }
+      availableSlots = status.getAvailableReduceSlots();
+    }
+    return availableSlots;
+  }
+  
+  /**
+   * Get the {@link JobInProgress} for which the fallow slot(s) are held.
+   * @param taskType {@link TaskType} of the task
+   * @return the task for which the fallow slot(s) are held, 
+   *         <code>null</code> if there are no fallow slots
+   */
+  public JobInProgress getJobForFallowSlot(TaskType taskType) {
+    return 
+      (taskType == TaskType.MAP) ? jobForFallowMapSlot : jobForFallowReduceSlot;
+  }
+
+  /**
+   * Reserve specified number of slots for a given <code>job</code>.
+   * @param taskType {@link TaskType} of the task
+   * @param job the job for which slots on this <code>TaskTracker</code>
+   *             are to be reserved
+   * @param numSlots number of slots to be reserved
+   */
+  public void reserveSlots(TaskType taskType, JobInProgress job, int numSlots) {
+    JobID jobId = job.getJobID();
+    if (taskType == TaskType.MAP) {
+      if (jobForFallowMapSlot != null && 
+          !jobForFallowMapSlot.getJobID().equals(jobId)) {
+        throw new RuntimeException(trackerName + " already has " + 
+                                   "slots reserved for " + 
+                                   jobForFallowMapSlot + "; being"  +
+                                   " asked to reserve " + numSlots + " for " + 
+                                   jobId);
+      }
+
+      jobForFallowMapSlot = job;
+    } else if (taskType == TaskType.REDUCE){
+      if (jobForFallowReduceSlot != null && 
+          !jobForFallowReduceSlot.getJobID().equals(jobId)) {
+        throw new RuntimeException(trackerName + " already has " + 
+                                   "slots reserved for " + 
+                                   jobForFallowReduceSlot + "; being"  +
+                                   " asked to reserve " + numSlots + " for " + 
+                                   jobId);
+      }
+
+      jobForFallowReduceSlot = job;
+    }
+    
+    job.reserveTaskTracker(this, taskType, numSlots);
+    LOG.info(trackerName + ": Reserved " + numSlots + " " + taskType + 
+             " slots for " + jobId);
+  }
+  
+  /**
+   * Free map slots on this <code>TaskTracker</code> which were reserved for 
+   * <code>taskType</code>.
+   * @param taskType {@link TaskType} of the task
+   * @param job job whose slots are being un-reserved
+   */
+  public void unreserveSlots(TaskType taskType, JobInProgress job) {
+    JobID jobId = job.getJobID();
+    if (taskType == TaskType.MAP) {
+      if (jobForFallowMapSlot == null || 
+          !jobForFallowMapSlot.getJobID().equals(jobId)) {
+        throw new RuntimeException(trackerName + " already has " + 
+                                   "slots reserved for " + 
+                                   jobForFallowMapSlot + "; being"  +
+                                   " asked to un-reserve for " + jobId);
+      }
+
+      jobForFallowMapSlot = null;
+    } else {
+      if (jobForFallowReduceSlot == null || 
+          !jobForFallowReduceSlot.getJobID().equals(jobId)) {
+        throw new RuntimeException(trackerName + " already has " + 
+                                   "slots reserved for " + 
+                                   jobForFallowReduceSlot + "; being"  +
+                                   " asked to un-reserve for " + jobId);
+      }
+      
+      jobForFallowReduceSlot = null;
+    }
+    
+    job.unreserveTaskTracker(this, taskType);
+    LOG.info(trackerName + ": Unreserved " + taskType + " slots for " + jobId);
+  }
+  
+  /**
+   * Cleanup when the {@link TaskTracker} is declared as 'lost' by the 
+   * JobTracker.
+   */
+  public void lost() {
+    // Inform jobs which have reserved slots on this tasktracker
+    if (jobForFallowMapSlot != null) {
+      unreserveSlots(TaskType.MAP, jobForFallowMapSlot);
+    }
+    if (jobForFallowReduceSlot != null) {
+      unreserveSlots(TaskType.REDUCE, jobForFallowReduceSlot);
+    }
+  }
+}

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobHistory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobHistory.java?rev=1076949&r1=1076948&r2=1076949&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobHistory.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobHistory.java Fri Mar  4 03:25:10 2011
@@ -703,7 +703,8 @@ public class TestJobHistory extends Test
             ts.getFinishTime() == Long.parseLong(attempt.get(Keys.FINISH_TIME)));
 
 
-        TaskTrackerStatus ttStatus = jt.getTaskTracker(ts.getTaskTracker());
+        TaskTrackerStatus ttStatus = 
+          jt.getTaskTrackerStatus(ts.getTaskTracker());
 
         if (ttStatus != null) {
           assertTrue("http port of task attempt " + idStr + " obtained from " +

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueInformation.java?rev=1076949&r1=1076948&r2=1076949&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueInformation.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueInformation.java Fri Mar  4 03:25:10 2011
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.MiniDFSClu
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
 
 import junit.framework.TestCase;
 
@@ -66,7 +67,7 @@ public class TestJobQueueInformation ext
   public static class TestTaskScheduler extends LimitTasksPerJobTaskScheduler {
 
     @Override
-    public synchronized List<Task> assignTasks(TaskTrackerStatus taskTracker)
+    public synchronized List<Task> assignTasks(TaskTracker taskTracker)
         throws IOException {
       Collection<JobInProgress> jips = jobQueueJobInProgressListener
           .getJobQueue();

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueTaskScheduler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueTaskScheduler.java?rev=1076949&r1=1076948&r2=1076949&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueTaskScheduler.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueTaskScheduler.java Fri Mar  4 03:25:10 2011
@@ -27,6 +27,7 @@ import java.util.Map;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
 
 public class TestJobQueueTaskScheduler extends TestCase {
   
@@ -75,7 +76,8 @@ public class TestJobQueueTaskScheduler e
     public Task obtainNewMapTask(final TaskTrackerStatus tts, int clusterSize,
         int ignored) throws IOException {
       TaskAttemptID attemptId = getTaskAttemptID(true);
-      Task task = new MapTask("", attemptId, 0, "", new BytesWritable(), getJobConf().getUser()) {
+      Task task = new MapTask("", attemptId, 0, "", new BytesWritable(), 1, 
+                              getJobConf().getUser()) {
         @Override
         public String toString() {
           return String.format("%s on %s", getTaskID(), tts.getTrackerName());
@@ -90,7 +92,8 @@ public class TestJobQueueTaskScheduler e
     public Task obtainNewReduceTask(final TaskTrackerStatus tts,
         int clusterSize, int ignored) throws IOException {
       TaskAttemptID attemptId = getTaskAttemptID(false);
-      Task task = new ReduceTask("", attemptId, 0, 10, getJobConf().getUser()) {
+      Task task = new ReduceTask("", attemptId, 0, 10, 1, 
+                                 getJobConf().getUser()) {
         @Override
         public String toString() {
           return String.format("%s on %s", getTaskID(), tts.getTrackerName());
@@ -118,18 +121,24 @@ public class TestJobQueueTaskScheduler e
       new ArrayList<JobInProgressListener>();
     QueueManager queueManager;
     
-    private Map<String, TaskTrackerStatus> trackers =
-      new HashMap<String, TaskTrackerStatus>();
+    private Map<String, TaskTracker> trackers = 
+      new HashMap<String, TaskTracker>();
 
     public FakeTaskTrackerManager() {
       JobConf conf = new JobConf();
       queueManager = new QueueManager(conf);
-      trackers.put("tt1", new TaskTrackerStatus("tt1", "tt1.host", 1,
-                   new ArrayList<TaskStatus>(), 0,
-                   maxMapTasksPerTracker, maxReduceTasksPerTracker));
-      trackers.put("tt2", new TaskTrackerStatus("tt2", "tt2.host", 2,
-                   new ArrayList<TaskStatus>(), 0,
-                   maxMapTasksPerTracker, maxReduceTasksPerTracker));
+      
+      TaskTracker tt1 = new TaskTracker("tt1");
+      tt1.setStatus(new TaskTrackerStatus("tt1", "tt1.host", 1,
+                    new ArrayList<TaskStatus>(), 0,
+                    maxMapTasksPerTracker, maxReduceTasksPerTracker));
+      trackers.put("tt1", tt1);
+      
+      TaskTracker tt2 = new TaskTracker("tt2");
+      tt2.setStatus(new TaskTrackerStatus("tt2", "tt2.host", 2,
+                    new ArrayList<TaskStatus>(), 0,
+                    maxMapTasksPerTracker, maxReduceTasksPerTracker));
+      trackers.put("tt2", tt2);
     }
     
     @Override
@@ -150,7 +159,11 @@ public class TestJobQueueTaskScheduler e
 
     @Override
     public Collection<TaskTrackerStatus> taskTrackers() {
-      return trackers.values();
+      List<TaskTrackerStatus> taskTrackers = new ArrayList<TaskTrackerStatus>();
+      for (TaskTracker tt : trackers.values()) {
+        taskTrackers.add(tt.getStatus());
+      }
+      return taskTrackers;
     }
 
 
@@ -200,7 +213,7 @@ public class TestJobQueueTaskScheduler e
       }
     }
     
-    public TaskTrackerStatus getTaskTracker(String trackerID) {
+    public TaskTracker getTaskTracker(String trackerID) {
       return trackers.get(trackerID);
     }
     
@@ -217,7 +230,7 @@ public class TestJobQueueTaskScheduler e
         }
       };
       status.setRunState(TaskStatus.State.RUNNING);
-      trackers.get(taskTrackerName).getTaskReports().add(status);
+      trackers.get(taskTrackerName).getStatus().getTaskReports().add(status);
     }
     
   }
@@ -293,14 +306,14 @@ public class TestJobQueueTaskScheduler e
     checkAssignment(scheduler, tracker(taskTrackerManager, "tt2"), new String[] {});
   }
 
-  static TaskTrackerStatus tracker(FakeTaskTrackerManager taskTrackerManager,
+  static TaskTracker tracker(FakeTaskTrackerManager taskTrackerManager,
                                       String taskTrackerName) {
     return taskTrackerManager.getTaskTracker(taskTrackerName);
   }
   
-  static void checkAssignment(TaskScheduler scheduler, TaskTrackerStatus tts,
+  static void checkAssignment(TaskScheduler scheduler, TaskTracker taskTracker,
       String[] expectedTaskStrings) throws IOException {
-    List<Task> tasks = scheduler.assignTasks(tts);
+    List<Task> tasks = scheduler.assignTasks(taskTracker);
     assertNotNull(tasks);
     assertEquals(expectedTaskStrings.length, tasks.size());
     for (int i=0; i < expectedTaskStrings.length; ++i) {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobTrackerSafeMode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobTrackerSafeMode.java?rev=1076949&r1=1076948&r2=1076949&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobTrackerSafeMode.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobTrackerSafeMode.java Fri Mar  4 03:25:10 2011
@@ -194,7 +194,7 @@ public class TestJobTrackerSafeMode exte
     long jobtrackerRecoveryFinishTime = 
       jobtracker.getStartTime() + jobtracker.getRecoveryDuration();
     for (String trackerName : present) {
-      TaskTrackerStatus status = jobtracker.getTaskTracker(trackerName);
+      TaskTrackerStatus status = jobtracker.getTaskTrackerStatus(trackerName);
       // check if the status is present and also the tracker has contacted back
       // after restart
       if (status == null 
@@ -203,7 +203,7 @@ public class TestJobTrackerSafeMode exte
       }
     }
     for (String trackerName : absent) {
-      TaskTrackerStatus status = jobtracker.getTaskTracker(trackerName);
+      TaskTrackerStatus status = jobtracker.getTaskTrackerStatus(trackerName);
       // check if the status is still present
       if ( status != null) {
         return false;

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestResourceEstimation.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestResourceEstimation.java?rev=1076949&r1=1076948&r2=1076949&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestResourceEstimation.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestResourceEstimation.java Fri Mar  4 03:25:10 2011
@@ -46,7 +46,8 @@ public class TestResourceEstimation exte
       ts.setOutputSize(singleMapOutputSize);
       RawSplit split = new RawSplit();
       split.setDataLength(0);
-      TaskInProgress tip = new TaskInProgress(jid, "", split, null, jc, jip, 0);
+      TaskInProgress tip = 
+        new TaskInProgress(jid, "", split, null, jc, jip, 0, 1);
       re.updateWithCompletedTask(ts, tip);
     }
     assertEquals(2* singleMapOutputSize, re.getEstimatedMapOutputSize());
@@ -81,7 +82,8 @@ public class TestResourceEstimation exte
       ts.setOutputSize(singleMapOutputSize);
       RawSplit split = new RawSplit();
       split.setDataLength(singleMapInputSize);
-      TaskInProgress tip = new TaskInProgress(jid, "", split, null, jc, jip, 0);
+      TaskInProgress tip = 
+        new TaskInProgress(jid, "", split, null, jc, jip, 0, 1);
       re.updateWithCompletedTask(ts, tip);
     }
     
@@ -93,7 +95,8 @@ public class TestResourceEstimation exte
     ts.setOutputSize(singleMapOutputSize);
     RawSplit split = new RawSplit();
     split.setDataLength(0);
-    TaskInProgress tip = new TaskInProgress(jid, "", split, null, jc, jip, 0);
+    TaskInProgress tip = 
+      new TaskInProgress(jid, "", split, null, jc, jip, 0, 1);
     re.updateWithCompletedTask(ts, tip);
     
     long expectedTotalMapOutSize = (singleMapOutputSize*11) * 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestTTMemoryReporting.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestTTMemoryReporting.java?rev=1076949&r1=1076948&r2=1076949&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestTTMemoryReporting.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestTTMemoryReporting.java Fri Mar  4 03:25:10 2011
@@ -26,6 +26,7 @@ import org.apache.hadoop.examples.SleepJ
 import org.apache.hadoop.util.LinuxMemoryCalculatorPlugin;
 import org.apache.hadoop.util.MemoryCalculatorPlugin;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
 
 import junit.framework.TestCase;
 
@@ -66,9 +67,10 @@ public class TestTTMemoryReporting exten
     }
     
     @Override
-    public List<Task> assignTasks(TaskTrackerStatus status)
+    public List<Task> assignTasks(TaskTracker taskTracker)
         throws IOException {
-
+      TaskTrackerStatus status = taskTracker.getStatus();
+      
       long totalVirtualMemoryOnTT =
           getConf().getLong("totalVmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
       long totalPhysicalMemoryOnTT =
@@ -111,7 +113,7 @@ public class TestTTMemoryReporting exten
           || reduceSlotMemorySize != reportedReduceSlotMemorySize) {
         hasPassed = false;
       }
-      return super.assignTasks(status);
+      return super.assignTasks(taskTracker);
     }
   }
 
@@ -126,7 +128,7 @@ public class TestTTMemoryReporting exten
     try {
       // Memory values are disabled by default.
       conf.setClass(
-          TaskTracker.MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
+          org.apache.hadoop.mapred.TaskTracker.MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
           DummyMemoryCalculatorPlugin.class, MemoryCalculatorPlugin.class);
       setUpCluster(conf);
       runSleepJob(miniMRCluster.createJobConf());
@@ -150,7 +152,7 @@ public class TestTTMemoryReporting exten
     conf.setLong("reduceSlotMemorySize", 1 * 1024L);
 
     conf.setClass(
-        TaskTracker.MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
+        org.apache.hadoop.mapred.TaskTracker.MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
         DummyMemoryCalculatorPlugin.class, MemoryCalculatorPlugin.class);
     conf.setLong(DummyMemoryCalculatorPlugin.MAXVMEM_TESTING_PROPERTY,
         4 * 1024 * 1024 * 1024L);

Modified: hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobfailures.jsp
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobfailures.jsp?rev=1076949&r1=1076948&r2=1076949&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobfailures.jsp (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobfailures.jsp Fri Mar  4 03:25:10 2011
@@ -26,7 +26,7 @@
       if ((failState == null && (taskState == TaskStatus.State.FAILED || 
           taskState == TaskStatus.State.KILLED)) || taskState == failState) {
         String taskTrackerName = statuses[i].getTaskTracker();
-        TaskTrackerStatus taskTracker = tracker.getTaskTracker(taskTrackerName);
+        TaskTrackerStatus taskTracker = tracker.getTaskTrackerStatus(taskTrackerName);
         out.print("<tr><td>" + statuses[i].getTaskID() +
                   "</td><td><a href=\"taskdetails.jsp?jobid="+ jobId + 
                   "&tipid=" + tipId + "\">" + tipId +

Modified: hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/machines.jsp
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/machines.jsp?rev=1076949&r1=1076948&r2=1076949&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/machines.jsp (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/machines.jsp Fri Mar  4 03:25:10 2011
@@ -63,8 +63,8 @@
         out.print(tt.getHost() + ":" + tt.getHttpPort() + "/\">");
         out.print(tt.getTrackerName() + "</a></td><td>");
         out.print(tt.getHost() + "</td><td>" + numCurTasks +
-                  "</td><td>" + tt.getMaxMapTasks() +
-                  "</td><td>" + tt.getMaxReduceTasks() + 
+                  "</td><td>" + tt.getMaxMapSlots() +
+                  "</td><td>" + tt.getMaxReduceSlots() + 
                   "</td><td>" + numFailures + 
                   "</td><td>" + sinceHeartbeat + "</td></tr>\n");
       }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/taskdetails.jsp
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/taskdetails.jsp?rev=1076949&r1=1076948&r2=1076949&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/taskdetails.jsp (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/taskdetails.jsp Fri Mar  4 03:25:10 2011
@@ -118,7 +118,7 @@
     for (int i = 0; i < ts.length; i++) {
       TaskStatus status = ts[i];
       String taskTrackerName = status.getTaskTracker();
-      TaskTrackerStatus taskTracker = tracker.getTaskTracker(taskTrackerName);
+      TaskTrackerStatus taskTracker = tracker.getTaskTrackerStatus(taskTrackerName);
       out.print("<tr><td>" + status.getTaskID() + "</td>");
       String taskAttemptTracker = null;
       String cleanupTrackerName = null;
@@ -127,7 +127,7 @@
       boolean hasCleanupAttempt = false;
       if (tip != null && tip.isCleanupAttempt(status.getTaskID())) {
         cleanupTrackerName = tip.machineWhereCleanupRan(status.getTaskID());
-        cleanupTracker = tracker.getTaskTracker(cleanupTrackerName);
+        cleanupTracker = tracker.getTaskTrackerStatus(cleanupTrackerName);
         if (cleanupTracker != null) {
           cleanupAttemptTracker = "http://" + cleanupTracker.getHost() + ":"
             + cleanupTracker.getHttpPort();