Browse Source

HADOOP-3533. Add deprecated methods to provide API compatibility
between 0.18 and 0.17. Remove the deprecated methods in trunk.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.18@669409 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 17 years ago
parent
commit
c0b3c7b3b4

+ 3 - 0
CHANGES.txt

@@ -620,6 +620,9 @@ Release 0.18.0 - Unreleased
     HADOOP-3586. Provide deprecated, backwards compatibile semantics for the
     HADOOP-3586. Provide deprecated, backwards compatibile semantics for the
     combiner to be run once and only once on each record. (cdouglas)
     combiner to be run once and only once on each record. (cdouglas)
 
 
+    HADOOP-3533. Add deprecated methods to provide API compatibility
+    between 0.18 and 0.17. Remove the deprecated methods in trunk. (omalley)
+
 Release 0.17.1 - Unreleased
 Release 0.17.1 - Unreleased
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 10 - 2
src/core/org/apache/hadoop/ipc/Client.java

@@ -79,7 +79,7 @@ public class Client {
   private int refCount = 1;
   private int refCount = 1;
   
   
   final private static String PING_INTERVAL_NAME = "ipc.ping.interval";
   final private static String PING_INTERVAL_NAME = "ipc.ping.interval";
-  final public static int DEFAULT_PING_INTERVAL = 60000; // 1 min
+  final static int DEFAULT_PING_INTERVAL = 60000; // 1 min
   final static int PING_CALL_ID = -1;
   final static int PING_CALL_ID = -1;
   
   
   /**
   /**
@@ -523,7 +523,15 @@ public class Client {
         notifyAll();
         notifyAll();
       }
       }
     }
     }
-    
+
+    /**
+     * No longer used.
+     */
+    @Deprecated
+    public void setTimeout(int len) {
+      // NOTHING
+    }
+
     /** Close the connection. */
     /** Close the connection. */
     private synchronized void close() {
     private synchronized void close() {
       if (!shouldCloseConnection.get()) {
       if (!shouldCloseConnection.get()) {

+ 9 - 1
src/core/org/apache/hadoop/ipc/Server.java

@@ -968,7 +968,15 @@ public abstract class Server {
     } catch (IOException e) {
     } catch (IOException e) {
     }
     }
   }
   }
-  
+
+  /**
+   * No longer used.
+   */
+  @Deprecated
+  public void setTimeout(int len) {
+    // NOTHING
+  }
+
   /** Sets the socket buffer size used for responding to RPCs */
   /** Sets the socket buffer size used for responding to RPCs */
   public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
   public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
 
 

+ 1 - 1
src/core/org/apache/hadoop/metrics/util/MetricsIntValue.java

@@ -32,7 +32,7 @@ import org.apache.commons.logging.LogFactory;
  */
  */
 public class MetricsIntValue {  
 public class MetricsIntValue {  
 
 
-  protected static final Log LOG =
+  private static final Log LOG =
     LogFactory.getLog("org.apache.hadoop.metrics.util");
     LogFactory.getLog("org.apache.hadoop.metrics.util");
 
 
   private String name;
   private String name;

+ 1 - 1
src/core/org/apache/hadoop/metrics/util/MetricsTimeVaryingInt.java

@@ -34,7 +34,7 @@ import org.apache.commons.logging.LogFactory;
  */
  */
 public class MetricsTimeVaryingInt {
 public class MetricsTimeVaryingInt {
 
 
-  protected static final Log LOG =
+  private static final Log LOG =
     LogFactory.getLog("org.apache.hadoop.metrics.util");
     LogFactory.getLog("org.apache.hadoop.metrics.util");
   
   
   private String name;
   private String name;

+ 1 - 1
src/core/org/apache/hadoop/metrics/util/MetricsTimeVaryingRate.java

@@ -34,7 +34,7 @@ import org.apache.commons.logging.LogFactory;
  */
  */
 public class MetricsTimeVaryingRate {
 public class MetricsTimeVaryingRate {
 
 
-  protected static final Log LOG =
+  private static final Log LOG =
     LogFactory.getLog("org.apache.hadoop.metrics.util");
     LogFactory.getLog("org.apache.hadoop.metrics.util");
 
 
   static class Metrics {
   static class Metrics {

+ 116 - 3
src/mapred/org/apache/hadoop/mapred/JobHistory.java

@@ -62,7 +62,7 @@ import org.apache.hadoop.util.StringUtils;
  */
  */
 public class JobHistory {
 public class JobHistory {
   
   
-  public static final Log LOG = LogFactory.getLog("org.apache.hadoop.mapred.JobHistory");
+  private static final Log LOG = LogFactory.getLog("org.apache.hadoop.mapred.JobHistory");
   private static final String DELIMITER = " ";
   private static final String DELIMITER = " ";
   private static final String KEY = "(\\w+)";
   private static final String KEY = "(\\w+)";
   private static final String VALUE = "[[^\"]?]+"; // anything but a " in ""
   private static final String VALUE = "[[^\"]?]+"; // anything but a " in ""
@@ -338,7 +338,12 @@ public class JobHistory {
      * Returns all map and reduce tasks <taskid-Task>. 
      * Returns all map and reduce tasks <taskid-Task>. 
      */
      */
     public Map<String, Task> getAllTasks() { return allTasks; }
     public Map<String, Task> getAllTasks() { return allTasks; }
-    
+
+    @Deprecated
+    public static String getLocalJobFilePath(String jobid) {
+      return getLocalJobFilePath(JobID.forName(jobid));
+    }
+
     /**
     /**
      * Get the path of the locally stored job file
      * Get the path of the locally stored job file
      * @param jobId id of the job
      * @param jobId id of the job
@@ -417,7 +422,14 @@ public class JobHistory {
       }
       }
       return decodedFileName;
       return decodedFileName;
     }
     }
-    
+
+    @Deprecated
+    public static void logSubmitted(String jobid, JobConf jobConf,
+                                    String jobConfPath, long submitTime
+                                   ) throws IOException {
+      logSubmitted(JobID.forName(jobid), jobConf, jobConfPath, submitTime);
+    }
+
     /**
     /**
      * Log job submitted event to history. Creates a new file in history 
      * Log job submitted event to history. Creates a new file in history 
      * for the job. if history file creation fails, it disables history 
      * for the job. if history file creation fails, it disables history 
@@ -576,6 +588,13 @@ public class JobHistory {
         }
         }
       } 
       } 
     }
     }
+
+    @Deprecated
+    public static void logStarted(String jobid, long startTime, int totalMaps,
+                                  int totalReduces) {
+      logStarted(JobID.forName(jobid), startTime, totalMaps, totalReduces);
+    }
+
     /**
     /**
      * Logs launch time of job. 
      * Logs launch time of job. 
      * @param jobId job id, assigned by jobtracker. 
      * @param jobId job id, assigned by jobtracker. 
@@ -595,6 +614,16 @@ public class JobHistory {
         }
         }
       }
       }
     }
     }
+    
+    @Deprecated
+    public static void logFinished(String jobId, long finishTime,
+                                   int finishedMaps, int finishedReduces,
+                                   int failedMaps, int failedReduces,
+                                   Counters counters) {
+      logFinished(JobID.forName(jobId), finishTime, finishedMaps,
+                  finishedReduces, failedMaps, failedReduces, counters);
+    }
+
     /**
     /**
      * Log job finished. closes the job file in history. 
      * Log job finished. closes the job file in history. 
      * @param jobId job id, assigned by jobtracker. 
      * @param jobId job id, assigned by jobtracker. 
@@ -637,6 +666,13 @@ public class JobHistory {
         historyCleaner.start(); 
         historyCleaner.start(); 
       }
       }
     }
     }
+    
+    @Deprecated
+    public static void logFailed(String jobid, long timestamp, 
+                                 int finishedMaps, int finishedReduces) {
+      logFailed(JobID.forName(jobid), timestamp, finishedMaps, finishedReduces);
+    }
+
     /**
     /**
      * Logs job failed event. Closes the job history log file. 
      * Logs job failed event. Closes the job history log file. 
      * @param jobid job id
      * @param jobid job id
@@ -670,6 +706,12 @@ public class JobHistory {
   public static class Task extends KeyValuePair{
   public static class Task extends KeyValuePair{
     private Map <String, TaskAttempt> taskAttempts = new TreeMap<String, TaskAttempt>(); 
     private Map <String, TaskAttempt> taskAttempts = new TreeMap<String, TaskAttempt>(); 
 
 
+    @Deprecated
+    public static void logStarted(String jobId, String taskId, String taskType,
+                                  long startTime) {
+      logStarted(TaskID.forName(taskId), taskType, startTime, "n/a");
+    }
+
     /**
     /**
      * Log start time of task (TIP).
      * Log start time of task (TIP).
      * @param taskId task id
      * @param taskId task id
@@ -692,6 +734,13 @@ public class JobHistory {
         }
         }
       }
       }
     }
     }
+    
+    @Deprecated
+    public static void logFinished(String jobid, String taskid, String taskType,
+                                   long finishTime, Counters counters) {
+      logFinished(TaskID.forName(taskid), taskType, finishTime, counters);
+    }
+
     /**
     /**
      * Log finish time of task. 
      * Log finish time of task. 
      * @param taskId task id
      * @param taskId task id
@@ -715,6 +764,13 @@ public class JobHistory {
         }
         }
       }
       }
     }
     }
+    
+    @Deprecated
+    public static void logFailed(String jobid, String taskid, String taskType,
+                                 long time, String error) {
+      logFailed(TaskID.forName(taskid), taskType, time, error);
+    }
+
     /**
     /**
      * Log job failed event.
      * Log job failed event.
      * @param taskId task id
      * @param taskId task id
@@ -753,6 +809,12 @@ public class JobHistory {
    * a Map Attempt on a node.
    * a Map Attempt on a node.
    */
    */
   public static class MapAttempt extends TaskAttempt{
   public static class MapAttempt extends TaskAttempt{
+    @Deprecated
+    public static void logStarted(String jobid, String taskid, String attemptid,
+                                  long startTime, String hostName) {
+      logStarted(TaskAttemptID.forName(attemptid), startTime, hostName);
+    }
+
     /**
     /**
      * Log start time of this map task attempt. 
      * Log start time of this map task attempt. 
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
@@ -774,6 +836,13 @@ public class JobHistory {
         }
         }
       }
       }
     }
     }
+    
+    @Deprecated
+    public static void logFinished(String jobid, String taskid, 
+                                   String attemptid, long time, String host) {
+      logFinished(TaskAttemptID.forName(attemptid), time, host);
+    }
+
     /**
     /**
      * Log finish time of map task attempt. 
      * Log finish time of map task attempt. 
      * @param taskAttemptId task attempt id 
      * @param taskAttemptId task attempt id 
@@ -798,6 +867,13 @@ public class JobHistory {
       }
       }
     }
     }
 
 
+    @Deprecated
+    public static void logFailed(String jobid, String taskid,
+                                 String attemptid, long timestamp, String host, 
+                                 String err) {
+      logFailed(TaskAttemptID.forName(attemptid), timestamp, host, err);
+    }
+
     /**
     /**
      * Log task attempt failed event.  
      * Log task attempt failed event.  
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
@@ -821,6 +897,13 @@ public class JobHistory {
         }
         }
       }
       }
     }
     }
+
+    @Deprecated
+    public static void logKilled(String jobid, String taskid, String attemptid,
+                                 long timestamp, String hostname, String error){
+      logKilled(TaskAttemptID.forName(attemptid), timestamp, hostname, error);
+    }
+
     /**
     /**
      * Log task attempt killed event.  
      * Log task attempt killed event.  
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
@@ -850,6 +933,13 @@ public class JobHistory {
    * a Map Attempt on a node.
    * a Map Attempt on a node.
    */
    */
   public static class ReduceAttempt extends TaskAttempt{
   public static class ReduceAttempt extends TaskAttempt{
+    
+    @Deprecated
+    public static void logStarted(String jobid, String taskid, String attemptid,
+                                  long startTime, String hostName) {
+      logStarted(TaskAttemptID.forName(attemptid), startTime, hostName);
+    }
+
     /**
     /**
      * Log start time of  Reduce task attempt. 
      * Log start time of  Reduce task attempt. 
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
@@ -871,6 +961,15 @@ public class JobHistory {
         }
         }
       }
       }
     }
     }
+    
+    @Deprecated
+    public static void logFinished(String jobid, String taskid, String attemptid,
+                                   long shuffleFinished, long sortFinished,
+                                   long finishTime, String hostname) {
+      logFinished(TaskAttemptID.forName(attemptid), shuffleFinished, 
+                  sortFinished, finishTime, hostname);
+    }
+
     /**
     /**
      * Log finished event of this task. 
      * Log finished event of this task. 
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
@@ -899,6 +998,13 @@ public class JobHistory {
         }
         }
       }
       }
     }
     }
+
+    @Deprecated
+    public static void logFailed(String jobid, String taskid, String attemptid,
+                                 long timestamp, String hostname, String error){
+      logFailed(TaskAttemptID.forName(attemptid), timestamp, hostname, error);
+    }
+
     /**
     /**
      * Log failed reduce task attempt. 
      * Log failed reduce task attempt. 
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
@@ -922,6 +1028,13 @@ public class JobHistory {
         }
         }
       }
       }
     }
     }
+    
+    @Deprecated
+    public static void logKilled(String jobid, String taskid, String attemptid,
+                                 long timestamp, String hostname, String error){
+      logKilled(TaskAttemptID.forName(attemptid), timestamp, hostname, error);
+    }
+
     /**
     /**
      * Log killed reduce task attempt. 
      * Log killed reduce task attempt. 
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id

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

@@ -58,6 +58,12 @@ public class JobStatus implements Writable {
   public JobStatus() {
   public JobStatus() {
   }
   }
 
 
+  @Deprecated
+  public JobStatus(String jobid, float mapProgress, float reduceProgress, 
+                   int runState) {
+    this(JobID.forName(jobid), mapProgress, reduceProgress, runState);
+  }
+
   /**
   /**
    * Create a job status object for a given jobid.
    * Create a job status object for a given jobid.
    * @param jobid The jobid of the job
    * @param jobid The jobid of the job

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

@@ -1728,6 +1728,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     return new JobID(getTrackerIdentifier(), nextJobId++);
     return new JobID(getTrackerIdentifier(), nextJobId++);
   }
   }
 
 
+  @Deprecated
+  public JobStatus submitJob(String jobid) throws IOException {
+    return submitJob(JobID.forName(jobid));
+  }
+
   /**
   /**
    * JobTracker.submitJob() kicks off a new job.  
    * JobTracker.submitJob() kicks off a new job.  
    *
    *
@@ -1800,12 +1805,22 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                                state);          
                                state);          
     }
     }
   }
   }
-    
+
+  @Deprecated
+  public void killJob(String id) {
+    killJob(JobID.forName(id));
+  }
+
   public synchronized void killJob(JobID jobid) {
   public synchronized void killJob(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     job.kill();
     job.kill();
   }
   }
 
 
+  @Deprecated
+  public JobProfile getJobProfile(String id) {
+    return getJobProfile(JobID.forName(id));
+  }
+
   public synchronized JobProfile getJobProfile(JobID jobid) {
   public synchronized JobProfile getJobProfile(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
@@ -1814,6 +1829,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       return completedJobStatusStore.readJobProfile(jobid);
       return completedJobStatusStore.readJobProfile(jobid);
     }
     }
   }
   }
+  
+  @Deprecated
+  public JobStatus getJobStatus(String id) {
+    return getJobStatus(JobID.forName(id));
+  }
+
   public synchronized JobStatus getJobStatus(JobID jobid) {
   public synchronized JobStatus getJobStatus(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
@@ -1822,6 +1843,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       return completedJobStatusStore.readJobStatus(jobid);
       return completedJobStatusStore.readJobStatus(jobid);
     }
     }
   }
   }
+
+  @Deprecated
+  public Counters getJobCounters(String id) {
+    return getJobCounters(JobID.forName(id));
+  }
+
   public synchronized Counters getJobCounters(JobID jobid) {
   public synchronized Counters getJobCounters(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
@@ -1830,6 +1857,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       return completedJobStatusStore.readCounters(jobid);
       return completedJobStatusStore.readCounters(jobid);
     }
     }
   }
   }
+  
+  @Deprecated
+  public TaskReport[] getMapTaskReports(String jobid) {
+    return getMapTaskReports(JobID.forName(jobid));
+  }
+
   public synchronized TaskReport[] getMapTaskReports(JobID jobid) {
   public synchronized TaskReport[] getMapTaskReports(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job == null) {
     if (job == null) {
@@ -1852,6 +1885,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     }
     }
   }
   }
 
 
+  @Deprecated
+  public TaskReport[] getReduceTaskReports(String jobid) {
+    return getReduceTaskReports(JobID.forName(jobid));
+  }
+
   public synchronized TaskReport[] getReduceTaskReports(JobID jobid) {
   public synchronized TaskReport[] getReduceTaskReports(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job == null) {
     if (job == null) {
@@ -1871,7 +1909,14 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       return reports.toArray(new TaskReport[reports.size()]);
       return reports.toArray(new TaskReport[reports.size()]);
     }
     }
   }
   }
-    
+
+  @Deprecated
+  public TaskCompletionEvent[] getTaskCompletionEvents(String jobid, int fromid, 
+                                                       int maxevents
+                                                      ) throws IOException {
+    return getTaskCompletionEvents(JobID.forName(jobid), fromid, maxevents);
+  }
+
   /* 
   /* 
    * Returns a list of TaskCompletionEvent for the given job, 
    * Returns a list of TaskCompletionEvent for the given job, 
    * starting from fromEventId.
    * starting from fromEventId.
@@ -1891,6 +1936,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     return events;
     return events;
   }
   }
 
 
+  @Deprecated
+  public String[] getTaskDiagnostics(String jobid, String tipid, 
+                                     String taskid) throws IOException {
+    return getTaskDiagnostics(TaskAttemptID.forName(taskid));
+  }
+
   /**
   /**
    * Get the diagnostics for a given task
    * Get the diagnostics for a given task
    * @param taskId the id of the task
    * @param taskId the id of the task
@@ -1943,7 +1994,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     JobInProgress job = jobs.get(tipid.getJobID());
     JobInProgress job = jobs.get(tipid.getJobID());
     return (job == null ? null : job.getTaskInProgress(tipid));
     return (job == null ? null : job.getTaskInProgress(tipid));
   }
   }
-    
+
+  @Deprecated
+  public boolean killTask(String taskId, boolean shouldFail) throws IOException{
+    return killTask(TaskAttemptID.forName(taskId), shouldFail);
+  }
+
   /** Mark a Task to be killed */
   /** Mark a Task to be killed */
   public synchronized boolean killTask(TaskAttemptID taskid, boolean shouldFail) throws IOException{
   public synchronized boolean killTask(TaskAttemptID taskid, boolean shouldFail) throws IOException{
     TaskInProgress tip = taskidToTIPMap.get(taskid);
     TaskInProgress tip = taskidToTIPMap.get(taskid);
@@ -1955,7 +2011,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       return false;
       return false;
     }
     }
   }
   }
-  
+
+  @Deprecated
+  public String getAssignedTracker(String taskid) {
+    return getAssignedTracker(TaskAttemptID.forName(taskid));
+  }
+
   /**
   /**
    * Get tracker name for a given task id.
    * Get tracker name for a given task id.
    * @param taskId the name of the task
    * @param taskId the name of the task
@@ -2003,6 +2064,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   ///////////////////////////////////////////////////////////////
   ///////////////////////////////////////////////////////////////
   // JobTracker methods
   // JobTracker methods
   ///////////////////////////////////////////////////////////////
   ///////////////////////////////////////////////////////////////
+  @Deprecated
+  public JobInProgress getJob(String jobid) {
+    return getJob(JobID.forName(jobid));
+  }
+
   public JobInProgress getJob(JobID jobid) {
   public JobInProgress getJob(JobID jobid) {
     return jobs.get(jobid);
     return jobs.get(jobid);
   }
   }
@@ -2300,6 +2366,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   }
   }
   
   
 
 
+  @Deprecated
+  public String getLocalJobFilePath(String jobid) {
+    return getLocalJobFilePath(JobID.forName(jobid));
+  }
+
   /**
   /**
    * Get the localized job file path on the job trackers local file system
    * Get the localized job file path on the job trackers local file system
    * @param jobId id of the job
    * @param jobId id of the job

+ 5 - 0
src/mapred/org/apache/hadoop/mapred/LineRecordReader.java

@@ -222,6 +222,11 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
     }
     }
     this.pos = start;
     this.pos = start;
   }
   }
+
+  @Deprecated
+  public LineRecordReader(InputStream in, long offset, long endOffset) {
+    this(in, offset, endOffset, Integer.MAX_VALUE);
+  }
   
   
   public LineRecordReader(InputStream in, long offset, long endOffset,
   public LineRecordReader(InputStream in, long offset, long endOffset,
                           int maxLineLength) {
                           int maxLineLength) {

+ 12 - 0
src/mapred/org/apache/hadoop/mapred/TaskCompletionEvent.java

@@ -46,6 +46,18 @@ public class TaskCompletionEvent implements Writable{
    *
    *
    */
    */
   public TaskCompletionEvent(){}
   public TaskCompletionEvent(){}
+  
+  @Deprecated
+  public TaskCompletionEvent(int eventId, 
+                             String taskId,
+                             int idWithinJob,
+                             boolean isMap,
+                             Status status, 
+                             String taskTrackerHttp){
+    this(eventId, TaskAttemptID.forName(taskId), idWithinJob, isMap, status, 
+         taskTrackerHttp);
+  }
+
   /**
   /**
    * Constructor. eventId should be created externally and incremented
    * Constructor. eventId should be created externally and incremented
    * per event for each job. 
    * per event for each job. 

+ 6 - 1
src/mapred/org/apache/hadoop/mapred/TaskLog.java

@@ -49,6 +49,11 @@ public class TaskLog {
     }
     }
   }
   }
 
 
+  @Deprecated
+  public static File getTaskLogFile(String taskid, LogName filter) {
+    return getTaskLogFile(TaskAttemptID.forName(taskid), filter);
+  }
+
   public static File getTaskLogFile(TaskAttemptID taskid, LogName filter) {
   public static File getTaskLogFile(TaskAttemptID taskid, LogName filter) {
     return new File(new File(LOG_DIR, taskid.toString()), filter.toString());
     return new File(new File(LOG_DIR, taskid.toString()), filter.toString());
   }
   }
@@ -116,7 +121,7 @@ public class TaskLog {
     }
     }
   }
   }
 
 
-  public static class Reader extends InputStream {
+  static class Reader extends InputStream {
     private long bytesRemaining;
     private long bytesRemaining;
     private FileInputStream file;
     private FileInputStream file;
     /**
     /**

+ 52 - 1
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -1986,6 +1986,12 @@ public class TaskTracker
   // ///////////////////////////////////////////////////////////////
   // ///////////////////////////////////////////////////////////////
   // TaskUmbilicalProtocol
   // TaskUmbilicalProtocol
   /////////////////////////////////////////////////////////////////
   /////////////////////////////////////////////////////////////////
+  
+  @Deprecated
+  public Task getTask(String id) throws IOException {
+    return getTask(TaskAttemptID.forName(id));
+  }
+
   /**
   /**
    * Called upon startup by the child process, to fetch Task data.
    * Called upon startup by the child process, to fetch Task data.
    */
    */
@@ -1998,6 +2004,12 @@ public class TaskTracker
     }
     }
   }
   }
 
 
+  @Deprecated
+  public boolean statusUpdate(String taskid, 
+                              TaskStatus status) throws IOException {
+    return statusUpdate(TaskAttemptID.forName(taskid), status);
+  }
+
   /**
   /**
    * Called periodically to report Task progress, from 0.0 to 1.0.
    * Called periodically to report Task progress, from 0.0 to 1.0.
    */
    */
@@ -2014,11 +2026,19 @@ public class TaskTracker
     }
     }
   }
   }
 
 
+  @Deprecated
+  public void reportDiagnosticInfo(String taskid, 
+                                   String info) throws IOException {
+    reportDiagnosticInfo(TaskAttemptID.forName(taskid), info);
+  }
+
   /**
   /**
    * Called when the task dies before completion, and we want to report back
    * Called when the task dies before completion, and we want to report back
    * diagnostic info
    * diagnostic info
    */
    */
-  public synchronized void reportDiagnosticInfo(TaskAttemptID taskid, String info) throws IOException {
+  public synchronized void reportDiagnosticInfo(TaskAttemptID taskid, 
+                                                String info
+                                                ) throws IOException {
     TaskInProgress tip = tasks.get(taskid);
     TaskInProgress tip = tasks.get(taskid);
     if (tip != null) {
     if (tip != null) {
       tip.reportDiagnosticInfo(info);
       tip.reportDiagnosticInfo(info);
@@ -2027,11 +2047,21 @@ public class TaskTracker
     }
     }
   }
   }
 
 
+  @Deprecated
+  public boolean ping(String taskid) throws IOException {
+    return ping(TaskAttemptID.forName(taskid));
+  }
+
   /** Child checking to see if we're alive.  Normally does nothing.*/
   /** Child checking to see if we're alive.  Normally does nothing.*/
   public synchronized boolean ping(TaskAttemptID taskid) throws IOException {
   public synchronized boolean ping(TaskAttemptID taskid) throws IOException {
     return tasks.get(taskid) != null;
     return tasks.get(taskid) != null;
   }
   }
 
 
+  @Deprecated
+  public void done(String taskid, boolean shouldPromote) throws IOException {
+    done(TaskAttemptID.forName(taskid), shouldPromote);
+  }
+
   /**
   /**
    * The task is done.
    * The task is done.
    */
    */
@@ -2045,6 +2075,10 @@ public class TaskTracker
     }
     }
   }
   }
 
 
+  @Deprecated 
+  public void shuffleError(String taskid, String msg) throws IOException {
+    shuffleError(TaskAttemptID.forName(taskid), msg);
+  }
 
 
   /** 
   /** 
    * A reduce-task failed to shuffle the map-outputs. Kill the task.
    * A reduce-task failed to shuffle the map-outputs. Kill the task.
@@ -2057,6 +2091,11 @@ public class TaskTracker
     purgeTask(tip, true);
     purgeTask(tip, true);
   }
   }
 
 
+  @Deprecated
+  public void fsError(String taskid, String msg) throws IOException {
+    fsError(TaskAttemptID.forName(taskid), msg);
+  }
+
   /** 
   /** 
    * A child task had a local filesystem error. Kill the task.
    * A child task had a local filesystem error. Kill the task.
    */  
    */  
@@ -2068,6 +2107,13 @@ public class TaskTracker
     purgeTask(tip, true);
     purgeTask(tip, true);
   }
   }
 
 
+  @Deprecated
+  public TaskCompletionEvent[] getMapCompletionEvents(String jobid, int fromid, 
+                                                      int maxlocs
+                                                     ) throws IOException {
+    return getMapCompletionEvents(JobID.forName(jobid), fromid, maxlocs);
+  }
+
   public TaskCompletionEvent[] getMapCompletionEvents(JobID jobId
   public TaskCompletionEvent[] getMapCompletionEvents(JobID jobId
       , int fromEventId, int maxLocs) throws IOException {
       , int fromEventId, int maxLocs) throws IOException {
       
       
@@ -2109,6 +2155,11 @@ public class TaskTracker
     }
     }
   }
   }
 
 
+  @Deprecated
+  public void mapOutputLost(String taskid, String msg) throws IOException {
+    mapOutputLost(TaskAttemptID.forName(taskid), msg);
+  }
+
   /**
   /**
    * A completed map task's output has been lost.
    * A completed map task's output has been lost.
    */
    */

+ 8 - 9
src/test/org/apache/hadoop/dfs/NNBench.java

@@ -57,7 +57,6 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.TaskTracker;
 
 
 /**
 /**
  * This program executes a specified operation that applies load to 
  * This program executes a specified operation that applies load to 
@@ -79,7 +78,7 @@ import org.apache.hadoop.mapred.TaskTracker;
  */
  */
 
 
 public class NNBench {
 public class NNBench {
-  protected static final Log LOG = LogFactory.getLog(
+  private static final Log LOG = LogFactory.getLog(
           "org.apache.hadoop.dfs.NNBench");
           "org.apache.hadoop.dfs.NNBench");
   
   
   protected static String CONTROL_DIR_NAME = "control";
   protected static String CONTROL_DIR_NAME = "control";
@@ -655,7 +654,7 @@ public class NNBench {
       
       
       // If the sleep time is greater than 0, then sleep and return
       // If the sleep time is greater than 0, then sleep and return
       if (sleepTime > 0) {
       if (sleepTime > 0) {
-        TaskTracker.LOG.info("Waiting in barrier for: " + sleepTime + " ms");
+        LOG.info("Waiting in barrier for: " + sleepTime + " ms");
       
       
         try {
         try {
           Thread.sleep(sleepTime);
           Thread.sleep(sleepTime);
@@ -773,7 +772,7 @@ public class NNBench {
 
 
             reporter.setStatus("Finish "+ l + " files");
             reporter.setStatus("Finish "+ l + " files");
           } catch (IOException e) {
           } catch (IOException e) {
-            TaskTracker.LOG.info("Exception recorded in op: " +
+            LOG.info("Exception recorded in op: " +
                     "Create/Write/Close");
                     "Create/Write/Close");
  
  
             numOfExceptions++;
             numOfExceptions++;
@@ -816,7 +815,7 @@ public class NNBench {
 
 
             reporter.setStatus("Finish "+ l + " files");
             reporter.setStatus("Finish "+ l + " files");
           } catch (IOException e) {
           } catch (IOException e) {
-            TaskTracker.LOG.info("Exception recorded in op: OpenRead " + e);
+            LOG.info("Exception recorded in op: OpenRead " + e);
             numOfExceptions++;
             numOfExceptions++;
           }
           }
         }
         }
@@ -848,7 +847,7 @@ public class NNBench {
 
 
             reporter.setStatus("Finish "+ l + " files");
             reporter.setStatus("Finish "+ l + " files");
           } catch (IOException e) {
           } catch (IOException e) {
-            TaskTracker.LOG.info("Exception recorded in op: Rename");
+            LOG.info("Exception recorded in op: Rename");
 
 
             numOfExceptions++;
             numOfExceptions++;
           }
           }
@@ -879,7 +878,7 @@ public class NNBench {
 
 
             reporter.setStatus("Finish "+ l + " files");
             reporter.setStatus("Finish "+ l + " files");
           } catch (IOException e) {
           } catch (IOException e) {
-            TaskTracker.LOG.info("Exception in recorded op: Delete");
+            LOG.info("Exception in recorded op: Delete");
 
 
             numOfExceptions++;
             numOfExceptions++;
           }
           }
@@ -897,13 +896,13 @@ public class NNBench {
     protected String hostName;
     protected String hostName;
 
 
     public NNBenchReducer () {
     public NNBenchReducer () {
-      TaskTracker.LOG.info("Starting NNBenchReducer !!!");
+      LOG.info("Starting NNBenchReducer !!!");
       try {
       try {
         hostName = java.net.InetAddress.getLocalHost().getHostName();
         hostName = java.net.InetAddress.getLocalHost().getHostName();
       } catch(Exception e) {
       } catch(Exception e) {
         hostName = "localhost";
         hostName = "localhost";
       }
       }
-      TaskTracker.LOG.info("Starting NNBenchReducer on " + hostName);
+      LOG.info("Starting NNBenchReducer on " + hostName);
     }
     }
 
 
     /**
     /**

+ 5 - 3
src/test/org/apache/hadoop/fs/AccumulatingReducer.java

@@ -20,13 +20,14 @@ package org.apache.hadoop.fs;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.Iterator;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.MapReduceBase;
 import org.apache.hadoop.mapred.MapReduceBase;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.TaskTracker;
 
 
 /**
 /**
  * Reducer that accumulates values based on their type.
  * Reducer that accumulates values based on their type.
@@ -46,17 +47,18 @@ import org.apache.hadoop.mapred.TaskTracker;
  */
  */
 public class AccumulatingReducer extends MapReduceBase
 public class AccumulatingReducer extends MapReduceBase
     implements Reducer<UTF8, UTF8, UTF8, UTF8> {
     implements Reducer<UTF8, UTF8, UTF8, UTF8> {
+  private static final Log LOG = LogFactory.getLog(AccumulatingReducer.class);
   
   
   protected String hostName;
   protected String hostName;
   
   
   public AccumulatingReducer () {
   public AccumulatingReducer () {
-    TaskTracker.LOG.info("Starting AccumulatingReducer !!!");
+    LOG.info("Starting AccumulatingReducer !!!");
     try {
     try {
       hostName = java.net.InetAddress.getLocalHost().getHostName();
       hostName = java.net.InetAddress.getLocalHost().getHostName();
     } catch(Exception e) {
     } catch(Exception e) {
       hostName = "localhost";
       hostName = "localhost";
     }
     }
-    TaskTracker.LOG.info("Starting AccumulatingReducer on " + hostName);
+    LOG.info("Starting AccumulatingReducer on " + hostName);
   }
   }
   
   
   public void reduce(UTF8 key, 
   public void reduce(UTF8 key,