Pārlūkot izejas kodu

HADOOP-544. This issue introduces new classes JobID, TaskInProgressID and TaskID, which should be used instead of their string counterparts. Functions in JobClient, TaskReport, RunningJob, jobcontrol.Job and TaskCompletionEvent that use string arguments are deprecated in favor of the corresponding ones that use ID objects. Applications can use xxxID.toString() and xxxID.forName() methods to convert/restore objects to/from strings. Contributed by Enis Soztutar.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@652364 13f79535-47bb-0310-9956-ffa450edef68
Devaraj Das 17 gadi atpakaļ
vecāks
revīzija
177c86f9aa
56 mainītis faili ar 1069 papildinājumiem un 958 dzēšanām
  1. 8 1
      CHANGES.txt
  2. 2 1
      src/contrib/data_join/src/java/org/apache/hadoop/contrib/utils/join/DataJoinJob.java
  3. 21 15
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
  4. 1 1
      src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
  5. 14 10
      src/java/org/apache/hadoop/mapred/CompletedJobStatusStore.java
  6. 2 2
      src/java/org/apache/hadoop/mapred/FileOutputFormat.java
  7. 5 4
      src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java
  8. 16 20
      src/java/org/apache/hadoop/mapred/IsolationRunner.java
  9. 65 37
      src/java/org/apache/hadoop/mapred/JobClient.java
  10. 4 1
      src/java/org/apache/hadoop/mapred/JobEndNotifier.java
  11. 50 67
      src/java/org/apache/hadoop/mapred/JobHistory.java
  12. 56 71
      src/java/org/apache/hadoop/mapred/JobInProgress.java
  13. 21 16
      src/java/org/apache/hadoop/mapred/JobProfile.java
  14. 8 8
      src/java/org/apache/hadoop/mapred/JobStatus.java
  15. 16 15
      src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java
  16. 91 101
      src/java/org/apache/hadoop/mapred/JobTracker.java
  17. 8 7
      src/java/org/apache/hadoop/mapred/KillJobAction.java
  18. 7 6
      src/java/org/apache/hadoop/mapred/KillTaskAction.java
  19. 38 53
      src/java/org/apache/hadoop/mapred/LocalJobRunner.java
  20. 15 15
      src/java/org/apache/hadoop/mapred/MapOutputFile.java
  21. 23 19
      src/java/org/apache/hadoop/mapred/MapOutputLocation.java
  22. 33 21
      src/java/org/apache/hadoop/mapred/MapTask.java
  23. 2 2
      src/java/org/apache/hadoop/mapred/MapTaskRunner.java
  24. 7 1
      src/java/org/apache/hadoop/mapred/MapTaskStatus.java
  25. 75 64
      src/java/org/apache/hadoop/mapred/ReduceTask.java
  26. 2 2
      src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java
  27. 23 11
      src/java/org/apache/hadoop/mapred/ReduceTaskStatus.java
  28. 13 2
      src/java/org/apache/hadoop/mapred/RunningJob.java
  29. 3 2
      src/java/org/apache/hadoop/mapred/StatusHttpServer.java
  30. 29 39
      src/java/org/apache/hadoop/mapred/Task.java
  31. 29 6
      src/java/org/apache/hadoop/mapred/TaskCompletionEvent.java
  32. 44 95
      src/java/org/apache/hadoop/mapred/TaskInProgress.java
  33. 15 5
      src/java/org/apache/hadoop/mapred/TaskLog.java
  34. 3 2
      src/java/org/apache/hadoop/mapred/TaskLogAppender.java
  35. 11 6
      src/java/org/apache/hadoop/mapred/TaskLogServlet.java
  36. 8 5
      src/java/org/apache/hadoop/mapred/TaskReport.java
  37. 14 13
      src/java/org/apache/hadoop/mapred/TaskRunner.java
  38. 15 14
      src/java/org/apache/hadoop/mapred/TaskStatus.java
  39. 87 79
      src/java/org/apache/hadoop/mapred/TaskTracker.java
  40. 14 11
      src/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java
  41. 38 23
      src/java/org/apache/hadoop/mapred/jobcontrol/Job.java
  42. 2 1
      src/java/org/apache/hadoop/mapred/pipes/Application.java
  43. 11 9
      src/test/org/apache/hadoop/mapred/TestJobStatusPersistency.java
  44. 21 11
      src/test/org/apache/hadoop/mapred/TestMapRed.java
  45. 14 9
      src/test/org/apache/hadoop/mapred/TestMiniMRLocalFS.java
  46. 13 11
      src/test/org/apache/hadoop/mapred/TestMiniMRMapRedDebugScript.java
  47. 14 7
      src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java
  48. 1 1
      src/webapps/job/jobblacklistedtrackers.jsp
  49. 1 1
      src/webapps/job/jobconf.jsp
  50. 8 4
      src/webapps/job/jobdetails.jsp
  51. 12 8
      src/webapps/job/jobfailures.jsp
  52. 10 5
      src/webapps/job/jobtasks.jsp
  53. 1 1
      src/webapps/job/jobtracker.jsp
  54. 15 12
      src/webapps/job/taskdetails.jsp
  55. 8 3
      src/webapps/job/taskstats.jsp
  56. 2 2
      src/webapps/task/tasktracker.jsp

+ 8 - 1
CHANGES.txt

@@ -22,7 +22,14 @@ Trunk (unreleased changes)
     is not supported. If upgrading from 0.13 or earlier is required,
     is not supported. If upgrading from 0.13 or earlier is required,
     please upgrade to an intermediate version (0.14-0.17) and then
     please upgrade to an intermediate version (0.14-0.17) and then
     to this version. (rangadi)
     to this version. (rangadi)
- 
+
+    HADOOP-544. This issue introduces new classes JobID, TaskInProgressID
+    and TaskID, which should be used instead of their string counterparts.
+    Functions in JobClient, TaskReport, RunningJob, jobcontrol.Job and 
+    TaskCompletionEvent that use string arguments are deprecated in favor 
+    of the corresponding ones that use ID objects. Applications can use 
+    xxxID.toString() and xxxID.forName() methods to convert/restore objects 
+    to/from strings. (Enis Soztutar via ddas)
 
 
   NEW FEATURES
   NEW FEATURES
 
 

+ 2 - 1
src/contrib/data_join/src/java/org/apache/hadoop/contrib/utils/join/DataJoinJob.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.JobID;
 
 
 /**
 /**
  * This class implements the main function for creating a map/reduce
  * This class implements the main function for creating a map/reduce
@@ -127,7 +128,7 @@ public class DataJoinJob {
     RunningJob running = null;
     RunningJob running = null;
     try {
     try {
       running = jc.submitJob(job);
       running = jc.submitJob(job);
-      String jobId = running.getJobID();
+      JobID jobId = running.getID();
       System.out.println("Job " + jobId + " is submitted");
       System.out.println("Job " + jobId + " is submitted");
       while (!running.isComplete()) {
       while (!running.isComplete()) {
         System.out.println("Job " + jobId + " is still running.");
         System.out.println("Job " + jobId + " is still running.");

+ 21 - 15
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java

@@ -32,7 +32,12 @@ import java.util.Map;
 import java.util.TreeMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.TreeSet;
 
 
-import org.apache.commons.cli2.*; 
+import org.apache.commons.cli2.Argument;
+import org.apache.commons.cli2.CommandLine;
+import org.apache.commons.cli2.Group;
+import org.apache.commons.cli2.Option;
+import org.apache.commons.cli2.OptionException;
+import org.apache.commons.cli2.WriteableCommandLine;
 import org.apache.commons.cli2.builder.ArgumentBuilder;
 import org.apache.commons.cli2.builder.ArgumentBuilder;
 import org.apache.commons.cli2.builder.DefaultOptionBuilder;
 import org.apache.commons.cli2.builder.DefaultOptionBuilder;
 import org.apache.commons.cli2.builder.GroupBuilder;
 import org.apache.commons.cli2.builder.GroupBuilder;
@@ -42,29 +47,28 @@ import org.apache.commons.cli2.resource.ResourceConstants;
 import org.apache.commons.cli2.util.HelpFormatter;
 import org.apache.commons.cli2.util.HelpFormatter;
 import org.apache.commons.cli2.validation.InvalidArgumentException;
 import org.apache.commons.cli2.validation.InvalidArgumentException;
 import org.apache.commons.cli2.validation.Validator;
 import org.apache.commons.cli2.validation.Validator;
-import org.apache.commons.logging.*;
-
-import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorCombiner;
-import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorReducer;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.InvalidJobConfException;
 import org.apache.hadoop.mapred.InvalidJobConfException;
-import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapred.KeyValueTextInputFormat;
 import org.apache.hadoop.mapred.KeyValueTextInputFormat;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileAsTextInputFormat;
 import org.apache.hadoop.mapred.SequenceFileAsTextInputFormat;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.TextOutputFormat;
-import org.apache.hadoop.filecache.*;
-import org.apache.hadoop.util.*;
+import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorCombiner;
+import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorReducer;
+import org.apache.hadoop.util.StringUtils;
 
 
 /** All the client-side work happens here.
 /** All the client-side work happens here.
  * (Jar packaging, MapRed job submission and monitoring)
  * (Jar packaging, MapRed job submission and monitoring)
@@ -638,7 +642,7 @@ public class StreamJob {
     Iterator it = userJobConfProps_.keySet().iterator();
     Iterator it = userJobConfProps_.keySet().iterator();
     while (it.hasNext()) {
     while (it.hasNext()) {
       String key = (String) it.next();
       String key = (String) it.next();
-      String val = (String)userJobConfProps_.get(key);
+      String val = userJobConfProps_.get(key);
       boolean earlyName = key.equals("fs.default.name");
       boolean earlyName = key.equals("fs.default.name");
       earlyName |= key.equals("stream.shipped.hadoopstreaming");
       earlyName |= key.equals("stream.shipped.hadoopstreaming");
       if (doEarlyProps == earlyName) {
       if (doEarlyProps == earlyName) {
@@ -919,7 +923,7 @@ public class StreamJob {
     String lastReport = null;
     String lastReport = null;
     try {
     try {
       running_ = jc_.submitJob(jobConf_);
       running_ = jc_.submitJob(jobConf_);
-      jobId_ = running_.getJobID();
+      jobId_ = running_.getID();
 
 
       LOG.info("getLocalDirs(): " + Arrays.asList(jobConf_.getLocalDirs()));
       LOG.info("getLocalDirs(): " + Arrays.asList(jobConf_.getLocalDirs()));
       LOG.info("Running job: " + jobId_);
       LOG.info("Running job: " + jobId_);
@@ -984,12 +988,14 @@ public class StreamJob {
       this.optionString = optionString;
       this.optionString = optionString;
     }
     }
 
 
+    @Override
     public boolean canProcess(final WriteableCommandLine commandLine,
     public boolean canProcess(final WriteableCommandLine commandLine,
                               final String argument) {
                               final String argument) {
       boolean ret = (argument != null) && argument.startsWith(optionString);
       boolean ret = (argument != null) && argument.startsWith(optionString);
         
         
       return ret;
       return ret;
     }    
     }    
+    @Override
     public void process(final WriteableCommandLine commandLine,
     public void process(final WriteableCommandLine commandLine,
                         final ListIterator arguments) throws OptionException {
                         final ListIterator arguments) throws OptionException {
       final String arg = (String) arguments.next();
       final String arg = (String) arguments.next();
@@ -1070,7 +1076,7 @@ public class StreamJob {
   protected long minRecWrittenToEnableSkip_;
   protected long minRecWrittenToEnableSkip_;
 
 
   protected RunningJob running_;
   protected RunningJob running_;
-  protected String jobId_;
+  protected JobID jobId_;
   protected static String LINK_URI = "You need to specify the uris as hdfs://host:port/#linkname," +
   protected static String LINK_URI = "You need to specify the uris as hdfs://host:port/#linkname," +
     "Please specify a different link name for all of your caching URIs";
     "Please specify a different link name for all of your caching URIs";
 }
 }

+ 1 - 1
src/docs/src/documentation/content/xdocs/mapred_tutorial.xml

@@ -1320,7 +1320,7 @@
           example, speculative tasks) trying to open and/or write to the same 
           example, speculative tasks) trying to open and/or write to the same 
           file (path) on the <code>FileSystem</code>. Hence the 
           file (path) on the <code>FileSystem</code>. Hence the 
           application-writer will have to pick unique names per task-attempt 
           application-writer will have to pick unique names per task-attempt 
-          (using the taskid, say <code>task_200709221812_0001_m_000000_0</code>), 
+          (using the attemptid, say <code>attempt_200709221812_0001_m_000000_0</code>), 
           not just per task.</p> 
           not just per task.</p> 
  
  
           <p>To avoid these issues the Map-Reduce framework maintains a special 
           <p>To avoid these issues the Map-Reduce framework maintains a special 

+ 14 - 10
src/java/org/apache/hadoop/mapred/CompletedJobStatusStore.java

@@ -17,12 +17,16 @@
  */  
  */  
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
+import java.io.IOException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-
-import java.io.IOException;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 
 /**
 /**
  * Persists and retrieves the Job info of a job into/from DFS.
  * Persists and retrieves the Job info of a job into/from DFS.
@@ -118,7 +122,7 @@ public class CompletedJobStatusStore implements Runnable {
     }
     }
   }
   }
 
 
-  private Path getInfoFilePath(String jobId) {
+  private Path getInfoFilePath(JobID jobId) {
     return new Path(jobInfoDir, jobId + ".info");
     return new Path(jobInfoDir, jobId + ".info");
   }
   }
   
   
@@ -129,7 +133,7 @@ public class CompletedJobStatusStore implements Runnable {
    */
    */
   public void store(JobInProgress job) {
   public void store(JobInProgress job) {
     if (active && retainTime > 0) {
     if (active && retainTime > 0) {
-      String jobId = job.getStatus().getJobId();
+      JobID jobId = job.getStatus().getJobID();
       Path jobStatusFile = getInfoFilePath(jobId);
       Path jobStatusFile = getInfoFilePath(jobId);
       try {
       try {
         FSDataOutputStream dataOut = fs.create(jobStatusFile);
         FSDataOutputStream dataOut = fs.create(jobStatusFile);
@@ -161,7 +165,7 @@ public class CompletedJobStatusStore implements Runnable {
     }
     }
   }
   }
 
 
-  private FSDataInputStream getJobInfoFile(String jobId) throws IOException {
+  private FSDataInputStream getJobInfoFile(JobID jobId) throws IOException {
     Path jobStatusFile = getInfoFilePath(jobId);
     Path jobStatusFile = getInfoFilePath(jobId);
     return (fs.exists(jobStatusFile)) ? fs.open(jobStatusFile) : null;
     return (fs.exists(jobStatusFile)) ? fs.open(jobStatusFile) : null;
   }
   }
@@ -213,7 +217,7 @@ public class CompletedJobStatusStore implements Runnable {
    * @param jobId the jobId for which jobStatus is queried
    * @param jobId the jobId for which jobStatus is queried
    * @return JobStatus object, null if not able to retrieve
    * @return JobStatus object, null if not able to retrieve
    */
    */
-  public JobStatus readJobStatus(String jobId) {
+  public JobStatus readJobStatus(JobID jobId) {
     JobStatus jobStatus = null;
     JobStatus jobStatus = null;
     if (active) {
     if (active) {
       try {
       try {
@@ -236,7 +240,7 @@ public class CompletedJobStatusStore implements Runnable {
    * @param jobId the jobId for which jobProfile is queried
    * @param jobId the jobId for which jobProfile is queried
    * @return JobProfile object, null if not able to retrieve
    * @return JobProfile object, null if not able to retrieve
    */
    */
-  public JobProfile readJobProfile(String jobId) {
+  public JobProfile readJobProfile(JobID jobId) {
     JobProfile jobProfile = null;
     JobProfile jobProfile = null;
     if (active) {
     if (active) {
       try {
       try {
@@ -260,7 +264,7 @@ public class CompletedJobStatusStore implements Runnable {
    * @param jobId the jobId for which Counters is queried
    * @param jobId the jobId for which Counters is queried
    * @return Counters object, null if not able to retrieve
    * @return Counters object, null if not able to retrieve
    */
    */
-  public Counters readCounters(String jobId) {
+  public Counters readCounters(JobID jobId) {
     Counters counters = null;
     Counters counters = null;
     if (active) {
     if (active) {
       try {
       try {
@@ -287,7 +291,7 @@ public class CompletedJobStatusStore implements Runnable {
    * @param maxEvents   max number of events
    * @param maxEvents   max number of events
    * @return TaskCompletionEvent[], empty array if not able to retrieve
    * @return TaskCompletionEvent[], empty array if not able to retrieve
    */
    */
-  public TaskCompletionEvent[] readJobTaskCompletionEvents(String jobId,
+  public TaskCompletionEvent[] readJobTaskCompletionEvents(JobID jobId,
                                                                int fromEventId,
                                                                int fromEventId,
                                                                int maxEvents) {
                                                                int maxEvents) {
     TaskCompletionEvent[] events = TaskCompletionEvent.EMPTY_ARRAY;
     TaskCompletionEvent[] events = TaskCompletionEvent.EMPTY_ARRAY;

+ 2 - 2
src/java/org/apache/hadoop/mapred/FileOutputFormat.java

@@ -157,8 +157,8 @@ public abstract class FileOutputFormat<K, V> implements OutputFormat<K, V> {
    * <p>In such cases there could be issues with 2 instances of the same TIP 
    * <p>In such cases there could be issues with 2 instances of the same TIP 
    * (running simultaneously e.g. speculative tasks) trying to open/write-to the
    * (running simultaneously e.g. speculative tasks) trying to open/write-to the
    * same file (path) on HDFS. Hence the application-writer will have to pick 
    * same file (path) on HDFS. Hence the application-writer will have to pick 
-   * unique names per task-attempt (e.g. using the taskid, say 
-   * <tt>task_200709221812_0001_m_000000_0</tt>), not just per TIP.</p> 
+   * unique names per task-attempt (e.g. using the attemptid, say 
+   * <tt>attempt_200709221812_0001_m_000000_0</tt>), not just per TIP.</p> 
    * 
    * 
    * <p>To get around this the Map-Reduce framework helps the application-writer 
    * <p>To get around this the Map-Reduce framework helps the application-writer 
    * out by maintaining a special 
    * out by maintaining a special 

+ 5 - 4
src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java

@@ -18,7 +18,7 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import java.io.*;
+import java.io.IOException;
 
 
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.ipc.VersionedProtocol;
 
 
@@ -40,8 +40,9 @@ interface InterTrackerProtocol extends VersionedProtocol {
    * Version 8: HeartbeatResponse is added with the next heartbeat interval.
    * Version 8: HeartbeatResponse is added with the next heartbeat interval.
    * version 9 changes the counter representation for HADOOP-2248
    * version 9 changes the counter representation for HADOOP-2248
    * version 10 changes the TaskStatus representation for HADOOP-2208
    * version 10 changes the TaskStatus representation for HADOOP-2208
+   * version 11 changes string to JobID in getTaskCompletionEvents().
    */
    */
-  public static final long versionID = 10L;
+  public static final long versionID = 11L;
   
   
   public final static int TRACKERS_OK = 0;
   public final static int TRACKERS_OK = 0;
   public final static int UNKNOWN_TASKTRACKER = 1;
   public final static int UNKNOWN_TASKTRACKER = 1;
@@ -97,8 +98,8 @@ interface InterTrackerProtocol extends VersionedProtocol {
    * @return array of task completion events. 
    * @return array of task completion events. 
    * @throws IOException
    * @throws IOException
    */
    */
-  TaskCompletionEvent[] getTaskCompletionEvents(
-                                                String jobid, int fromEventId, int maxEvents) throws IOException;
+  TaskCompletionEvent[] getTaskCompletionEvents(JobID jobid, int fromEventId
+      , int maxEvents) throws IOException;
   
   
 }
 }
 
 

+ 16 - 20
src/java/org/apache/hadoop/mapred/IsolationRunner.java

@@ -44,27 +44,27 @@ public class IsolationRunner {
       return TaskUmbilicalProtocol.versionID;
       return TaskUmbilicalProtocol.versionID;
     }
     }
     
     
-    public void done(String taskid, boolean shouldPromote) throws IOException {
+    public void done(TaskAttemptID taskid, boolean shouldPromote) throws IOException {
       LOG.info("Task " + taskid + " reporting done.");
       LOG.info("Task " + taskid + " reporting done.");
     }
     }
 
 
-    public void fsError(String taskId, String message) throws IOException {
+    public void fsError(TaskAttemptID taskId, String message) throws IOException {
       LOG.info("Task " + taskId + " reporting file system error: " + message);
       LOG.info("Task " + taskId + " reporting file system error: " + message);
     }
     }
 
 
-    public void shuffleError(String taskId, String message) throws IOException {
+    public void shuffleError(TaskAttemptID taskId, String message) throws IOException {
       LOG.info("Task " + taskId + " reporting shuffle error: " + message);
       LOG.info("Task " + taskId + " reporting shuffle error: " + message);
     }
     }
 
 
-    public Task getTask(String taskid) throws IOException {
+    public Task getTask(TaskAttemptID taskid) throws IOException {
       return null;
       return null;
     }
     }
 
 
-    public boolean ping(String taskid) throws IOException {
+    public boolean ping(TaskAttemptID taskid) throws IOException {
       return true;
       return true;
     }
     }
 
 
-    public boolean statusUpdate(String taskId, TaskStatus taskStatus) 
+    public boolean statusUpdate(TaskAttemptID taskId, TaskStatus taskStatus) 
     throws IOException, InterruptedException {
     throws IOException, InterruptedException {
       StringBuffer buf = new StringBuffer("Task ");
       StringBuffer buf = new StringBuffer("Task ");
       buf.append(taskId);
       buf.append(taskId);
@@ -81,11 +81,11 @@ public class IsolationRunner {
       return true;
       return true;
     }
     }
 
 
-    public void reportDiagnosticInfo(String taskid, String trace) throws IOException {
+    public void reportDiagnosticInfo(TaskAttemptID taskid, String trace) throws IOException {
       LOG.info("Task " + taskid + " has problem " + trace);
       LOG.info("Task " + taskid + " has problem " + trace);
     }
     }
     
     
-    public TaskCompletionEvent[] getMapCompletionEvents(String jobId, 
+    public TaskCompletionEvent[] getMapCompletionEvents(JobID jobId, 
                                                         int fromEventId, int maxLocs) throws IOException {
                                                         int fromEventId, int maxLocs) throws IOException {
       return TaskCompletionEvent.EMPTY_ARRAY;
       return TaskCompletionEvent.EMPTY_ARRAY;
     }
     }
@@ -116,14 +116,13 @@ public class IsolationRunner {
    * @param conf the jobconf
    * @param conf the jobconf
    * @throws IOException if something goes wrong writing
    * @throws IOException if something goes wrong writing
    */
    */
-  private static void fillInMissingMapOutputs(FileSystem fs,
-                                              String jobId,
-                                              String taskId,
+  private static void fillInMissingMapOutputs(FileSystem fs, 
+                                              TaskAttemptID taskId,
                                               int numMaps,
                                               int numMaps,
                                               JobConf conf) throws IOException {
                                               JobConf conf) throws IOException {
     Class keyClass = conf.getMapOutputKeyClass();
     Class keyClass = conf.getMapOutputKeyClass();
     Class valueClass = conf.getMapOutputValueClass();
     Class valueClass = conf.getMapOutputValueClass();
-    MapOutputFile namer = new MapOutputFile(jobId);
+    MapOutputFile namer = new MapOutputFile(taskId.getJobID());
     namer.setConf(conf);
     namer.setConf(conf);
     for(int i=0; i<numMaps; i++) {
     for(int i=0; i<numMaps; i++) {
       Path f = namer.getInputFile(i, taskId);
       Path f = namer.getInputFile(i, taskId);
@@ -151,9 +150,8 @@ public class IsolationRunner {
       System.exit(1);
       System.exit(1);
     }
     }
     JobConf conf = new JobConf(new Path(jobFilename.toString()));
     JobConf conf = new JobConf(new Path(jobFilename.toString()));
-    String taskId = conf.get("mapred.task.id");
+    TaskAttemptID taskId = TaskAttemptID.forName(conf.get("mapred.task.id"));
     boolean isMap = conf.getBoolean("mapred.task.is.map", true);
     boolean isMap = conf.getBoolean("mapred.task.is.map", true);
-    String jobId = conf.get("mapred.job.id");
     int partition = conf.getInt("mapred.task.partition", 0);
     int partition = conf.getInt("mapred.task.partition", 0);
     
     
     // setup the local and user working directories
     // setup the local and user working directories
@@ -161,7 +159,7 @@ public class IsolationRunner {
     LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
     LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
     File workDirName = new File(lDirAlloc.getLocalPathToRead(
     File workDirName = new File(lDirAlloc.getLocalPathToRead(
                                   TaskTracker.getJobCacheSubdir() 
                                   TaskTracker.getJobCacheSubdir() 
-                                  + Path.SEPARATOR + jobId 
+                                  + Path.SEPARATOR + taskId.getJobID() 
                                   + Path.SEPARATOR + taskId
                                   + Path.SEPARATOR + taskId
                                   + Path.SEPARATOR + "work",
                                   + Path.SEPARATOR + "work",
                                   conf). toString());
                                   conf). toString());
@@ -182,13 +180,11 @@ public class IsolationRunner {
       BytesWritable split = new BytesWritable();
       BytesWritable split = new BytesWritable();
       split.readFields(splitFile);
       split.readFields(splitFile);
       splitFile.close();
       splitFile.close();
-      task = new MapTask(jobId, jobFilename.toString(), conf.get("mapred.tip.id"), 
-                         taskId, partition, splitClass, split);
+      task = new MapTask(jobFilename.toString(), taskId, partition, splitClass, split);
     } else {
     } else {
       int numMaps = conf.getNumMapTasks();
       int numMaps = conf.getNumMapTasks();
-      fillInMissingMapOutputs(local, jobId, taskId, numMaps, conf);
-      task = new ReduceTask(jobId, jobFilename.toString(), conf.get("mapred.tip.id"), taskId, 
-                            partition, numMaps);
+      fillInMissingMapOutputs(local, taskId, numMaps, conf);
+      task = new ReduceTask(jobFilename.toString(), taskId, partition, numMaps);
     }
     }
     task.setConf(conf);
     task.setConf(conf);
     task.run(conf, new FakeUmbilical());
     task.run(conf, new FakeUmbilical());

+ 65 - 37
src/java/org/apache/hadoop/mapred/JobClient.java

@@ -64,8 +64,6 @@ import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.mapred.TaskInProgress;
-import org.apache.hadoop.mapred.DefaultJobHistoryParser.*;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -181,7 +179,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
      */
      */
     public NetworkedJob(JobStatus job) throws IOException {
     public NetworkedJob(JobStatus job) throws IOException {
       this.status = job;
       this.status = job;
-      this.profile = jobSubmitClient.getJobProfile(job.getJobId());
+      this.profile = jobSubmitClient.getJobProfile(job.getJobID());
       this.statustime = System.currentTimeMillis();
       this.statustime = System.currentTimeMillis();
     }
     }
 
 
@@ -191,7 +189,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
      */
      */
     synchronized void ensureFreshStatus() throws IOException {
     synchronized void ensureFreshStatus() throws IOException {
       if (System.currentTimeMillis() - statustime > MAX_JOBPROFILE_AGE) {
       if (System.currentTimeMillis() - statustime > MAX_JOBPROFILE_AGE) {
-        this.status = jobSubmitClient.getJobStatus(profile.getJobId());
+        this.status = jobSubmitClient.getJobStatus(profile.getJobID());
         this.statustime = System.currentTimeMillis();
         this.statustime = System.currentTimeMillis();
       }
       }
     }
     }
@@ -199,8 +197,15 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     /**
     /**
      * An identifier for the job
      * An identifier for the job
      */
      */
+    public JobID getID() {
+      return profile.getJobID();
+    }
+    
+    /** @deprecated This method is deprecated and will be removed. Applications should 
+     * rather use {@link #getID()}.*/
+    @Deprecated
     public String getJobID() {
     public String getJobID() {
-      return profile.getJobId();
+      return profile.getJobID().toString();
     }
     }
     
     
     /**
     /**
@@ -275,7 +280,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
      * Tells the service to terminate the current job.
      * Tells the service to terminate the current job.
      */
      */
     public synchronized void killJob() throws IOException {
     public synchronized void killJob() throws IOException {
-      jobSubmitClient.killJob(getJobID());
+      jobSubmitClient.killJob(getID());
     }
     }
     
     
     /**
     /**
@@ -284,17 +289,23 @@ public class JobClient extends Configured implements MRConstants, Tool  {
      * @param shouldFail if true the task is failed and added to failed tasks list, otherwise
      * @param shouldFail if true the task is failed and added to failed tasks list, otherwise
      * it is just killed, w/o affecting job failure status.
      * it is just killed, w/o affecting job failure status.
      */
      */
-    public synchronized void killTask(String taskId, boolean shouldFail) throws IOException {
+    public synchronized void killTask(TaskAttemptID taskId, boolean shouldFail) throws IOException {
       jobSubmitClient.killTask(taskId, shouldFail);
       jobSubmitClient.killTask(taskId, shouldFail);
     }
     }
 
 
+    /** @deprecated Applications should rather use {@link #killTask(TaskAttemptID, boolean)}*/
+    @Deprecated
+    public synchronized void killTask(String taskId, boolean shouldFail) throws IOException {
+      killTask(TaskAttemptID.forName(taskId), shouldFail);
+    }
+    
     /**
     /**
      * Fetch task completion events from jobtracker for this job. 
      * Fetch task completion events from jobtracker for this job. 
      */
      */
     public synchronized TaskCompletionEvent[] getTaskCompletionEvents(
     public synchronized TaskCompletionEvent[] getTaskCompletionEvents(
                                                                       int startFrom) throws IOException{
                                                                       int startFrom) throws IOException{
       return jobSubmitClient.getTaskCompletionEvents(
       return jobSubmitClient.getTaskCompletionEvents(
-                                                     getJobID(), startFrom, 10); 
+                                                     getID(), startFrom, 10); 
     }
     }
 
 
     /**
     /**
@@ -306,7 +317,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
         ensureFreshStatus();
         ensureFreshStatus();
       } catch (IOException e) {
       } catch (IOException e) {
       }
       }
-      return "Job: " + profile.getJobId() + "\n" + 
+      return "Job: " + profile.getJobID() + "\n" + 
         "file: " + profile.getJobFile() + "\n" + 
         "file: " + profile.getJobFile() + "\n" + 
         "tracking URL: " + profile.getURL() + "\n" + 
         "tracking URL: " + profile.getURL() + "\n" + 
         "map() completion: " + status.mapProgress() + "\n" + 
         "map() completion: " + status.mapProgress() + "\n" + 
@@ -317,7 +328,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
      * Returns the counters for this job
      * Returns the counters for this job
      */
      */
     public Counters getCounters() throws IOException {
     public Counters getCounters() throws IOException {
-      return jobSubmitClient.getJobCounters(getJobID());
+      return jobSubmitClient.getJobCounters(getID());
     }
     }
   }
   }
 
 
@@ -693,8 +704,8 @@ public class JobClient extends Configured implements MRConstants, Tool  {
      * configure the command line options correctly on the submitting dfs
      * configure the command line options correctly on the submitting dfs
      */
      */
     
     
-    String jobId = jobSubmitClient.getNewJobId();
-    Path submitJobDir = new Path(job.getSystemDir(), jobId);
+    JobID jobId = jobSubmitClient.getNewJobId();
+    Path submitJobDir = new Path(job.getSystemDir(), jobId.toString());
     Path submitJarFile = new Path(submitJobDir, "job.jar");
     Path submitJarFile = new Path(submitJobDir, "job.jar");
     Path submitSplitFile = new Path(submitJobDir, "job.split");
     Path submitSplitFile = new Path(submitJobDir, "job.split");
     configureCommandLineOptions(job, submitJobDir, submitJarFile);
     configureCommandLineOptions(job, submitJobDir, submitJarFile);
@@ -874,7 +885,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
    *         <code>jobid</code> doesn't correspond to any known job.
    *         <code>jobid</code> doesn't correspond to any known job.
    * @throws IOException
    * @throws IOException
    */
    */
-  public RunningJob getJob(String jobid) throws IOException {
+  public RunningJob getJob(JobID jobid) throws IOException {
     JobStatus status = jobSubmitClient.getJobStatus(jobid);
     JobStatus status = jobSubmitClient.getJobStatus(jobid);
     if (status != null) {
     if (status != null) {
       return new NetworkedJob(status);
       return new NetworkedJob(status);
@@ -883,6 +894,13 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     }
     }
   }
   }
 
 
+  /**@deprecated Applications should rather use {@link #getJob(JobID)}. 
+   */
+  @Deprecated
+  public RunningJob getJob(String jobid) throws IOException {
+    return getJob(JobID.forName(jobid));
+  }
+  
   /**
   /**
    * Get the information of the current state of the map tasks of a job.
    * Get the information of the current state of the map tasks of a job.
    * 
    * 
@@ -890,10 +908,16 @@ public class JobClient extends Configured implements MRConstants, Tool  {
    * @return the list of all of the map tips.
    * @return the list of all of the map tips.
    * @throws IOException
    * @throws IOException
    */
    */
-  public TaskReport[] getMapTaskReports(String jobId) throws IOException {
+  public TaskReport[] getMapTaskReports(JobID jobId) throws IOException {
     return jobSubmitClient.getMapTaskReports(jobId);
     return jobSubmitClient.getMapTaskReports(jobId);
   }
   }
-    
+  
+  /**@deprecated Applications should rather use {@link #getMapTaskReports(JobID)}*/
+  @Deprecated
+  public TaskReport[] getMapTaskReports(String jobId) throws IOException {
+    return getMapTaskReports(JobID.forName(jobId));
+  }
+  
   /**
   /**
    * Get the information of the current state of the reduce tasks of a job.
    * Get the information of the current state of the reduce tasks of a job.
    * 
    * 
@@ -901,10 +925,16 @@ public class JobClient extends Configured implements MRConstants, Tool  {
    * @return the list of all of the reduce tips.
    * @return the list of all of the reduce tips.
    * @throws IOException
    * @throws IOException
    */    
    */    
-  public TaskReport[] getReduceTaskReports(String jobId) throws IOException {
+  public TaskReport[] getReduceTaskReports(JobID jobId) throws IOException {
     return jobSubmitClient.getReduceTaskReports(jobId);
     return jobSubmitClient.getReduceTaskReports(jobId);
   }
   }
    
    
+  /**@deprecated Applications should rather use {@link #getReduceTaskReports(JobID)}*/
+  @Deprecated
+  public TaskReport[] getReduceTaskReports(String jobId) throws IOException {
+    return getReduceTaskReports(JobID.forName(jobId));
+  }
+  
   /**
   /**
    * Get status information about the Map-Reduce cluster.
    * Get status information about the Map-Reduce cluster.
    *  
    *  
@@ -933,7 +963,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
                                        "&plaintext=true&filter=profile"
                                        "&plaintext=true&filter=profile"
                                        ).openConnection();
                                        ).openConnection();
     InputStream in = connection.getInputStream();
     InputStream in = connection.getInputStream();
-    OutputStream out = new FileOutputStream(e.getTaskId() + ".profile");
+    OutputStream out = new FileOutputStream(e.getTaskID() + ".profile");
     IOUtils.copyBytes(in, out, 64 * 1024, true);
     IOUtils.copyBytes(in, out, 64 * 1024, true);
   }
   }
 
 
@@ -971,7 +1001,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     }
     }
     try {
     try {
       running = jc.submitJob(job);
       running = jc.submitJob(job);
-      String jobId = running.getJobID();
+      JobID jobId = running.getID();
       LOG.info("Running job: " + jobId);
       LOG.info("Running job: " + jobId);
       int eventCounter = 0;
       int eventCounter = 0;
       boolean profiling = job.getProfileEnabled();
       boolean profiling = job.getProfileEnabled();
@@ -1015,7 +1045,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
               if (event.getTaskStatus() == 
               if (event.getTaskStatus() == 
                 TaskCompletionEvent.Status.SUCCEEDED){
                 TaskCompletionEvent.Status.SUCCEEDED){
                 LOG.info(event.toString());
                 LOG.info(event.toString());
-                displayTaskLogs(event.getTaskId(), event.getTaskTrackerHttp());
+                displayTaskLogs(event.getTaskID(), event.getTaskTrackerHttp());
               }
               }
               break; 
               break; 
             case FAILED:
             case FAILED:
@@ -1023,18 +1053,16 @@ public class JobClient extends Configured implements MRConstants, Tool  {
                 TaskCompletionEvent.Status.FAILED){
                 TaskCompletionEvent.Status.FAILED){
                 LOG.info(event.toString());
                 LOG.info(event.toString());
                 // Displaying the task diagnostic information
                 // Displaying the task diagnostic information
-                String taskId = event.getTaskId();
-                String tipId = TaskInProgress.getTipId(taskId);
+                TaskAttemptID taskId = event.getTaskID();
                 String[] taskDiagnostics = 
                 String[] taskDiagnostics = 
-                  jc.jobSubmitClient.getTaskDiagnostics(jobId, tipId, 
-                                                        taskId); 
+                  jc.jobSubmitClient.getTaskDiagnostics(taskId); 
                 if (taskDiagnostics != null) {
                 if (taskDiagnostics != null) {
                   for(String diagnostics : taskDiagnostics){
                   for(String diagnostics : taskDiagnostics){
                     System.err.println(diagnostics);
                     System.err.println(diagnostics);
                   }
                   }
                 }
                 }
                 // Displaying the task logs
                 // Displaying the task logs
-                displayTaskLogs(event.getTaskId(), event.getTaskTrackerHttp());
+                displayTaskLogs(event.getTaskID(), event.getTaskTrackerHttp());
               }
               }
               break; 
               break; 
             case KILLED:
             case KILLED:
@@ -1044,7 +1072,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
               break; 
               break; 
             case ALL:
             case ALL:
               LOG.info(event.toString());
               LOG.info(event.toString());
-              displayTaskLogs(event.getTaskId(), event.getTaskTrackerHttp());
+              displayTaskLogs(event.getTaskID(), event.getTaskTrackerHttp());
               break;
               break;
             }
             }
           }
           }
@@ -1073,7 +1101,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     return running;
     return running;
   }
   }
 
 
-  private static void displayTaskLogs(String taskId, String baseUrl)
+  private static void displayTaskLogs(TaskAttemptID taskId, String baseUrl)
     throws IOException {
     throws IOException {
     // The tasktracker for a 'failed/killed' job might not be around...
     // The tasktracker for a 'failed/killed' job might not be around...
     if (baseUrl != null) {
     if (baseUrl != null) {
@@ -1085,7 +1113,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     }
     }
   }
   }
     
     
-  private static void getTaskLogs(String taskId, URL taskLogUrl, 
+  private static void getTaskLogs(TaskAttemptID taskId, URL taskLogUrl, 
                                   OutputStream out) {
                                   OutputStream out) {
     try {
     try {
       URLConnection connection = taskLogUrl.openConnection();
       URLConnection connection = taskLogUrl.openConnection();
@@ -1274,10 +1302,10 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     try {
     try {
       if (submitJobFile != null) {
       if (submitJobFile != null) {
         RunningJob job = submitJob(conf);
         RunningJob job = submitJob(conf);
-        System.out.println("Created job " + job.getJobID());
+        System.out.println("Created job " + job.getID());
         exitCode = 0;
         exitCode = 0;
       } else if (getStatus) {
       } else if (getStatus) {
-        RunningJob job = getJob(jobid);
+        RunningJob job = getJob(JobID.forName(jobid));
         if (job == null) {
         if (job == null) {
           System.out.println("Could not find job " + jobid);
           System.out.println("Could not find job " + jobid);
         } else {
         } else {
@@ -1287,7 +1315,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
           exitCode = 0;
           exitCode = 0;
         }
         }
       } else if (killJob) {
       } else if (killJob) {
-        RunningJob job = getJob(jobid);
+        RunningJob job = getJob(JobID.forName(jobid));
         if (job == null) {
         if (job == null) {
           System.out.println("Could not find job " + jobid);
           System.out.println("Could not find job " + jobid);
         } else {
         } else {
@@ -1299,7 +1327,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
         viewHistory(outputDir, viewAllHistory);
         viewHistory(outputDir, viewAllHistory);
         exitCode = 0;
         exitCode = 0;
       } else if (listEvents) {
       } else if (listEvents) {
-        listEvents(jobid, fromEvent, nEvents);
+        listEvents(JobID.forName(jobid), fromEvent, nEvents);
         exitCode = 0;
         exitCode = 0;
       } else if (listJobs) {
       } else if (listJobs) {
         listJobs();
         listJobs();
@@ -1308,7 +1336,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
           listAllJobs();
           listAllJobs();
           exitCode = 0;
           exitCode = 0;
       } else if(killTask) {
       } else if(killTask) {
-        if(jobSubmitClient.killTask(taskid, false)) {
+        if(jobSubmitClient.killTask(TaskAttemptID.forName(taskid), false)) {
           System.out.println("Killed task " + taskid);
           System.out.println("Killed task " + taskid);
           exitCode = 0;
           exitCode = 0;
         } else {
         } else {
@@ -1316,7 +1344,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
           exitCode = -1;
           exitCode = -1;
         }
         }
       } else if(failTask) {
       } else if(failTask) {
-        if(jobSubmitClient.killTask(taskid, true)) {
+        if(jobSubmitClient.killTask(TaskAttemptID.forName(taskid), true)) {
           System.out.println("Killed task " + taskid + " by failing it");
           System.out.println("Killed task " + taskid + " by failing it");
           exitCode = 0;
           exitCode = 0;
         } else {
         } else {
@@ -1342,7 +1370,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
    * @param jobId the job id for the job's events to list
    * @param jobId the job id for the job's events to list
    * @throws IOException
    * @throws IOException
    */
    */
-  private void listEvents(String jobId, int fromEventId, int numEvents)
+  private void listEvents(JobID jobId, int fromEventId, int numEvents)
     throws IOException {
     throws IOException {
     TaskCompletionEvent[] events = 
     TaskCompletionEvent[] events = 
       jobSubmitClient.getTaskCompletionEvents(jobId, fromEventId, numEvents);
       jobSubmitClient.getTaskCompletionEvents(jobId, fromEventId, numEvents);
@@ -1350,7 +1378,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     System.out.println("Number of events (from " + fromEventId + 
     System.out.println("Number of events (from " + fromEventId + 
                        ") are: " + events.length);
                        ") are: " + events.length);
     for(TaskCompletionEvent event: events) {
     for(TaskCompletionEvent event: events) {
-      System.out.println(event.getTaskStatus() + " " + event.getTaskId() + 
+      System.out.println(event.getTaskStatus() + " " + event.getTaskID() + 
                          " " + event.getTaskTrackerHttp());
                          " " + event.getTaskTrackerHttp());
     }
     }
   }
   }
@@ -1367,7 +1395,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     System.out.printf("%d jobs currently running\n", jobs.length);
     System.out.printf("%d jobs currently running\n", jobs.length);
     System.out.printf("JobId\tState\tStartTime\tUserName\n");
     System.out.printf("JobId\tState\tStartTime\tUserName\n");
     for (JobStatus job : jobs) {
     for (JobStatus job : jobs) {
-      System.out.printf("%s\t%d\t%d\t%s\n", job.getJobId(), job.getRunState(),
+      System.out.printf("%s\t%d\t%d\t%s\n", job.getJobID(), job.getRunState(),
           job.getStartTime(), job.getUsername());
           job.getStartTime(), job.getUsername());
     }
     }
   }
   }
@@ -1386,7 +1414,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
                        "\tFailed : 3\tPrep : 4\n");
                        "\tFailed : 3\tPrep : 4\n");
     System.out.printf("JobId\tState\tStartTime\tUserName\n");
     System.out.printf("JobId\tState\tStartTime\tUserName\n");
     for (JobStatus job : jobs) {
     for (JobStatus job : jobs) {
-      System.out.printf("%s\t%d\t%d\t%s\n", job.getJobId(), job.getRunState(),
+      System.out.printf("%s\t%d\t%d\t%s\n", job.getJobID(), job.getRunState(),
           job.getStartTime(), job.getUsername());
           job.getStartTime(), job.getUsername());
     }
     }
   }
   }

+ 4 - 1
src/java/org/apache/hadoop/mapred/JobEndNotifier.java

@@ -101,7 +101,7 @@ public class JobEndNotifier {
       int retryAttempts = conf.getInt("job.end.retry.attempts", 0) + 1;
       int retryAttempts = conf.getInt("job.end.retry.attempts", 0) + 1;
       long retryInterval = conf.getInt("job.end.retry.interval", 30000);
       long retryInterval = conf.getInt("job.end.retry.interval", 30000);
       if (uri.contains("$jobId")) {
       if (uri.contains("$jobId")) {
-        uri = uri.replace("$jobId", status.getJobId());
+        uri = uri.replace("$jobId", status.getJobID().toString());
       }
       }
       if (uri.contains("$jobStatus")) {
       if (uri.contains("$jobStatus")) {
         String statusStr =
         String statusStr =
@@ -214,6 +214,7 @@ public class JobEndNotifier {
       return (int)(delayTime - ((JobEndStatusInfo)d).delayTime);
       return (int)(delayTime - ((JobEndStatusInfo)d).delayTime);
     }
     }
 
 
+    @Override
     public boolean equals(Object o) {
     public boolean equals(Object o) {
       if (!(o instanceof JobEndStatusInfo)) {
       if (!(o instanceof JobEndStatusInfo)) {
         return false;
         return false;
@@ -224,10 +225,12 @@ public class JobEndNotifier {
       return false;
       return false;
     }
     }
 
 
+    @Override
     public int hashCode() {
     public int hashCode() {
       return 37 * 17 + (int) (delayTime^(delayTime>>>32));
       return 37 * 17 + (int) (delayTime^(delayTime>>>32));
     }
     }
       
       
+    @Override
     public String toString() {
     public String toString() {
       return "URL: " + uri + " remaining retries: " + retryAttempts +
       return "URL: " + uri + " remaining retries: " + retryAttempts +
         " interval: " + retryInterval;
         " interval: " + retryInterval;

+ 50 - 67
src/java/org/apache/hadoop/mapred/JobHistory.java

@@ -22,7 +22,6 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.File;
 import java.io.FileFilter;
 import java.io.FileFilter;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
-import java.io.FileReader;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
@@ -31,7 +30,6 @@ import java.net.URLDecoder;
 import java.net.URLEncoder;
 import java.net.URLEncoder;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
@@ -346,7 +344,7 @@ public class JobHistory {
      * @param jobId id of the job
      * @param jobId id of the job
      * @return the path of the job file on the local file system 
      * @return the path of the job file on the local file system 
      */
      */
-    public static String getLocalJobFilePath(String jobId){
+    public static String getLocalJobFilePath(JobID jobId){
       return System.getProperty("hadoop.log.dir") + File.separator +
       return System.getProperty("hadoop.log.dir") + File.separator +
                jobId + "_conf.xml";
                jobId + "_conf.xml";
     }
     }
@@ -430,7 +428,7 @@ public class JobHistory {
      * @param submitTime time when job tracker received the job
      * @param submitTime time when job tracker received the job
      * @throws IOException
      * @throws IOException
      */
      */
-    public static void logSubmitted(String jobId, JobConf jobConf, 
+    public static void logSubmitted(JobID jobId, JobConf jobConf, 
                                     String jobConfPath, long submitTime) 
                                     String jobConfPath, long submitTime) 
     throws IOException {
     throws IOException {
       FileSystem fs = null;
       FileSystem fs = null;
@@ -502,7 +500,7 @@ public class JobHistory {
           //add to writer as well 
           //add to writer as well 
           JobHistory.log(writers, RecordTypes.Job, 
           JobHistory.log(writers, RecordTypes.Job, 
                          new Keys[]{Keys.JOBID, Keys.JOBNAME, Keys.USER, Keys.SUBMIT_TIME, Keys.JOBCONF }, 
                          new Keys[]{Keys.JOBID, Keys.JOBNAME, Keys.USER, Keys.SUBMIT_TIME, Keys.JOBCONF }, 
-                         new String[]{jobId, jobName, user, 
+                         new String[]{jobId.toString(), jobName, user, 
                                       String.valueOf(submitTime) , jobConfPath}
                                       String.valueOf(submitTime) , jobConfPath}
                         ); 
                         ); 
              
              
@@ -585,7 +583,7 @@ public class JobHistory {
      * @param totalMaps total maps assigned by jobtracker. 
      * @param totalMaps total maps assigned by jobtracker. 
      * @param totalReduces total reduces. 
      * @param totalReduces total reduces. 
      */
      */
-    public static void logStarted(String jobId, long startTime, int totalMaps, int totalReduces){
+    public static void logStarted(JobID jobId, long startTime, int totalMaps, int totalReduces){
       if (!disableHistory){
       if (!disableHistory){
         String logFileKey =  JOBTRACKER_UNIQUE_STRING + jobId; 
         String logFileKey =  JOBTRACKER_UNIQUE_STRING + jobId; 
         ArrayList<PrintWriter> writer = openJobs.get(logFileKey); 
         ArrayList<PrintWriter> writer = openJobs.get(logFileKey); 
@@ -593,7 +591,7 @@ public class JobHistory {
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Job, 
           JobHistory.log(writer, RecordTypes.Job, 
                          new Keys[] {Keys.JOBID, Keys.LAUNCH_TIME, Keys.TOTAL_MAPS, Keys.TOTAL_REDUCES },
                          new Keys[] {Keys.JOBID, Keys.LAUNCH_TIME, Keys.TOTAL_MAPS, Keys.TOTAL_REDUCES },
-                         new String[] {jobId,  String.valueOf(startTime), String.valueOf(totalMaps), String.valueOf(totalReduces)}); 
+                         new String[] {jobId.toString(),  String.valueOf(startTime), String.valueOf(totalMaps), String.valueOf(totalReduces)}); 
         }
         }
       }
       }
     }
     }
@@ -607,7 +605,7 @@ public class JobHistory {
      * @param failedReduces no of failed reduce tasks. 
      * @param failedReduces no of failed reduce tasks. 
      * @param counters the counters from the job
      * @param counters the counters from the job
      */ 
      */ 
-    public static void logFinished(String jobId, long finishTime, 
+    public static void logFinished(JobID jobId, long finishTime, 
                                    int finishedMaps, int finishedReduces,
                                    int finishedMaps, int finishedReduces,
                                    int failedMaps, int failedReduces,
                                    int failedMaps, int failedReduces,
                                    Counters counters){
                                    Counters counters){
@@ -623,7 +621,7 @@ public class JobHistory {
                                      Keys.FINISHED_REDUCES,
                                      Keys.FINISHED_REDUCES,
                                      Keys.FAILED_MAPS, Keys.FAILED_REDUCES,
                                      Keys.FAILED_MAPS, Keys.FAILED_REDUCES,
                                      Keys.COUNTERS},
                                      Keys.COUNTERS},
-                         new String[] {jobId,  Long.toString(finishTime), 
+                         new String[] {jobId.toString(),  Long.toString(finishTime), 
                                        Values.SUCCESS.name(), 
                                        Values.SUCCESS.name(), 
                                        String.valueOf(finishedMaps), 
                                        String.valueOf(finishedMaps), 
                                        String.valueOf(finishedReduces),
                                        String.valueOf(finishedReduces),
@@ -646,7 +644,7 @@ public class JobHistory {
      * @param finishedMaps no finished map tasks. 
      * @param finishedMaps no finished map tasks. 
      * @param finishedReduces no of finished reduce tasks. 
      * @param finishedReduces no of finished reduce tasks. 
      */
      */
-    public static void logFailed(String jobid, long timestamp, int finishedMaps, int finishedReduces){
+    public static void logFailed(JobID jobid, long timestamp, int finishedMaps, int finishedReduces){
       if (!disableHistory){
       if (!disableHistory){
         String logFileKey =  JOBTRACKER_UNIQUE_STRING + jobid; 
         String logFileKey =  JOBTRACKER_UNIQUE_STRING + jobid; 
         ArrayList<PrintWriter> writer = openJobs.get(logFileKey); 
         ArrayList<PrintWriter> writer = openJobs.get(logFileKey); 
@@ -654,7 +652,7 @@ public class JobHistory {
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Job,
           JobHistory.log(writer, RecordTypes.Job,
                          new Keys[] {Keys.JOBID, Keys.FINISH_TIME, Keys.JOB_STATUS, Keys.FINISHED_MAPS, Keys.FINISHED_REDUCES },
                          new Keys[] {Keys.JOBID, Keys.FINISH_TIME, Keys.JOB_STATUS, Keys.FINISHED_MAPS, Keys.FINISHED_REDUCES },
-                         new String[] {jobid,  String.valueOf(timestamp), Values.FAILED.name(), String.valueOf(finishedMaps), 
+                         new String[] {jobid.toString(),  String.valueOf(timestamp), Values.FAILED.name(), String.valueOf(finishedMaps), 
                                        String.valueOf(finishedReduces)}); 
                                        String.valueOf(finishedReduces)}); 
           for (PrintWriter out : writer) {
           for (PrintWriter out : writer) {
             out.close();
             out.close();
@@ -674,43 +672,41 @@ public class JobHistory {
 
 
     /**
     /**
      * Log start time of task (TIP).
      * Log start time of task (TIP).
-     * @param jobId job id
      * @param taskId task id
      * @param taskId task id
      * @param taskType MAP or REDUCE
      * @param taskType MAP or REDUCE
      * @param startTime startTime of tip. 
      * @param startTime startTime of tip. 
      */
      */
-    public static void logStarted(String jobId, String taskId, String taskType, 
+    public static void logStarted(TaskID taskId, String taskType, 
                                   long startTime){
                                   long startTime){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Task, 
           JobHistory.log(writer, RecordTypes.Task, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE , Keys.START_TIME}, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE , Keys.START_TIME}, 
-                         new String[]{taskId, taskType, String.valueOf(startTime)});
+                         new String[]{taskId.toString(), taskType, String.valueOf(startTime)});
         }
         }
       }
       }
     }
     }
     /**
     /**
      * Log finish time of task. 
      * Log finish time of task. 
-     * @param jobId job id
      * @param taskId task id
      * @param taskId task id
      * @param taskType MAP or REDUCE
      * @param taskType MAP or REDUCE
      * @param finishTime finish timeof task in ms
      * @param finishTime finish timeof task in ms
      */
      */
-    public static void logFinished(String jobId, String taskId, String taskType, 
+    public static void logFinished(TaskID taskId, String taskType, 
                                    long finishTime, Counters counters){
                                    long finishTime, Counters counters){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Task, 
           JobHistory.log(writer, RecordTypes.Task, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE, 
                                     Keys.TASK_STATUS, Keys.FINISH_TIME,
                                     Keys.TASK_STATUS, Keys.FINISH_TIME,
                                     Keys.COUNTERS}, 
                                     Keys.COUNTERS}, 
-                         new String[]{ taskId, taskType, Values.SUCCESS.name(), 
+                         new String[]{ taskId.toString(), taskType, Values.SUCCESS.name(), 
                                        String.valueOf(finishTime),
                                        String.valueOf(finishTime),
                                        counters.makeCompactString()});
                                        counters.makeCompactString()});
         }
         }
@@ -718,22 +714,21 @@ public class JobHistory {
     }
     }
     /**
     /**
      * Log job failed event.
      * Log job failed event.
-     * @param jobId jobid
      * @param taskId task id
      * @param taskId task id
      * @param taskType MAP or REDUCE.
      * @param taskType MAP or REDUCE.
      * @param time timestamp when job failed detected. 
      * @param time timestamp when job failed detected. 
      * @param error error message for failure. 
      * @param error error message for failure. 
      */
      */
-    public static void logFailed(String jobId, String taskId, String taskType, long time, String error){
+    public static void logFailed(TaskID taskId, String taskType, long time, String error){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Task, 
           JobHistory.log(writer, RecordTypes.Task, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE, 
                                     Keys.TASK_STATUS, Keys.FINISH_TIME, Keys.ERROR}, 
                                     Keys.TASK_STATUS, Keys.FINISH_TIME, Keys.ERROR}, 
-                         new String[]{ taskId,  taskType, Values.FAILED.name(), String.valueOf(time) , error});
+                         new String[]{ taskId.toString(),  taskType, Values.FAILED.name(), String.valueOf(time) , error});
         }
         }
       }
       }
     }
     }
@@ -757,48 +752,44 @@ public class JobHistory {
   public static class MapAttempt extends TaskAttempt{
   public static class MapAttempt extends TaskAttempt{
     /**
     /**
      * Log start time of this map task attempt. 
      * Log start time of this map task attempt. 
-     * @param jobId job id
-     * @param taskId task id
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
      * @param startTime start time of task attempt as reported by task tracker. 
      * @param startTime start time of task attempt as reported by task tracker. 
      * @param hostName host name of the task attempt. 
      * @param hostName host name of the task attempt. 
      */
      */
-    public static void logStarted(String jobId, String taskId, String taskAttemptId, long startTime, String hostName){
+    public static void logStarted(TaskAttemptID taskAttemptId, long startTime, String hostName){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskAttemptId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.MapAttempt, 
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
                                      Keys.TASK_ATTEMPT_ID, Keys.START_TIME, 
                                      Keys.TASK_ATTEMPT_ID, Keys.START_TIME, 
                                      Keys.HOSTNAME},
                                      Keys.HOSTNAME},
-                         new String[]{Values.MAP.name(),  taskId, 
-                                      taskAttemptId, String.valueOf(startTime), hostName}); 
+                         new String[]{Values.MAP.name(),  taskAttemptId.getTaskID().toString(), 
+                                      taskAttemptId.toString(), String.valueOf(startTime), hostName}); 
         }
         }
       }
       }
     }
     }
     /**
     /**
      * Log finish time of map task attempt. 
      * Log finish time of map task attempt. 
-     * @param jobId job id
-     * @param taskId task id
      * @param taskAttemptId task attempt id 
      * @param taskAttemptId task attempt id 
      * @param finishTime finish time
      * @param finishTime finish time
      * @param hostName host name 
      * @param hostName host name 
      */
      */
-    public static void logFinished(String jobId, String taskId, 
-                                   String taskAttemptId, long finishTime, 
+    public static void logFinished(TaskAttemptID taskAttemptId, long finishTime, 
                                    String hostName){
                                    String hostName){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskAttemptId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.MapAttempt, 
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
                                      Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                      Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                      Keys.FINISH_TIME, Keys.HOSTNAME},
                                      Keys.FINISH_TIME, Keys.HOSTNAME},
-                         new String[]{Values.MAP.name(), taskId, taskAttemptId, Values.SUCCESS.name(),  
+                         new String[]{Values.MAP.name(), taskAttemptId.getTaskID().toString(),
+                                      taskAttemptId.toString(), Values.SUCCESS.name(),  
                                       String.valueOf(finishTime), hostName}); 
                                       String.valueOf(finishTime), hostName}); 
         }
         }
       }
       }
@@ -806,48 +797,46 @@ public class JobHistory {
 
 
     /**
     /**
      * Log task attempt failed event.  
      * Log task attempt failed event.  
-     * @param jobId jobid
-     * @param taskId taskid
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
      * @param timestamp timestamp
      * @param timestamp timestamp
      * @param hostName hostname of this task attempt.
      * @param hostName hostname of this task attempt.
      * @param error error message if any for this task attempt. 
      * @param error error message if any for this task attempt. 
      */
      */
-    public static void logFailed(String jobId, String taskId, String taskAttemptId, 
+    public static void logFailed(TaskAttemptID taskAttemptId, 
                                  long timestamp, String hostName, String error){
                                  long timestamp, String hostName, String error){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskAttemptId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.MapAttempt, 
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                          new Keys[]{Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                     Keys.FINISH_TIME, Keys.HOSTNAME, Keys.ERROR},
                                     Keys.FINISH_TIME, Keys.HOSTNAME, Keys.ERROR},
-                         new String[]{ Values.MAP.name(), taskId, taskAttemptId, Values.FAILED.name(),
+                         new String[]{ Values.MAP.name(), taskAttemptId.getTaskID().toString(),
+                                       taskAttemptId.toString(), Values.FAILED.name(),
                                        String.valueOf(timestamp), hostName, error}); 
                                        String.valueOf(timestamp), hostName, error}); 
         }
         }
       }
       }
     }
     }
     /**
     /**
      * Log task attempt killed event.  
      * Log task attempt killed event.  
-     * @param jobId jobid
-     * @param taskId taskid
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
      * @param timestamp timestamp
      * @param timestamp timestamp
      * @param hostName hostname of this task attempt.
      * @param hostName hostname of this task attempt.
      * @param error error message if any for this task attempt. 
      * @param error error message if any for this task attempt. 
      */
      */
-    public static void logKilled(String jobId, String taskId, String taskAttemptId, 
+    public static void logKilled(TaskAttemptID taskAttemptId, 
                                  long timestamp, String hostName, String error){
                                  long timestamp, String hostName, String error){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskAttemptId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.MapAttempt, 
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                          new Keys[]{Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                     Keys.FINISH_TIME, Keys.HOSTNAME, Keys.ERROR},
                                     Keys.FINISH_TIME, Keys.HOSTNAME, Keys.ERROR},
-                         new String[]{ Values.MAP.name(), taskId, taskAttemptId, Values.KILLED.name(),
+                         new String[]{ Values.MAP.name(), taskAttemptId.getTaskID().toString(), 
+                                       taskAttemptId.toString(), Values.KILLED.name(),
                                        String.valueOf(timestamp), hostName, error}); 
                                        String.valueOf(timestamp), hostName, error}); 
         }
         }
       }
       }
@@ -860,44 +849,39 @@ public class JobHistory {
   public static class ReduceAttempt extends TaskAttempt{
   public static class ReduceAttempt extends TaskAttempt{
     /**
     /**
      * Log start time of  Reduce task attempt. 
      * Log start time of  Reduce task attempt. 
-     * @param jobId job id
-     * @param taskId task id (tip)
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
      * @param startTime start time
      * @param startTime start time
      * @param hostName host name 
      * @param hostName host name 
      */
      */
-    public static void logStarted(String jobId, String taskId, String taskAttemptId, 
+    public static void logStarted(TaskAttemptID taskAttemptId, 
                                   long startTime, String hostName){
                                   long startTime, String hostName){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskAttemptId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
                          new Keys[]{  Keys.TASK_TYPE, Keys.TASKID, 
                          new Keys[]{  Keys.TASK_TYPE, Keys.TASKID, 
                                       Keys.TASK_ATTEMPT_ID, Keys.START_TIME, Keys.HOSTNAME},
                                       Keys.TASK_ATTEMPT_ID, Keys.START_TIME, Keys.HOSTNAME},
-                         new String[]{Values.REDUCE.name(),  taskId, 
-                                      taskAttemptId, String.valueOf(startTime), hostName}); 
+                         new String[]{Values.REDUCE.name(),  taskAttemptId.getTaskID().toString(), 
+                                      taskAttemptId.toString(), String.valueOf(startTime), hostName}); 
         }
         }
       }
       }
     }
     }
     /**
     /**
      * Log finished event of this task. 
      * Log finished event of this task. 
-     * @param jobId job id
-     * @param taskId task id
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
      * @param shuffleFinished shuffle finish time
      * @param shuffleFinished shuffle finish time
      * @param sortFinished sort finish time
      * @param sortFinished sort finish time
      * @param finishTime finish time of task
      * @param finishTime finish time of task
      * @param hostName host name where task attempt executed
      * @param hostName host name where task attempt executed
      */
      */
-    public static void logFinished(String jobId, String taskId, 
-                                   String taskAttemptId, long shuffleFinished, 
+    public static void logFinished(TaskAttemptID taskAttemptId, long shuffleFinished, 
                                    long sortFinished, long finishTime, 
                                    long sortFinished, long finishTime, 
                                    String hostName){
                                    String hostName){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskAttemptId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
@@ -905,7 +889,8 @@ public class JobHistory {
                                      Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                      Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                      Keys.SHUFFLE_FINISHED, Keys.SORT_FINISHED,
                                      Keys.SHUFFLE_FINISHED, Keys.SORT_FINISHED,
                                      Keys.FINISH_TIME, Keys.HOSTNAME},
                                      Keys.FINISH_TIME, Keys.HOSTNAME},
-                         new String[]{Values.REDUCE.name(),  taskId, taskAttemptId, Values.SUCCESS.name(), 
+                         new String[]{Values.REDUCE.name(),  taskAttemptId.getTaskID().toString(), 
+                                      taskAttemptId.toString(), Values.SUCCESS.name(), 
                                       String.valueOf(shuffleFinished), String.valueOf(sortFinished),
                                       String.valueOf(shuffleFinished), String.valueOf(sortFinished),
                                       String.valueOf(finishTime), hostName}); 
                                       String.valueOf(finishTime), hostName}); 
         }
         }
@@ -913,42 +898,39 @@ public class JobHistory {
     }
     }
     /**
     /**
      * Log failed reduce task attempt. 
      * Log failed reduce task attempt. 
-     * @param jobId job id 
-     * @param taskId task id
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
      * @param timestamp time stamp when task failed
      * @param timestamp time stamp when task failed
      * @param hostName host name of the task attempt.  
      * @param hostName host name of the task attempt.  
      * @param error error message of the task. 
      * @param error error message of the task. 
      */
      */
-    public static void logFailed(String jobId, String taskId, String taskAttemptId, long timestamp, 
+    public static void logFailed(TaskAttemptID taskAttemptId, long timestamp, 
                                  String hostName, String error){
                                  String hostName, String error){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskAttemptId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
                          new Keys[]{  Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                          new Keys[]{  Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                       Keys.FINISH_TIME, Keys.HOSTNAME, Keys.ERROR },
                                       Keys.FINISH_TIME, Keys.HOSTNAME, Keys.ERROR },
-                         new String[]{ Values.REDUCE.name(), taskId, taskAttemptId, Values.FAILED.name(), 
+                         new String[]{ Values.REDUCE.name(), taskAttemptId.getTaskID().toString(), 
+                                       taskAttemptId.toString(), Values.FAILED.name(), 
                                        String.valueOf(timestamp), hostName, error }); 
                                        String.valueOf(timestamp), hostName, error }); 
         }
         }
       }
       }
     }
     }
     /**
     /**
      * Log killed reduce task attempt. 
      * Log killed reduce task attempt. 
-     * @param jobId job id 
-     * @param taskId task id
      * @param taskAttemptId task attempt id
      * @param taskAttemptId task attempt id
      * @param timestamp time stamp when task failed
      * @param timestamp time stamp when task failed
      * @param hostName host name of the task attempt.  
      * @param hostName host name of the task attempt.  
      * @param error error message of the task. 
      * @param error error message of the task. 
      */
      */
-    public static void logKilled(String jobId, String taskId, String taskAttemptId, long timestamp, 
+    public static void logKilled(TaskAttemptID taskAttemptId, long timestamp, 
                                  String hostName, String error){
                                  String hostName, String error){
       if (!disableHistory){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
-                                                     + jobId); 
+                                                     + taskAttemptId.getJobID()); 
 
 
         if (null != writer){
         if (null != writer){
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
@@ -956,7 +938,8 @@ public class JobHistory {
                                       Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                       Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                       Keys.FINISH_TIME, Keys.HOSTNAME, 
                                       Keys.FINISH_TIME, Keys.HOSTNAME, 
                                       Keys.ERROR },
                                       Keys.ERROR },
-                         new String[]{ Values.REDUCE.name(), taskId, taskAttemptId, Values.KILLED.name(), 
+                         new String[]{ Values.REDUCE.name(), taskAttemptId.getTaskID().toString(), 
+                                       taskAttemptId.toString(), Values.KILLED.name(), 
                                        String.valueOf(timestamp), hostName, error }); 
                                        String.valueOf(timestamp), hostName, error }); 
         }
         }
       }
       }

+ 56 - 71
src/java/org/apache/hadoop/mapred/JobInProgress.java

@@ -23,9 +23,9 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.IdentityHashMap;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.Iterator;
-import java.util.List;
 import java.util.LinkedHashSet;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Set;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeMap;
@@ -42,7 +42,8 @@ import org.apache.hadoop.mapred.JobTracker.JobTrackerMetrics;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.net.*;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.Node;
 
 
 /*************************************************************
 /*************************************************************
  * JobInProgress maintains all the info for keeping
  * JobInProgress maintains all the info for keeping
@@ -126,7 +127,7 @@ class JobInProgress {
   boolean tasksInited = false;
   boolean tasksInited = false;
 
 
   private LocalFileSystem localFs;
   private LocalFileSystem localFs;
-  private String jobId;
+  private JobID jobId;
   private boolean hasSpeculativeMaps;
   private boolean hasSpeculativeMaps;
   private boolean hasSpeculativeReduces;
   private boolean hasSpeculativeReduces;
 
 
@@ -148,14 +149,14 @@ class JobInProgress {
   private static final int MAX_FETCH_FAILURES_NOTIFICATIONS = 3;
   private static final int MAX_FETCH_FAILURES_NOTIFICATIONS = 3;
   
   
   // Map of mapTaskId -> no. of fetch failures
   // Map of mapTaskId -> no. of fetch failures
-  private Map<String, Integer> mapTaskIdToFetchFailuresMap =
-    new TreeMap<String, Integer>();
+  private Map<TaskAttemptID, Integer> mapTaskIdToFetchFailuresMap =
+    new TreeMap<TaskAttemptID, Integer>();
   
   
   /**
   /**
    * Create a JobInProgress with the given job file, plus a handle
    * Create a JobInProgress with the given job file, plus a handle
    * to the tracker.
    * to the tracker.
    */
    */
-  public JobInProgress(String jobid, JobTracker jobtracker, 
+  public JobInProgress(JobID jobid, JobTracker jobtracker, 
                        JobConf default_conf) throws IOException {
                        JobConf default_conf) throws IOException {
     this.jobId = jobid;
     this.jobId = jobid;
     String url = "http://" + jobtracker.getJobTrackerMachine() + ":" 
     String url = "http://" + jobtracker.getJobTrackerMachine() + ":" 
@@ -199,7 +200,7 @@ class JobInProgress {
     this.jobMetrics.setTag("user", conf.getUser());
     this.jobMetrics.setTag("user", conf.getUser());
     this.jobMetrics.setTag("sessionId", conf.getSessionId());
     this.jobMetrics.setTag("sessionId", conf.getSessionId());
     this.jobMetrics.setTag("jobName", conf.getJobName());
     this.jobMetrics.setTag("jobName", conf.getJobName());
-    this.jobMetrics.setTag("jobId", jobid);
+    this.jobMetrics.setTag("jobId", jobid.toString());
     hasSpeculativeMaps = conf.getMapSpeculativeExecution();
     hasSpeculativeMaps = conf.getMapSpeculativeExecution();
     hasSpeculativeReduces = conf.getReduceSpeculativeExecution();
     hasSpeculativeReduces = conf.getReduceSpeculativeExecution();
     this.maxLevel = jobtracker.getNumTaskCacheLevels();
     this.maxLevel = jobtracker.getNumTaskCacheLevels();
@@ -218,10 +219,9 @@ class JobInProgress {
     Counters counters = getCounters();
     Counters counters = getCounters();
     for (Counters.Group group : counters) {
     for (Counters.Group group : counters) {
       jobMetrics.setTag("group", group.getDisplayName());
       jobMetrics.setTag("group", group.getDisplayName());
-          
       for (Counters.Counter counter : group) {
       for (Counters.Counter counter : group) {
         jobMetrics.setTag("counter", counter.getDisplayName());
         jobMetrics.setTag("counter", counter.getDisplayName());
-        jobMetrics.setMetric("value", (float) counter.getCounter());
+        jobMetrics.setMetric("value", counter.getCounter());
         jobMetrics.update();
         jobMetrics.update();
       }
       }
     }
     }
@@ -329,9 +329,9 @@ class JobInProgress {
       status.setReduceProgress(1.0f);
       status.setReduceProgress(1.0f);
       status.setRunState(JobStatus.SUCCEEDED);
       status.setRunState(JobStatus.SUCCEEDED);
       tasksInited = true;
       tasksInited = true;
-      JobHistory.JobInfo.logStarted(profile.getJobId(), 
+      JobHistory.JobInfo.logStarted(profile.getJobID(), 
                                     System.currentTimeMillis(), 0, 0);
                                     System.currentTimeMillis(), 0, 0);
-      JobHistory.JobInfo.logFinished(profile.getJobId(), 
+      JobHistory.JobInfo.logFinished(profile.getJobID(), 
                                      System.currentTimeMillis(), 0, 0, 0, 0,
                                      System.currentTimeMillis(), 0, 0, 0, 0,
                                      getCounters());
                                      getCounters());
       // Special case because the Job is not queued
       // Special case because the Job is not queued
@@ -361,10 +361,10 @@ class JobInProgress {
       }
       }
     }
     }
 
 
-    this.status = new JobStatus(status.getJobId(), 0.0f, 0.0f, JobStatus.RUNNING);
+    this.status = new JobStatus(status.getJobID(), 0.0f, 0.0f, JobStatus.RUNNING);
     tasksInited = true;
     tasksInited = true;
         
         
-    JobHistory.JobInfo.logStarted(profile.getJobId(), System.currentTimeMillis(), numMapTasks, numReduceTasks);
+    JobHistory.JobInfo.logStarted(profile.getJobID(), System.currentTimeMillis(), numMapTasks, numReduceTasks);
   }
   }
 
 
   /////////////////////////////////////////////////////
   /////////////////////////////////////////////////////
@@ -482,7 +482,7 @@ class JobInProgress {
 
 
       if (state == TaskStatus.State.COMMIT_PENDING) {
       if (state == TaskStatus.State.COMMIT_PENDING) {
         JobWithTaskContext j = new JobWithTaskContext(this, tip, 
         JobWithTaskContext j = new JobWithTaskContext(this, tip, 
-                                                      status.getTaskId(),
+                                                      status.getTaskID(),
                                                       metrics);
                                                       metrics);
         jobtracker.addToCommitQueue(j);
         jobtracker.addToCommitQueue(j);
       }
       }
@@ -496,14 +496,14 @@ class JobInProgress {
         }
         }
         httpTaskLogLocation = "http://" + host + ":" + 
         httpTaskLogLocation = "http://" + host + ":" + 
           ttStatus.getHttpPort() + "/tasklog?plaintext=true&taskid=" +
           ttStatus.getHttpPort() + "/tasklog?plaintext=true&taskid=" +
-          status.getTaskId();
+          status.getTaskID();
       }
       }
 
 
       TaskCompletionEvent taskEvent = null;
       TaskCompletionEvent taskEvent = null;
       if (state == TaskStatus.State.SUCCEEDED) {
       if (state == TaskStatus.State.SUCCEEDED) {
         taskEvent = new TaskCompletionEvent(
         taskEvent = new TaskCompletionEvent(
                                             taskCompletionEventTracker, 
                                             taskCompletionEventTracker, 
-                                            status.getTaskId(),
+                                            status.getTaskID(),
                                             tip.idWithinJob(),
                                             tip.idWithinJob(),
                                             status.getIsMap(),
                                             status.getIsMap(),
                                             TaskCompletionEvent.Status.SUCCEEDED,
                                             TaskCompletionEvent.Status.SUCCEEDED,
@@ -527,12 +527,12 @@ class JobInProgress {
         if ((eventNumber = tip.getSuccessEventNumber()) != -1) {
         if ((eventNumber = tip.getSuccessEventNumber()) != -1) {
           TaskCompletionEvent t = 
           TaskCompletionEvent t = 
             this.taskCompletionEvents.get(eventNumber);
             this.taskCompletionEvents.get(eventNumber);
-          if (t.getTaskId().equals(status.getTaskId()))
+          if (t.getTaskID().equals(status.getTaskID()))
             t.setTaskStatus(TaskCompletionEvent.Status.OBSOLETE);
             t.setTaskStatus(TaskCompletionEvent.Status.OBSOLETE);
         }
         }
         
         
         // Tell the job to fail the relevant task
         // Tell the job to fail the relevant task
-        failedTask(tip, status.getTaskId(), status, status.getTaskTracker(),
+        failedTask(tip, status.getTaskID(), status, status.getTaskTracker(),
                    wasRunning, wasComplete, metrics);
                    wasRunning, wasComplete, metrics);
 
 
         // Did the task failure lead to tip failure?
         // Did the task failure lead to tip failure?
@@ -544,7 +544,7 @@ class JobInProgress {
           taskCompletionStatus = TaskCompletionEvent.Status.TIPFAILED;
           taskCompletionStatus = TaskCompletionEvent.Status.TIPFAILED;
         }
         }
         taskEvent = new TaskCompletionEvent(taskCompletionEventTracker, 
         taskEvent = new TaskCompletionEvent(taskCompletionEventTracker, 
-                                            status.getTaskId(),
+                                            status.getTaskID(),
                                             tip.idWithinJob(),
                                             tip.idWithinJob(),
                                             status.getIsMap(),
                                             status.getIsMap(),
                                             taskCompletionStatus, 
                                             taskCompletionStatus, 
@@ -651,11 +651,10 @@ class JobInProgress {
                                             int clusterSize
                                             int clusterSize
                                            ) throws IOException {
                                            ) throws IOException {
     if (!tasksInited) {
     if (!tasksInited) {
-      LOG.info("Cannot create task split for " + profile.getJobId());
+      LOG.info("Cannot create task split for " + profile.getJobID());
       return null;
       return null;
     }
     }
-    
-    
+        
     int target = findNewMapTask(tts, clusterSize, status.mapProgress());
     int target = findNewMapTask(tts, clusterSize, status.mapProgress());
     if (target == -1) {
     if (target == -1) {
       return null;
       return null;
@@ -664,11 +663,9 @@ class JobInProgress {
     Task result = maps[target].getTaskToRun(tts.getTrackerName());
     Task result = maps[target].getTaskToRun(tts.getTrackerName());
     if (result != null) {
     if (result != null) {
       runningMapTasks += 1;
       runningMapTasks += 1;
-
       boolean wasRunning = maps[target].isRunning();
       boolean wasRunning = maps[target].isRunning();
       if (!wasRunning) {
       if (!wasRunning) {
-        JobHistory.Task.logStarted(profile.getJobId(), 
-                                   maps[target].getTIPId(), Values.MAP.name(),
+        JobHistory.Task.logStarted(maps[target].getTIPId(), Values.MAP.name(),
                                    System.currentTimeMillis());
                                    System.currentTimeMillis());
       }
       }
 
 
@@ -687,7 +684,7 @@ class JobInProgress {
                                                int clusterSize
                                                int clusterSize
                                               ) throws IOException {
                                               ) throws IOException {
     if (!tasksInited) {
     if (!tasksInited) {
-      LOG.info("Cannot create task split for " + profile.getJobId());
+      LOG.info("Cannot create task split for " + profile.getJobID());
       return null;
       return null;
     }
     }
 
 
@@ -699,11 +696,9 @@ class JobInProgress {
     Task result = reduces[target].getTaskToRun(tts.getTrackerName());
     Task result = reduces[target].getTaskToRun(tts.getTrackerName());
     if (result != null) {
     if (result != null) {
       runningReduceTasks += 1;
       runningReduceTasks += 1;
-
       boolean wasRunning = reduces[target].isRunning();
       boolean wasRunning = reduces[target].isRunning();
       if (!wasRunning) {
       if (!wasRunning) {
-        JobHistory.Task.logStarted(profile.getJobId(), 
-                                   reduces[target].getTIPId(), Values.REDUCE.name(),
+        JobHistory.Task.logStarted(reduces[target].getTIPId(), Values.REDUCE.name(),
                                    System.currentTimeMillis());
                                    System.currentTimeMillis());
       }
       }
 
 
@@ -1286,7 +1281,7 @@ class JobInProgress {
                                          TaskStatus status,
                                          TaskStatus status,
                                          JobTrackerMetrics metrics) 
                                          JobTrackerMetrics metrics) 
   {
   {
-    String taskid = status.getTaskId();
+    TaskAttemptID taskid = status.getTaskID();
         
         
     // Sanity check: is the TIP already complete? 
     // Sanity check: is the TIP already complete? 
     // It _is_ safe to not decrement running{Map|Reduce}Tasks and
     // It _is_ safe to not decrement running{Map|Reduce}Tasks and
@@ -1314,24 +1309,20 @@ class JobInProgress {
     // Update jobhistory 
     // Update jobhistory 
     String taskTrackerName = status.getTaskTracker();
     String taskTrackerName = status.getTaskTracker();
     if (status.getIsMap()){
     if (status.getIsMap()){
-      JobHistory.MapAttempt.logStarted(profile.getJobId(), 
-                                       tip.getTIPId(), status.getTaskId(), status.getStartTime(), 
+      JobHistory.MapAttempt.logStarted(status.getTaskID(), status.getStartTime(), 
                                        taskTrackerName); 
                                        taskTrackerName); 
-      JobHistory.MapAttempt.logFinished(profile.getJobId(), 
-                                        tip.getTIPId(), status.getTaskId(), status.getFinishTime(), 
+      JobHistory.MapAttempt.logFinished(status.getTaskID(), status.getFinishTime(), 
                                         taskTrackerName); 
                                         taskTrackerName); 
-      JobHistory.Task.logFinished(profile.getJobId(), tip.getTIPId(), 
+      JobHistory.Task.logFinished(tip.getTIPId(), 
                                   Values.MAP.name(), status.getFinishTime(),
                                   Values.MAP.name(), status.getFinishTime(),
                                   status.getCounters()); 
                                   status.getCounters()); 
     }else{
     }else{
-      JobHistory.ReduceAttempt.logStarted(profile.getJobId(), 
-                                          tip.getTIPId(), status.getTaskId(), status.getStartTime(), 
+      JobHistory.ReduceAttempt.logStarted( status.getTaskID(), status.getStartTime(), 
                                           taskTrackerName); 
                                           taskTrackerName); 
-      JobHistory.ReduceAttempt.logFinished(profile.getJobId(), 
-                                           tip.getTIPId(), status.getTaskId(), status.getShuffleFinishTime(),
+      JobHistory.ReduceAttempt.logFinished(status.getTaskID(), status.getShuffleFinishTime(),
                                            status.getSortFinishTime(), status.getFinishTime(), 
                                            status.getSortFinishTime(), status.getFinishTime(), 
                                            taskTrackerName); 
                                            taskTrackerName); 
-      JobHistory.Task.logFinished(profile.getJobId(), tip.getTIPId(), 
+      JobHistory.Task.logFinished(tip.getTIPId(), 
                                   Values.REDUCE.name(), status.getFinishTime(),
                                   Values.REDUCE.name(), status.getFinishTime(),
                                   status.getCounters()); 
                                   status.getCounters()); 
     }
     }
@@ -1394,9 +1385,9 @@ class JobInProgress {
       this.status.setReduceProgress(1.0f);
       this.status.setReduceProgress(1.0f);
       this.finishTime = System.currentTimeMillis();
       this.finishTime = System.currentTimeMillis();
       garbageCollect();
       garbageCollect();
-      LOG.info("Job " + this.status.getJobId() + 
+      LOG.info("Job " + this.status.getJobID() + 
                " has completed successfully.");
                " has completed successfully.");
-      JobHistory.JobInfo.logFinished(this.status.getJobId(), finishTime, 
+      JobHistory.JobInfo.logFinished(this.status.getJobID(), finishTime, 
                                      this.finishedMapTasks, 
                                      this.finishedMapTasks, 
                                      this.finishedReduceTasks, failedMapTasks, 
                                      this.finishedReduceTasks, failedMapTasks, 
                                      failedReduceTasks, getCounters());
                                      failedReduceTasks, getCounters());
@@ -1411,8 +1402,8 @@ class JobInProgress {
    */
    */
   public synchronized void kill() {
   public synchronized void kill() {
     if (status.getRunState() != JobStatus.FAILED) {
     if (status.getRunState() != JobStatus.FAILED) {
-      LOG.info("Killing job '" + this.status.getJobId() + "'");
-      this.status = new JobStatus(status.getJobId(), 1.0f, 1.0f, JobStatus.FAILED);
+      LOG.info("Killing job '" + this.status.getJobID() + "'");
+      this.status = new JobStatus(status.getJobID(), 1.0f, 1.0f, JobStatus.FAILED);
       this.finishTime = System.currentTimeMillis();
       this.finishTime = System.currentTimeMillis();
       this.runningMapTasks = 0;
       this.runningMapTasks = 0;
       this.runningReduceTasks = 0;
       this.runningReduceTasks = 0;
@@ -1425,7 +1416,7 @@ class JobInProgress {
       for (int i = 0; i < reduces.length; i++) {
       for (int i = 0; i < reduces.length; i++) {
         reduces[i].kill();
         reduces[i].kill();
       }
       }
-      JobHistory.JobInfo.logFailed(this.status.getJobId(), finishTime, 
+      JobHistory.JobInfo.logFailed(this.status.getJobID(), finishTime, 
                                    this.finishedMapTasks, this.finishedReduceTasks);
                                    this.finishedMapTasks, this.finishedReduceTasks);
       garbageCollect();
       garbageCollect();
     }
     }
@@ -1443,7 +1434,7 @@ class JobInProgress {
    * we need to schedule reexecution so that downstream reduce tasks can 
    * we need to schedule reexecution so that downstream reduce tasks can 
    * obtain the map task's output.
    * obtain the map task's output.
    */
    */
-  private void failedTask(TaskInProgress tip, String taskid, 
+  private void failedTask(TaskInProgress tip, TaskAttemptID taskid, 
                           TaskStatus status, String trackerName,
                           TaskStatus status, String trackerName,
                           boolean wasRunning, boolean wasComplete,
                           boolean wasRunning, boolean wasComplete,
                           JobTrackerMetrics metrics) {
                           JobTrackerMetrics metrics) {
@@ -1495,29 +1486,23 @@ class JobInProgress {
     // update job history
     // update job history
     String taskTrackerName = status.getTaskTracker();
     String taskTrackerName = status.getTaskTracker();
     if (status.getIsMap()) {
     if (status.getIsMap()) {
-      JobHistory.MapAttempt.logStarted(profile.getJobId(), 
-                tip.getTIPId(), status.getTaskId(), status.getStartTime(), 
+      JobHistory.MapAttempt.logStarted(status.getTaskID(), status.getStartTime(), 
                 taskTrackerName);
                 taskTrackerName);
       if (status.getRunState() == TaskStatus.State.FAILED) {
       if (status.getRunState() == TaskStatus.State.FAILED) {
-        JobHistory.MapAttempt.logFailed(profile.getJobId(), 
-                tip.getTIPId(), status.getTaskId(), System.currentTimeMillis(),
+        JobHistory.MapAttempt.logFailed(status.getTaskID(), System.currentTimeMillis(),
                 taskTrackerName, status.getDiagnosticInfo());
                 taskTrackerName, status.getDiagnosticInfo());
       } else {
       } else {
-        JobHistory.MapAttempt.logKilled(profile.getJobId(), 
-                tip.getTIPId(), status.getTaskId(), System.currentTimeMillis(),
+        JobHistory.MapAttempt.logKilled(status.getTaskID(), System.currentTimeMillis(),
                 taskTrackerName, status.getDiagnosticInfo());
                 taskTrackerName, status.getDiagnosticInfo());
       }
       }
     } else {
     } else {
-      JobHistory.ReduceAttempt.logStarted(profile.getJobId(), 
-                tip.getTIPId(), status.getTaskId(), status.getStartTime(), 
+      JobHistory.ReduceAttempt.logStarted(status.getTaskID(), status.getStartTime(), 
                 taskTrackerName);
                 taskTrackerName);
       if (status.getRunState() == TaskStatus.State.FAILED) {
       if (status.getRunState() == TaskStatus.State.FAILED) {
-        JobHistory.ReduceAttempt.logFailed(profile.getJobId(), 
-                tip.getTIPId(), status.getTaskId(), System.currentTimeMillis(),
+        JobHistory.ReduceAttempt.logFailed(status.getTaskID(), System.currentTimeMillis(),
                 taskTrackerName, status.getDiagnosticInfo());
                 taskTrackerName, status.getDiagnosticInfo());
       } else {
       } else {
-        JobHistory.ReduceAttempt.logKilled(profile.getJobId(), 
-                tip.getTIPId(), status.getTaskId(), System.currentTimeMillis(),
+        JobHistory.ReduceAttempt.logKilled(status.getTaskID(), System.currentTimeMillis(),
                 taskTrackerName, status.getDiagnosticInfo());
                 taskTrackerName, status.getDiagnosticInfo());
       }
       }
     }
     }
@@ -1558,14 +1543,14 @@ class JobInProgress {
             ((++failedReduceTIPs*100) > (reduceFailuresPercent*numReduceTasks));
             ((++failedReduceTIPs*100) > (reduceFailuresPercent*numReduceTasks));
       
       
       if (killJob) {
       if (killJob) {
-        LOG.info("Aborting job " + profile.getJobId());
-        JobHistory.Task.logFailed(profile.getJobId(), tip.getTIPId(), 
+        LOG.info("Aborting job " + profile.getJobID());
+        JobHistory.Task.logFailed(tip.getTIPId(), 
                                   tip.isMapTask() ? 
                                   tip.isMapTask() ? 
                                           Values.MAP.name() : 
                                           Values.MAP.name() : 
                                           Values.REDUCE.name(),  
                                           Values.REDUCE.name(),  
                                   System.currentTimeMillis(), 
                                   System.currentTimeMillis(), 
                                   status.getDiagnosticInfo());
                                   status.getDiagnosticInfo());
-        JobHistory.JobInfo.logFailed(profile.getJobId(), 
+        JobHistory.JobInfo.logFailed(profile.getJobID(), 
                                      System.currentTimeMillis(), 
                                      System.currentTimeMillis(), 
                                      this.finishedMapTasks, 
                                      this.finishedMapTasks, 
                                      this.finishedReduceTasks
                                      this.finishedReduceTasks
@@ -1593,7 +1578,7 @@ class JobInProgress {
    * @param reason The reason that the task failed
    * @param reason The reason that the task failed
    * @param trackerName The task tracker the task failed on
    * @param trackerName The task tracker the task failed on
    */
    */
-  public void failedTask(TaskInProgress tip, String taskid, String reason, 
+  public void failedTask(TaskInProgress tip, TaskAttemptID taskid, String reason, 
                          TaskStatus.Phase phase, TaskStatus.State state, 
                          TaskStatus.Phase phase, TaskStatus.State state, 
                          String trackerName, JobTrackerMetrics metrics) {
                          String trackerName, JobTrackerMetrics metrics) {
     TaskStatus status = TaskStatus.createTaskStatus(tip.isMapTask(), 
     TaskStatus status = TaskStatus.createTaskStatus(tip.isMapTask(), 
@@ -1605,7 +1590,7 @@ class JobInProgress {
                                                     trackerName, phase,
                                                     trackerName, phase,
                                                     null);
                                                     null);
     updateTaskStatus(tip, status, metrics);
     updateTaskStatus(tip, status, metrics);
-    JobHistory.Task.logFailed(profile.getJobId(), tip.getTIPId(), 
+    JobHistory.Task.logFailed(tip.getTIPId(), 
                               tip.isMapTask() ? Values.MAP.name() : Values.REDUCE.name(), 
                               tip.isMapTask() ? Values.MAP.name() : Values.REDUCE.name(), 
                               System.currentTimeMillis(), reason); 
                               System.currentTimeMillis(), reason); 
   }
   }
@@ -1638,7 +1623,7 @@ class JobInProgress {
         
         
       // Delete temp dfs dirs created if any, like in case of 
       // Delete temp dfs dirs created if any, like in case of 
       // speculative exn of reduces.  
       // speculative exn of reduces.  
-      Path tempDir = new Path(conf.getSystemDir(), jobId); 
+      Path tempDir = new Path(conf.getSystemDir(), jobId.toString()); 
       fs.delete(tempDir, true); 
       fs.delete(tempDir, true); 
 
 
       // delete the temporary directory in output directory
       // delete the temporary directory in output directory
@@ -1651,7 +1636,7 @@ class JobInProgress {
         }
         }
       }
       }
     } catch (IOException e) {
     } catch (IOException e) {
-      LOG.warn("Error cleaning up "+profile.getJobId()+": "+e);
+      LOG.warn("Error cleaning up "+profile.getJobID()+": "+e);
     }
     }
     
     
     cleanUpMetrics();
     cleanUpMetrics();
@@ -1665,8 +1650,8 @@ class JobInProgress {
   /**
   /**
    * Return the TaskInProgress that matches the tipid.
    * Return the TaskInProgress that matches the tipid.
    */
    */
-  public TaskInProgress getTaskInProgress(String tipid){
-    if (TaskInProgress.isMapId(tipid)) {
+  public TaskInProgress getTaskInProgress(TaskID tipid){
+    if (tipid.isMap()) {
       for (int i = 0; i < maps.length; i++) {
       for (int i = 0; i < maps.length; i++) {
         if (tipid.equals(maps[i].getTIPId())){
         if (tipid.equals(maps[i].getTIPId())){
           return maps[i];
           return maps[i];
@@ -1712,7 +1697,7 @@ class JobInProgress {
   }
   }
   
   
   synchronized void fetchFailureNotification(TaskInProgress tip, 
   synchronized void fetchFailureNotification(TaskInProgress tip, 
-                                             String mapTaskId, 
+                                             TaskAttemptID mapTaskId, 
                                              String trackerName, 
                                              String trackerName, 
                                              JobTrackerMetrics metrics) {
                                              JobTrackerMetrics metrics) {
     Integer fetchFailures = mapTaskIdToFetchFailuresMap.get(mapTaskId);
     Integer fetchFailures = mapTaskIdToFetchFailuresMap.get(mapTaskId);
@@ -1743,10 +1728,10 @@ class JobInProgress {
   static class JobWithTaskContext {
   static class JobWithTaskContext {
     private JobInProgress job;
     private JobInProgress job;
     private TaskInProgress tip;
     private TaskInProgress tip;
-    private String taskId;
+    private TaskAttemptID taskId;
     private JobTrackerMetrics metrics;
     private JobTrackerMetrics metrics;
     JobWithTaskContext(JobInProgress job, TaskInProgress tip, 
     JobWithTaskContext(JobInProgress job, TaskInProgress tip, 
-        String taskId, JobTrackerMetrics metrics) {
+        TaskAttemptID taskId, JobTrackerMetrics metrics) {
       this.job = job;
       this.job = job;
       this.tip = tip;
       this.tip = tip;
       this.taskId = taskId;
       this.taskId = taskId;
@@ -1758,7 +1743,7 @@ class JobInProgress {
     TaskInProgress getTIP() {
     TaskInProgress getTIP() {
       return tip;
       return tip;
     }
     }
-    String getTaskId() {
+    TaskAttemptID getTaskID() {
       return taskId;
       return taskId;
     }
     }
     JobTrackerMetrics getJobTrackerMetrics() {
     JobTrackerMetrics getJobTrackerMetrics() {

+ 21 - 16
src/java/org/apache/hadoop/mapred/JobProfile.java

@@ -17,10 +17,15 @@
  */
  */
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import org.apache.hadoop.io.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URL;
 
 
-import java.io.*;
-import java.net.*;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
 
 
 /**************************************************
 /**************************************************
  * A JobProfile is a MapReduce primitive.  Tracks a job,
  * A JobProfile is a MapReduce primitive.  Tracks a job,
@@ -38,7 +43,7 @@ public class JobProfile implements Writable {
   }
   }
 
 
   String user;
   String user;
-  String jobid;
+  JobID jobid;
   String jobFile;
   String jobFile;
   String url;
   String url;
   String name;
   String name;
@@ -59,7 +64,7 @@ public class JobProfile implements Writable {
    * @param url link to the web-ui for details of the job.
    * @param url link to the web-ui for details of the job.
    * @param name user-specified job name.
    * @param name user-specified job name.
    */
    */
-  public JobProfile(String user, String jobid, String jobFile, String url,
+  public JobProfile(String user, JobID jobid, String jobFile, String url,
                     String name) {
                     String name) {
     this.user = user;
     this.user = user;
     this.jobid = jobid;
     this.jobid = jobid;
@@ -78,7 +83,7 @@ public class JobProfile implements Writable {
   /**
   /**
    * Get the job id.
    * Get the job id.
    */
    */
-  public String getJobId() {
+  public JobID getJobID() {
     return jobid;
     return jobid;
   }
   }
 
 
@@ -111,18 +116,18 @@ public class JobProfile implements Writable {
   // Writable
   // Writable
   ///////////////////////////////////////
   ///////////////////////////////////////
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    UTF8.writeString(out, jobid);
-    UTF8.writeString(out, jobFile);
-    UTF8.writeString(out, url);
-    UTF8.writeString(out, user);
-    UTF8.writeString(out, name);
+    jobid.write(out);
+    Text.writeString(out, jobFile);
+    Text.writeString(out, url);
+    Text.writeString(out, user);
+    Text.writeString(out, name);
   }
   }
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    this.jobid = UTF8.readString(in);
-    this.jobFile = UTF8.readString(in);
-    this.url = UTF8.readString(in);
-    this.user = UTF8.readString(in);
-    this.name = UTF8.readString(in);
+    this.jobid = JobID.read(in);
+    this.jobFile = Text.readString(in);
+    this.url = Text.readString(in);
+    this.user = Text.readString(in);
+    this.name = Text.readString(in);
   }
   }
 }
 }
 
 

+ 8 - 8
src/java/org/apache/hadoop/mapred/JobStatus.java

@@ -21,7 +21,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
 import org.apache.hadoop.io.WritableFactory;
@@ -46,7 +46,7 @@ public class JobStatus implements Writable {
   public static final int FAILED = 3;
   public static final int FAILED = 3;
   public static final int PREP = 4;
   public static final int PREP = 4;
 
 
-  private String jobid;
+  private JobID jobid;
   private float mapProgress;
   private float mapProgress;
   private float reduceProgress;
   private float reduceProgress;
   private int runState;
   private int runState;
@@ -65,7 +65,7 @@ public class JobStatus implements Writable {
    * @param reduceProgress The progress made on the reduces
    * @param reduceProgress The progress made on the reduces
    * @param runState The current state of the job
    * @param runState The current state of the job
    */
    */
-  public JobStatus(String jobid, float mapProgress, float reduceProgress, int runState) {
+  public JobStatus(JobID jobid, float mapProgress, float reduceProgress, int runState) {
     this.jobid = jobid;
     this.jobid = jobid;
     this.mapProgress = mapProgress;
     this.mapProgress = mapProgress;
     this.reduceProgress = reduceProgress;
     this.reduceProgress = reduceProgress;
@@ -76,7 +76,7 @@ public class JobStatus implements Writable {
   /**
   /**
    * @return The jobid of the Job
    * @return The jobid of the Job
    */
    */
-  public String getJobId() { return jobid; }
+  public JobID getJobID() { return jobid; }
     
     
   /**
   /**
    * @return Percentage of progress in maps 
    * @return Percentage of progress in maps 
@@ -141,20 +141,20 @@ public class JobStatus implements Writable {
   // Writable
   // Writable
   ///////////////////////////////////////
   ///////////////////////////////////////
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    UTF8.writeString(out, jobid);
+    jobid.write(out);
     out.writeFloat(mapProgress);
     out.writeFloat(mapProgress);
     out.writeFloat(reduceProgress);
     out.writeFloat(reduceProgress);
     out.writeInt(runState);
     out.writeInt(runState);
     out.writeLong(startTime);
     out.writeLong(startTime);
-    UTF8.writeString(out, user);
+    Text.writeString(out, user);
   }
   }
 
 
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    this.jobid = UTF8.readString(in);
+    this.jobid = JobID.read(in);
     this.mapProgress = in.readFloat();
     this.mapProgress = in.readFloat();
     this.reduceProgress = in.readFloat();
     this.reduceProgress = in.readFloat();
     this.runState = in.readInt();
     this.runState = in.readInt();
     this.startTime = in.readLong();
     this.startTime = in.readLong();
-    this.user = UTF8.readString(in);
+    this.user = Text.readString(in);
   }
   }
 }
 }

+ 16 - 15
src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.ipc.VersionedProtocol;
  * the current system status.
  * the current system status.
  */ 
  */ 
 public interface JobSubmissionProtocol extends VersionedProtocol {
 public interface JobSubmissionProtocol extends VersionedProtocol {
-  /*
+  /* 
    *Changing the versionID to 2L since the getTaskCompletionEvents method has
    *Changing the versionID to 2L since the getTaskCompletionEvents method has
    *changed.
    *changed.
    *Changed to 4 since killTask(String,boolean) is added
    *Changed to 4 since killTask(String,boolean) is added
@@ -37,22 +37,23 @@ public interface JobSubmissionProtocol extends VersionedProtocol {
    * max_map_tasks and max_reduce_tasks for HADOOP-1274
    * max_map_tasks and max_reduce_tasks for HADOOP-1274
    * Version 6: change the counters representation for HADOOP-2248
    * Version 6: change the counters representation for HADOOP-2248
    * Version 7: added getAllJobs for HADOOP-2487
    * Version 7: added getAllJobs for HADOOP-2487
+   * Version 8: change {job|task}id's to use corresponding objects rather that strings.
    */
    */
-  public static final long versionID = 7L;
+  public static final long versionID = 8L;
 
 
   /**
   /**
    * Allocate a name for the job.
    * Allocate a name for the job.
    * @return a unique job name for submitting jobs.
    * @return a unique job name for submitting jobs.
    * @throws IOException
    * @throws IOException
    */
    */
-  public String getNewJobId() throws IOException;
+  public JobID getNewJobId() throws IOException;
 
 
   /**
   /**
    * Submit a Job for execution.  Returns the latest profile for
    * Submit a Job for execution.  Returns the latest profile for
    * that job.
    * that job.
    * The job files should be submitted in <b>system-dir</b>/<b>jobName</b>.
    * The job files should be submitted in <b>system-dir</b>/<b>jobName</b>.
    */
    */
-  public JobStatus submitJob(String jobName) throws IOException;
+  public JobStatus submitJob(JobID jobName) throws IOException;
 
 
   /**
   /**
    * Get the current status of the cluster
    * Get the current status of the cluster
@@ -63,7 +64,7 @@ public interface JobSubmissionProtocol extends VersionedProtocol {
   /**
   /**
    * Kill the indicated job
    * Kill the indicated job
    */
    */
-  public void killJob(String jobid) throws IOException;
+  public void killJob(JobID jobid) throws IOException;
 
 
   /**
   /**
    * Kill indicated task attempt.
    * Kill indicated task attempt.
@@ -71,34 +72,34 @@ public interface JobSubmissionProtocol extends VersionedProtocol {
    * @param shouldFail if true the task is failed and added to failed tasks list, otherwise
    * @param shouldFail if true the task is failed and added to failed tasks list, otherwise
    * it is just killed, w/o affecting job failure status.  
    * it is just killed, w/o affecting job failure status.  
    */ 
    */ 
-  public boolean killTask(String taskId, boolean shouldFail) throws IOException;
+  public boolean killTask(TaskAttemptID taskId, boolean shouldFail) throws IOException;
   
   
   /**
   /**
    * Grab a handle to a job that is already known to the JobTracker.
    * Grab a handle to a job that is already known to the JobTracker.
    * @return Profile of the job, or null if not found. 
    * @return Profile of the job, or null if not found. 
    */
    */
-  public JobProfile getJobProfile(String jobid) throws IOException;
+  public JobProfile getJobProfile(JobID jobid) throws IOException;
 
 
   /**
   /**
    * Grab a handle to a job that is already known to the JobTracker.
    * Grab a handle to a job that is already known to the JobTracker.
    * @return Status of the job, or null if not found.
    * @return Status of the job, or null if not found.
    */
    */
-  public JobStatus getJobStatus(String jobid) throws IOException;
+  public JobStatus getJobStatus(JobID jobid) throws IOException;
 
 
   /**
   /**
    * Grab the current job counters
    * Grab the current job counters
    */
    */
-  public Counters getJobCounters(String jobid) throws IOException;
+  public Counters getJobCounters(JobID jobid) throws IOException;
     
     
   /**
   /**
    * Grab a bunch of info on the map tasks that make up the job
    * Grab a bunch of info on the map tasks that make up the job
    */
    */
-  public TaskReport[] getMapTaskReports(String jobid) throws IOException;
+  public TaskReport[] getMapTaskReports(JobID jobid) throws IOException;
 
 
   /**
   /**
    * Grab a bunch of info on the reduce tasks that make up the job
    * Grab a bunch of info on the reduce tasks that make up the job
    */
    */
-  public TaskReport[] getReduceTaskReports(String jobid) throws IOException;
+  public TaskReport[] getReduceTaskReports(JobID jobid) throws IOException;
 
 
   /**
   /**
    * A MapReduce system always operates on a single filesystem.  This 
    * A MapReduce system always operates on a single filesystem.  This 
@@ -130,15 +131,15 @@ public interface JobSubmissionProtocol extends VersionedProtocol {
    * @return array of task completion events. 
    * @return array of task completion events. 
    * @throws IOException
    * @throws IOException
    */
    */
-  public TaskCompletionEvent[] getTaskCompletionEvents(
-                                                       String jobid, int fromEventId, int maxEvents) throws IOException;
+  public TaskCompletionEvent[] getTaskCompletionEvents(JobID jobid
+      , int fromEventId, int maxEvents) throws IOException;
     
     
   /**
   /**
    * Get the diagnostics for a given task in a given job
    * Get the diagnostics for a given task in a given job
-   * @param jobId the id of the job
+   * @param taskId the id of the task
    * @return an array of the diagnostic messages
    * @return an array of the diagnostic messages
    */
    */
-  public String[] getTaskDiagnostics(String jobId, String tipId, String taskId) 
+  public String[] getTaskDiagnostics(TaskAttemptID taskId) 
   throws IOException;  
   throws IOException;  
   
   
 }
 }

+ 91 - 101
src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.net.BindException;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
-import java.text.NumberFormat;
 import java.text.SimpleDateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
@@ -37,7 +36,6 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Properties;
 import java.util.Set;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeMap;
-import java.util.HashMap;
 import java.util.TreeSet;
 import java.util.TreeSet;
 import java.util.Vector;
 import java.util.Vector;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -51,18 +49,18 @@ import org.apache.hadoop.fs.permission.AccessControlException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.RPC.VersionMismatch;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.RPC.VersionMismatch;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.Updater;
 import org.apache.hadoop.metrics.Updater;
 import org.apache.hadoop.metrics.jvm.JvmMetrics;
 import org.apache.hadoop.metrics.jvm.JvmMetrics;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.DNSToSwitchMapping;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
-import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.ScriptBasedMapping;
 import org.apache.hadoop.net.ScriptBasedMapping;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -108,15 +106,6 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    * jobs kept in memory per-user. 
    * jobs kept in memory per-user. 
    */
    */
   final int MAX_COMPLETE_USER_JOBS_IN_MEMORY;
   final int MAX_COMPLETE_USER_JOBS_IN_MEMORY;
-    
-  /**
-   * Used for formatting the id numbers
-   */
-  private static NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setMinimumIntegerDigits(4);
-    idFormat.setGroupingUsed(false);
-  }
 
 
   private int nextJobId = 1;
   private int nextJobId = 1;
 
 
@@ -183,10 +172,10 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     /**
     /**
      * This is a map of the tasks that have been assigned to task trackers,
      * This is a map of the tasks that have been assigned to task trackers,
      * but that have not yet been seen in a status report.
      * but that have not yet been seen in a status report.
-     * map: task-id (String) -> time-assigned (Long)
+     * map: task-id -> time-assigned 
      */
      */
-    private Map<String, Long> launchingTasks =
-      new LinkedHashMap<String, Long>();
+    private Map<TaskAttemptID, Long> launchingTasks =
+      new LinkedHashMap<TaskAttemptID, Long>();
       
       
     public void run() {
     public void run() {
       while (true) {
       while (true) {
@@ -197,11 +186,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
           LOG.debug("Starting launching task sweep");
           LOG.debug("Starting launching task sweep");
           synchronized (JobTracker.this) {
           synchronized (JobTracker.this) {
             synchronized (launchingTasks) {
             synchronized (launchingTasks) {
-              Iterator<Map.Entry<String, Long>> itr =
+              Iterator<Map.Entry<TaskAttemptID, Long>> itr =
                 launchingTasks.entrySet().iterator();
                 launchingTasks.entrySet().iterator();
               while (itr.hasNext()) {
               while (itr.hasNext()) {
-                Map.Entry<String, Long> pair = itr.next();
-                String taskId = pair.getKey();
+                Map.Entry<TaskAttemptID, Long> pair = itr.next();
+                TaskAttemptID taskId = pair.getKey();
                 long age = now - (pair.getValue()).longValue();
                 long age = now - (pair.getValue()).longValue();
                 LOG.info(taskId + " is " + age + " ms debug.");
                 LOG.info(taskId + " is " + age + " ms debug.");
                 if (age > TASKTRACKER_EXPIRY_INTERVAL) {
                 if (age > TASKTRACKER_EXPIRY_INTERVAL) {
@@ -243,14 +232,14 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       }
       }
     }
     }
       
       
-    public void addNewTask(String taskName) {
+    public void addNewTask(TaskAttemptID taskName) {
       synchronized (launchingTasks) {
       synchronized (launchingTasks) {
         launchingTasks.put(taskName, 
         launchingTasks.put(taskName, 
                            System.currentTimeMillis());
                            System.currentTimeMillis());
       }
       }
     }
     }
       
       
-    public void removeTask(String taskName) {
+    public void removeTask(TaskAttemptID taskName) {
       synchronized (launchingTasks) {
       synchronized (launchingTasks) {
         launchingTasks.remove(taskName);
         launchingTasks.remove(taskName);
       }
       }
@@ -365,7 +354,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                   synchronized (jobInitQueue) {
                   synchronized (jobInitQueue) {
                     for (JobInProgress job: retiredJobs) {
                     for (JobInProgress job: retiredJobs) {
                       removeJobTasks(job);
                       removeJobTasks(job);
-                      jobs.remove(job.getProfile().getJobId());
+                      jobs.remove(job.getProfile().getJobID());
                       jobInitQueue.remove(job);
                       jobInitQueue.remove(job);
                       jobsByPriority.remove(job);
                       jobsByPriority.remove(job);
                       String jobUser = job.getProfile().getUser();
                       String jobUser = job.getProfile().getUser();
@@ -380,7 +369,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                         }
                         }
                       }
                       }
                       LOG.info("Retired job with id: '" + 
                       LOG.info("Retired job with id: '" + 
-                               job.getProfile().getJobId() + "' of user '" +
+                               job.getProfile().getJobID() + "' of user '" +
                                jobUser + "'");
                                jobUser + "'");
                     }
                     }
                   }
                   }
@@ -536,7 +525,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   //
   //
 
 
   // All the known jobs.  (jobid->JobInProgress)
   // All the known jobs.  (jobid->JobInProgress)
-  Map<String, JobInProgress> jobs = new TreeMap<String, JobInProgress>();
+  Map<JobID, JobInProgress> jobs = new TreeMap<JobID, JobInProgress>();
   List<JobInProgress> jobsByPriority = new ArrayList<JobInProgress>();
   List<JobInProgress> jobsByPriority = new ArrayList<JobInProgress>();
 
 
   // (user -> list of JobInProgress)
   // (user -> list of JobInProgress)
@@ -544,19 +533,19 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     new TreeMap<String, ArrayList<JobInProgress>>();
     new TreeMap<String, ArrayList<JobInProgress>>();
     
     
   // All the known TaskInProgress items, mapped to by taskids (taskid->TIP)
   // All the known TaskInProgress items, mapped to by taskids (taskid->TIP)
-  Map<String, TaskInProgress> taskidToTIPMap =
-    new TreeMap<String, TaskInProgress>();
+  Map<TaskAttemptID, TaskInProgress> taskidToTIPMap =
+    new TreeMap<TaskAttemptID, TaskInProgress>();
 
 
   // (taskid --> trackerID) 
   // (taskid --> trackerID) 
-  TreeMap<String, String> taskidToTrackerMap = new TreeMap<String, String>();
+  TreeMap<TaskAttemptID, String> taskidToTrackerMap = new TreeMap<TaskAttemptID, String>();
 
 
   // (trackerID->TreeSet of taskids running at that tracker)
   // (trackerID->TreeSet of taskids running at that tracker)
-  TreeMap<String, Set<String>> trackerToTaskMap =
-    new TreeMap<String, Set<String>>();
+  TreeMap<String, Set<TaskAttemptID>> trackerToTaskMap =
+    new TreeMap<String, Set<TaskAttemptID>>();
 
 
   // (trackerID -> TreeSet of completed taskids running at that tracker)
   // (trackerID -> TreeSet of completed taskids running at that tracker)
-  TreeMap<String, Set<String>> trackerToMarkedTasksMap =
-    new TreeMap<String, Set<String>>();
+  TreeMap<String, Set<TaskAttemptID>> trackerToMarkedTasksMap =
+    new TreeMap<String, Set<TaskAttemptID>>();
 
 
   // (trackerID --> last sent HeartBeatResponse)
   // (trackerID --> last sent HeartBeatResponse)
   Map<String, HeartbeatResponse> trackerToHeartbeatResponseMap = 
   Map<String, HeartbeatResponse> trackerToHeartbeatResponseMap = 
@@ -879,16 +868,16 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   // Maintain lookup tables; called by JobInProgress
   // Maintain lookup tables; called by JobInProgress
   // and TaskInProgress
   // and TaskInProgress
   ///////////////////////////////////////////////////////
   ///////////////////////////////////////////////////////
-  void createTaskEntry(String taskid, String taskTracker, TaskInProgress tip) {
+  void createTaskEntry(TaskAttemptID taskid, String taskTracker, TaskInProgress tip) {
     LOG.info("Adding task '" + taskid + "' to tip " + tip.getTIPId() + ", for tracker '" + taskTracker + "'");
     LOG.info("Adding task '" + taskid + "' to tip " + tip.getTIPId() + ", for tracker '" + taskTracker + "'");
 
 
     // taskid --> tracker
     // taskid --> tracker
     taskidToTrackerMap.put(taskid, taskTracker);
     taskidToTrackerMap.put(taskid, taskTracker);
 
 
     // tracker --> taskid
     // tracker --> taskid
-    Set<String> taskset = trackerToTaskMap.get(taskTracker);
+    Set<TaskAttemptID> taskset = trackerToTaskMap.get(taskTracker);
     if (taskset == null) {
     if (taskset == null) {
-      taskset = new TreeSet<String>();
+      taskset = new TreeSet<TaskAttemptID>();
       trackerToTaskMap.put(taskTracker, taskset);
       trackerToTaskMap.put(taskTracker, taskset);
     }
     }
     taskset.add(taskid);
     taskset.add(taskid);
@@ -897,13 +886,13 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     taskidToTIPMap.put(taskid, tip);
     taskidToTIPMap.put(taskid, tip);
   }
   }
     
     
-  void removeTaskEntry(String taskid) {
+  void removeTaskEntry(TaskAttemptID taskid) {
     // taskid --> tracker
     // taskid --> tracker
     String tracker = taskidToTrackerMap.remove(taskid);
     String tracker = taskidToTrackerMap.remove(taskid);
 
 
     // tracker --> taskid
     // tracker --> taskid
     if (tracker != null) {
     if (tracker != null) {
-      Set<String> trackerSet = trackerToTaskMap.get(tracker);
+      Set<TaskAttemptID> trackerSet = trackerToTaskMap.get(tracker);
       if (trackerSet != null) {
       if (trackerSet != null) {
         trackerSet.remove(taskid);
         trackerSet.remove(taskid);
       }
       }
@@ -922,11 +911,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    * @param taskTracker the tasktracker at which the 'task' was running
    * @param taskTracker the tasktracker at which the 'task' was running
    * @param taskid completed (success/failure/killed) task
    * @param taskid completed (success/failure/killed) task
    */
    */
-  void markCompletedTaskAttempt(String taskTracker, String taskid) {
+  void markCompletedTaskAttempt(String taskTracker, TaskAttemptID taskid) {
     // tracker --> taskid
     // tracker --> taskid
-    Set<String> taskset = trackerToMarkedTasksMap.get(taskTracker);
+    Set<TaskAttemptID> taskset = trackerToMarkedTasksMap.get(taskTracker);
     if (taskset == null) {
     if (taskset == null) {
-      taskset = new TreeSet<String>();
+      taskset = new TreeSet<TaskAttemptID>();
       trackerToMarkedTasksMap.put(taskTracker, taskset);
       trackerToMarkedTasksMap.put(taskTracker, taskset);
     }
     }
     taskset.add(taskid);
     taskset.add(taskid);
@@ -946,7 +935,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         if (taskStatus.getRunState() != TaskStatus.State.RUNNING && 
         if (taskStatus.getRunState() != TaskStatus.State.RUNNING && 
             taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING) {
             taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING) {
           markCompletedTaskAttempt(taskStatus.getTaskTracker(), 
           markCompletedTaskAttempt(taskStatus.getTaskTracker(), 
-                                   taskStatus.getTaskId());
+                                   taskStatus.getTaskID());
         }
         }
       }
       }
     }
     }
@@ -955,7 +944,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         if (taskStatus.getRunState() != TaskStatus.State.RUNNING &&
         if (taskStatus.getRunState() != TaskStatus.State.RUNNING &&
             taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING) {
             taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING) {
           markCompletedTaskAttempt(taskStatus.getTaskTracker(), 
           markCompletedTaskAttempt(taskStatus.getTaskTracker(), 
-                                   taskStatus.getTaskId());
+                                   taskStatus.getTaskID());
         }
         }
       }
       }
     }
     }
@@ -970,10 +959,10 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    */
    */
   private void removeMarkedTasks(String taskTracker) {
   private void removeMarkedTasks(String taskTracker) {
     // Purge all the 'marked' tasks which were running at taskTracker
     // Purge all the 'marked' tasks which were running at taskTracker
-    Set<String> markedTaskSet = 
+    Set<TaskAttemptID> markedTaskSet = 
       trackerToMarkedTasksMap.get(taskTracker);
       trackerToMarkedTasksMap.get(taskTracker);
     if (markedTaskSet != null) {
     if (markedTaskSet != null) {
-      for (String taskid : markedTaskSet) {
+      for (TaskAttemptID taskid : markedTaskSet) {
         removeTaskEntry(taskid);
         removeTaskEntry(taskid);
         LOG.info("Removed completed task '" + taskid + "' from '" + 
         LOG.info("Removed completed task '" + taskid + "' from '" + 
                  taskTracker + "'");
                  taskTracker + "'");
@@ -997,12 +986,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   synchronized private void removeJobTasks(JobInProgress job) { 
   synchronized private void removeJobTasks(JobInProgress job) { 
     for (TaskInProgress tip : job.getMapTasks()) {
     for (TaskInProgress tip : job.getMapTasks()) {
       for (TaskStatus taskStatus : tip.getTaskStatuses()) {
       for (TaskStatus taskStatus : tip.getTaskStatuses()) {
-        removeTaskEntry(taskStatus.getTaskId());
+        removeTaskEntry(taskStatus.getTaskID());
       }
       }
     }
     }
     for (TaskInProgress tip : job.getReduceTasks()) {
     for (TaskInProgress tip : job.getReduceTasks()) {
       for (TaskStatus taskStatus : tip.getTaskStatuses()) {
       for (TaskStatus taskStatus : tip.getTaskStatuses()) {
-        removeTaskEntry(taskStatus.getTaskId());
+        removeTaskEntry(taskStatus.getTaskID());
       }
       }
     }
     }
   }
   }
@@ -1068,12 +1057,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                   removeJobTasks(rjob);
                   removeJobTasks(rjob);
                     
                     
                   userJobs.remove(0);
                   userJobs.remove(0);
-                  jobs.remove(rjob.getProfile().getJobId());
+                  jobs.remove(rjob.getProfile().getJobID());
                   jobInitQueue.remove(rjob);
                   jobInitQueue.remove(rjob);
                   jobsByPriority.remove(rjob);
                   jobsByPriority.remove(rjob);
                     
                     
                   LOG.info("Retired job with id: '" + 
                   LOG.info("Retired job with id: '" + 
-                           rjob.getProfile().getJobId() + "' of user: '" +
+                           rjob.getProfile().getJobID() + "' of user: '" +
                            jobUser + "'");
                            jobUser + "'");
                 } else {
                 } else {
                   // Do not remove jobs that aren't complete.
                   // Do not remove jobs that aren't complete.
@@ -1137,7 +1126,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    */
    */
   public synchronized List<JobInProgress> getRunningJobs() {
   public synchronized List<JobInProgress> getRunningJobs() {
     synchronized (jobs) {
     synchronized (jobs) {
-      return (List<JobInProgress>) runningJobs();
+      return runningJobs();
     }
     }
   }
   }
   public Vector<JobInProgress> failedJobs() {
   public Vector<JobInProgress> failedJobs() {
@@ -1308,7 +1297,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     if (acceptNewTasks) {
     if (acceptNewTasks) {
       Task task = getNewTaskForTaskTracker(trackerName);
       Task task = getNewTaskForTaskTracker(trackerName);
       if (task != null) {
       if (task != null) {
-        LOG.debug(trackerName + " -> LaunchTask: " + task.getTaskId());
+        LOG.debug(trackerName + " -> LaunchTask: " + task.getTaskID());
         actions.add(new LaunchTaskAction(task));
         actions.add(new LaunchTaskAction(task));
       }
       }
     }
     }
@@ -1458,6 +1447,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         } catch (InterruptedException ie) {}
         } catch (InterruptedException ie) {}
       }
       }
     }
     }
+    @Override
     public void run() {
     public void run() {
       while (!isInterrupted()) {
       while (!isInterrupted()) {
         try {
         try {
@@ -1584,7 +1574,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
 
 
           Task t = job.obtainNewMapTask(tts, numTaskTrackers);
           Task t = job.obtainNewMapTask(tts, numTaskTrackers);
           if (t != null) {
           if (t != null) {
-            expireLaunchingTasks.addNewTask(t.getTaskId());
+            expireLaunchingTasks.addNewTask(t.getTaskID());
             myMetrics.launchMap();
             myMetrics.launchMap();
             return t;
             return t;
           }
           }
@@ -1621,7 +1611,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
 
 
           Task t = job.obtainNewReduceTask(tts, numTaskTrackers);
           Task t = job.obtainNewReduceTask(tts, numTaskTrackers);
           if (t != null) {
           if (t != null) {
-            expireLaunchingTasks.addNewTask(t.getTaskId());
+            expireLaunchingTasks.addNewTask(t.getTaskID());
             myMetrics.launchReduce();
             myMetrics.launchReduce();
             return t;
             return t;
           }
           }
@@ -1654,11 +1644,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   private synchronized List<TaskTrackerAction> getTasksToKill(
   private synchronized List<TaskTrackerAction> getTasksToKill(
                                                               String taskTracker) {
                                                               String taskTracker) {
     
     
-    Set<String> taskIds = trackerToTaskMap.get(taskTracker);
+    Set<TaskAttemptID> taskIds = trackerToTaskMap.get(taskTracker);
     if (taskIds != null) {
     if (taskIds != null) {
       List<TaskTrackerAction> killList = new ArrayList<TaskTrackerAction>();
       List<TaskTrackerAction> killList = new ArrayList<TaskTrackerAction>();
-      Set<String> killJobIds = new TreeSet<String>(); 
-      for (String killTaskId : taskIds) {
+      Set<JobID> killJobIds = new TreeSet<JobID>(); 
+      for (TaskAttemptID killTaskId : taskIds) {
         TaskInProgress tip = taskidToTIPMap.get(killTaskId);
         TaskInProgress tip = taskidToTIPMap.get(killTaskId);
         if (tip.shouldClose(killTaskId)) {
         if (tip.shouldClose(killTaskId)) {
           // 
           // 
@@ -1670,13 +1660,13 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
             killList.add(new KillTaskAction(killTaskId));
             killList.add(new KillTaskAction(killTaskId));
             LOG.debug(taskTracker + " -> KillTaskAction: " + killTaskId);
             LOG.debug(taskTracker + " -> KillTaskAction: " + killTaskId);
           } else {
           } else {
-            String killJobId = tip.getJob().getStatus().getJobId(); 
+            JobID killJobId = tip.getJob().getStatus().getJobID(); 
             killJobIds.add(killJobId);
             killJobIds.add(killJobId);
           }
           }
         }
         }
       }
       }
             
             
-      for (String killJobId : killJobIds) {
+      for (JobID killJobId : killJobIds) {
         killList.add(new KillJobAction(killJobId));
         killList.add(new KillJobAction(killJobId));
         LOG.debug(taskTracker + " -> KillJobAction: " + killJobId);
         LOG.debug(taskTracker + " -> KillJobAction: " + killJobId);
       }
       }
@@ -1726,10 +1716,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   /**
   /**
    * Allocates a new JobId string.
    * Allocates a new JobId string.
    */
    */
-  public synchronized String getNewJobId() throws IOException {
+  public synchronized JobID getNewJobId() throws IOException {
     ensureRunning();
     ensureRunning();
-    return "job_" + getTrackerIdentifier() + "_" + 
-             idFormat.format(nextJobId++);
+    return new JobID(getTrackerIdentifier(), nextJobId++);
   }
   }
 
 
   /**
   /**
@@ -1744,7 +1733,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    * asynchronously to handle split-computation and build up
    * asynchronously to handle split-computation and build up
    * the right TaskTracker/Block mapping.
    * the right TaskTracker/Block mapping.
    */
    */
-  public synchronized JobStatus submitJob(String jobId) throws IOException {
+  public synchronized JobStatus submitJob(JobID jobId) throws IOException {
     ensureRunning();
     ensureRunning();
     if(jobs.containsKey(jobId)) {
     if(jobs.containsKey(jobId)) {
       //job already running, don't start twice
       //job already running, don't start twice
@@ -1756,7 +1745,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     synchronized (jobs) {
     synchronized (jobs) {
       synchronized (jobsByPriority) {
       synchronized (jobsByPriority) {
         synchronized (jobInitQueue) {
         synchronized (jobInitQueue) {
-          jobs.put(job.getProfile().getJobId(), job);
+          jobs.put(job.getProfile().getJobID(), job);
           jobsByPriority.add(job);
           jobsByPriority.add(job);
           jobInitQueue.add(job);
           jobInitQueue.add(job);
           resortPriority();
           resortPriority();
@@ -1805,12 +1794,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     }
     }
   }
   }
     
     
-  public synchronized void killJob(String jobid) {
+  public synchronized void killJob(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     job.kill();
     job.kill();
   }
   }
 
 
-  public synchronized JobProfile getJobProfile(String jobid) {
+  public synchronized JobProfile getJobProfile(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
       return job.getProfile();
       return job.getProfile();
@@ -1818,7 +1807,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       return completedJobStatusStore.readJobProfile(jobid);
       return completedJobStatusStore.readJobProfile(jobid);
     }
     }
   }
   }
-  public synchronized JobStatus getJobStatus(String jobid) {
+  public synchronized JobStatus getJobStatus(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
       return job.getStatus();
       return job.getStatus();
@@ -1826,7 +1815,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       return completedJobStatusStore.readJobStatus(jobid);
       return completedJobStatusStore.readJobStatus(jobid);
     }
     }
   }
   }
-  public synchronized Counters getJobCounters(String jobid) {
+  public synchronized Counters getJobCounters(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job != null) {
     if (job != null) {
       return job.getCounters();
       return job.getCounters();
@@ -1834,7 +1823,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       return completedJobStatusStore.readCounters(jobid);
       return completedJobStatusStore.readCounters(jobid);
     }
     }
   }
   }
-  public synchronized TaskReport[] getMapTaskReports(String jobid) {
+  public synchronized TaskReport[] getMapTaskReports(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job == null) {
     if (job == null) {
       return new TaskReport[0];
       return new TaskReport[0];
@@ -1856,7 +1845,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     }
     }
   }
   }
 
 
-  public synchronized TaskReport[] getReduceTaskReports(String jobid) {
+  public synchronized TaskReport[] getReduceTaskReports(JobID jobid) {
     JobInProgress job = jobs.get(jobid);
     JobInProgress job = jobs.get(jobid);
     if (job == null) {
     if (job == null) {
       return new TaskReport[0];
       return new TaskReport[0];
@@ -1882,8 +1871,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    * @see org.apache.hadoop.mapred.JobSubmissionProtocol#getTaskCompletionEvents(java.lang.String, int, int)
    * @see org.apache.hadoop.mapred.JobSubmissionProtocol#getTaskCompletionEvents(java.lang.String, int, int)
    */
    */
   public synchronized TaskCompletionEvent[] getTaskCompletionEvents(
   public synchronized TaskCompletionEvent[] getTaskCompletionEvents(
-                                                                    String jobid, int fromEventId, int maxEvents) throws IOException{
+      JobID jobid, int fromEventId, int maxEvents) throws IOException{
     TaskCompletionEvent[] events;
     TaskCompletionEvent[] events;
+
     JobInProgress job = this.jobs.get(jobid);
     JobInProgress job = this.jobs.get(jobid);
     if (null != job) {
     if (null != job) {
       events = job.getTaskCompletionEvents(fromEventId, maxEvents);
       events = job.getTaskCompletionEvents(fromEventId, maxEvents);
@@ -1896,15 +1886,14 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
 
 
   /**
   /**
    * Get the diagnostics for a given task
    * Get the diagnostics for a given task
-   * @param jobId the id of the job
-   * @param tipId the id of the tip 
    * @param taskId the id of the task
    * @param taskId the id of the task
    * @return an array of the diagnostic messages
    * @return an array of the diagnostic messages
    */
    */
-  public synchronized String[] getTaskDiagnostics(String jobId, 
-                                                  String tipId, 
-                                                  String taskId) 
-  throws IOException {
+  public synchronized String[] getTaskDiagnostics(TaskAttemptID taskId)  
+    throws IOException {
+    
+    JobID jobId = taskId.getJobID();
+    TaskID tipId = taskId.getTaskID();
     JobInProgress job = jobs.get(jobId);
     JobInProgress job = jobs.get(jobId);
     if (job == null) {
     if (job == null) {
       throw new IllegalArgumentException("Job " + jobId + " not found.");
       throw new IllegalArgumentException("Job " + jobId + " not found.");
@@ -1919,15 +1908,15 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   }
   }
     
     
   /** Get all the TaskStatuses from the tipid. */
   /** Get all the TaskStatuses from the tipid. */
-  TaskStatus[] getTaskStatuses(String jobid, String tipid) {
-    TaskInProgress tip = getTip(jobid, tipid);
+  TaskStatus[] getTaskStatuses(TaskID tipid) {
+    TaskInProgress tip = getTip(tipid);
     return (tip == null ? new TaskStatus[0] 
     return (tip == null ? new TaskStatus[0] 
             : tip.getTaskStatuses());
             : tip.getTaskStatuses());
   }
   }
 
 
   /** Returns the TaskStatus for a particular taskid. */
   /** Returns the TaskStatus for a particular taskid. */
-  TaskStatus getTaskStatus(String jobid, String tipid, String taskid) {
-    TaskInProgress tip = getTip(jobid, tipid);
+  TaskStatus getTaskStatus(TaskAttemptID taskid) {
+    TaskInProgress tip = getTip(taskid.getTaskID());
     return (tip == null ? null 
     return (tip == null ? null 
             : tip.getTaskStatus(taskid));
             : tip.getTaskStatus(taskid));
   }
   }
@@ -1935,21 +1924,21 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   /**
   /**
    * Returns the counters for the specified task in progress.
    * Returns the counters for the specified task in progress.
    */
    */
-  Counters getTipCounters(String jobid, String tipid) {
-    TaskInProgress tip = getTip(jobid, tipid);
+  Counters getTipCounters(TaskID tipid) {
+    TaskInProgress tip = getTip(tipid);
     return (tip == null ? null : tip.getCounters());
     return (tip == null ? null : tip.getCounters());
   }
   }
     
     
   /**
   /**
    * Returns specified TaskInProgress, or null.
    * Returns specified TaskInProgress, or null.
    */
    */
-  private TaskInProgress getTip(String jobid, String tipid) {
-    JobInProgress job = jobs.get(jobid);
+  private TaskInProgress getTip(TaskID tipid) {
+    JobInProgress job = jobs.get(tipid.getJobID());
     return (job == null ? null : job.getTaskInProgress(tipid));
     return (job == null ? null : job.getTaskInProgress(tipid));
   }
   }
     
     
   /** Mark a Task to be killed */
   /** Mark a Task to be killed */
-  public synchronized boolean killTask(String 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);
     if(tip != null) {
     if(tip != null) {
       return tip.killTask(taskid, shouldFail);
       return tip.killTask(taskid, shouldFail);
@@ -1965,7 +1954,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    * @param taskId the name of the task
    * @param taskId the name of the task
    * @return The name of the task tracker
    * @return The name of the task tracker
    */
    */
-  public synchronized String getAssignedTracker(String taskId) {
+  public synchronized String getAssignedTracker(TaskAttemptID taskId) {
     return taskidToTrackerMap.get(taskId);
     return taskidToTrackerMap.get(taskId);
   }
   }
     
     
@@ -1999,7 +1988,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
   ///////////////////////////////////////////////////////////////
   ///////////////////////////////////////////////////////////////
   // JobTracker methods
   // JobTracker methods
   ///////////////////////////////////////////////////////////////
   ///////////////////////////////////////////////////////////////
-  public JobInProgress getJob(String jobid) {
+  public JobInProgress getJob(JobID jobid) {
     return jobs.get(jobid);
     return jobs.get(jobid);
   }
   }
 
 
@@ -2008,7 +1997,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    * @param jobId job id
    * @param jobId job id
    * @param priority new {@link JobPriority} for the job
    * @param priority new {@link JobPriority} for the job
    */
    */
-  synchronized void setJobPriority(String jobId, JobPriority priority) {
+  synchronized void setJobPriority(JobID jobId, JobPriority priority) {
     JobInProgress job = jobs.get(jobId);
     JobInProgress job = jobs.get(jobId);
     if (job != null) {
     if (job != null) {
       job.setPriority(priority);
       job.setPriority(priority);
@@ -2034,19 +2023,19 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     String trackerName = status.getTrackerName();
     String trackerName = status.getTrackerName();
     for (TaskStatus report : status.getTaskReports()) {
     for (TaskStatus report : status.getTaskReports()) {
       report.setTaskTracker(trackerName);
       report.setTaskTracker(trackerName);
-      String taskId = report.getTaskId();
+      TaskAttemptID taskId = report.getTaskID();
       TaskInProgress tip = taskidToTIPMap.get(taskId);
       TaskInProgress tip = taskidToTIPMap.get(taskId);
       if (tip == null) {
       if (tip == null) {
-        LOG.info("Serious problem.  While updating status, cannot find taskid " + report.getTaskId());
+        LOG.info("Serious problem.  While updating status, cannot find taskid " + report.getTaskID());
       } else {
       } else {
         expireLaunchingTasks.removeTask(taskId);
         expireLaunchingTasks.removeTask(taskId);
         tip.getJob().updateTaskStatus(tip, report, myMetrics);
         tip.getJob().updateTaskStatus(tip, report, myMetrics);
       }
       }
       
       
       // Process 'failed fetch' notifications 
       // Process 'failed fetch' notifications 
-      List<String> failedFetchMaps = report.getFetchFailedMaps();
+      List<TaskAttemptID> failedFetchMaps = report.getFetchFailedMaps();
       if (failedFetchMaps != null) {
       if (failedFetchMaps != null) {
-        for (String mapTaskId : failedFetchMaps) {
+        for (TaskAttemptID mapTaskId : failedFetchMaps) {
           TaskInProgress failedFetchMap = taskidToTIPMap.get(mapTaskId);
           TaskInProgress failedFetchMap = taskidToTIPMap.get(mapTaskId);
           
           
           if (failedFetchMap != null) {
           if (failedFetchMap != null) {
@@ -2072,13 +2061,13 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    */
    */
   void lostTaskTracker(String trackerName) {
   void lostTaskTracker(String trackerName) {
     LOG.info("Lost tracker '" + trackerName + "'");
     LOG.info("Lost tracker '" + trackerName + "'");
-    Set<String> lostTasks = trackerToTaskMap.get(trackerName);
+    Set<TaskAttemptID> lostTasks = trackerToTaskMap.get(trackerName);
     trackerToTaskMap.remove(trackerName);
     trackerToTaskMap.remove(trackerName);
 
 
     if (lostTasks != null) {
     if (lostTasks != null) {
       // List of jobs which had any of their tasks fail on this tracker
       // List of jobs which had any of their tasks fail on this tracker
       Set<JobInProgress> jobsWithFailures = new HashSet<JobInProgress>(); 
       Set<JobInProgress> jobsWithFailures = new HashSet<JobInProgress>(); 
-      for (String taskId : lostTasks) {
+      for (TaskAttemptID taskId : lostTasks) {
         TaskInProgress tip = taskidToTIPMap.get(taskId);
         TaskInProgress tip = taskidToTIPMap.get(taskId);
 
 
         // Completed reduce tasks never need to be failed, because 
         // Completed reduce tasks never need to be failed, because 
@@ -2147,6 +2136,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       }
       }
     }
     }
        
        
+    @Override
     public void run() {
     public void run() {
       int  batchCommitSize = conf.getInt("jobtracker.task.commit.batch.size", 
       int  batchCommitSize = conf.getInt("jobtracker.task.commit.batch.size", 
                                          5000); 
                                          5000); 
@@ -2160,7 +2150,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
 
 
           JobInProgress[] jobs = new JobInProgress[jobList.size()];
           JobInProgress[] jobs = new JobInProgress[jobList.size()];
           TaskInProgress[] tips = new TaskInProgress[jobList.size()];
           TaskInProgress[] tips = new TaskInProgress[jobList.size()];
-          String[] taskids = new String[jobList.size()];
+          TaskAttemptID[] taskids = new TaskAttemptID[jobList.size()];
           JobTrackerMetrics[] metrics = new JobTrackerMetrics[jobList.size()];
           JobTrackerMetrics[] metrics = new JobTrackerMetrics[jobList.size()];
 
 
           Iterator<JobInProgress.JobWithTaskContext> iter = jobList.iterator();
           Iterator<JobInProgress.JobWithTaskContext> iter = jobList.iterator();
@@ -2170,7 +2160,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
             JobInProgress.JobWithTaskContext j = iter.next();
             JobInProgress.JobWithTaskContext j = iter.next();
             jobs[count] = j.getJob();
             jobs[count] = j.getJob();
             tips[count] = j.getTIP();
             tips[count] = j.getTIP();
-            taskids[count]= j.getTaskId();
+            taskids[count]= j.getTaskID();
             metrics[count] = j.getJobTrackerMetrics();
             metrics[count] = j.getJobTrackerMetrics();
             ++count;
             ++count;
           }
           }
@@ -2185,7 +2175,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
               synchronized (jobs[i]) {
               synchronized (jobs[i]) {
                 synchronized (tips[i]) {
                 synchronized (tips[i]) {
                   status[i] = tips[i].getTaskStatus(taskids[i]);
                   status[i] = tips[i].getTaskStatus(taskids[i]);
-                  tasks[i] = tips[i].getTaskObject(taskids[i]);
+                  tasks[i] = tips[i].getTask(taskids[i]);
                   states[i] = status[i].getRunState();
                   states[i] = status[i].getRunState();
                   isTipComplete[i] = tips[i].isComplete();
                   isTipComplete[i] = tips[i].isComplete();
                 }
                 }
@@ -2203,7 +2193,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
           //is detected so that the JT can immediately schedule another
           //is detected so that the JT can immediately schedule another
           //attempt for that task.
           //attempt for that task.
 
 
-          Set<String> seenTIPs = new HashSet<String>();
+          Set<TaskID> seenTIPs = new HashSet<TaskID>();
           for(int index = 0; index < jobList.size(); ++index) {
           for(int index = 0; index < jobList.size(); ++index) {
             try {
             try {
               if (states[index] == TaskStatus.State.COMMIT_PENDING) {
               if (states[index] == TaskStatus.State.COMMIT_PENDING) {
@@ -2227,12 +2217,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                 TaskStatus.Phase phase = (tips[index].isMapTask() 
                 TaskStatus.Phase phase = (tips[index].isMapTask() 
                                           ? TaskStatus.Phase.MAP 
                                           ? TaskStatus.Phase.MAP 
                                           : TaskStatus.Phase.REDUCE);
                                           : TaskStatus.Phase.REDUCE);
-                jobs[index].failedTask(tips[index], status[index].getTaskId(), 
+                jobs[index].failedTask(tips[index], status[index].getTaskID(), 
                                        reason, phase, TaskStatus.State.FAILED, 
                                        reason, phase, TaskStatus.State.FAILED, 
                                        status[index].getTaskTracker(), null);
                                        status[index].getTaskTracker(), null);
               }
               }
               LOG.info("Failed to rename the output of " 
               LOG.info("Failed to rename the output of " 
-                       + status[index].getTaskId() + " with " 
+                       + status[index].getTaskID() + " with " 
                        + StringUtils.stringifyException(ioe));
                        + StringUtils.stringifyException(ioe));
             }
             }
           }
           }
@@ -2255,7 +2245,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                         states[i] = TaskStatus.State.SUCCEEDED;
                         states[i] = TaskStatus.State.SUCCEEDED;
                       }
                       }
                     } else {
                     } else {
-                      tips[i].addDiagnosticInfo(tasks[i].getTaskId(), 
+                      tips[i].addDiagnosticInfo(tasks[i].getTaskID(), 
                                                 "Already completed  TIP");
                                                 "Already completed  TIP");
                       states[i] = TaskStatus.State.KILLED;
                       states[i] = TaskStatus.State.KILLED;
                     }
                     }
@@ -2296,7 +2286,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
    * @param jobId id of the job
    * @param jobId id of the job
    * @return the path of the job conf file on the local file system
    * @return the path of the job conf file on the local file system
    */
    */
-  public static String getLocalJobFilePath(String jobId){
+  public static String getLocalJobFilePath(JobID jobId){
     return JobHistory.JobInfo.getLocalJobFilePath(jobId);
     return JobHistory.JobInfo.getLocalJobFilePath(jobId);
   }
   }
   ////////////////////////////////////////////////////////////
   ////////////////////////////////////////////////////////////

+ 8 - 7
src/java/org/apache/hadoop/mapred/KillJobAction.java

@@ -22,7 +22,6 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.hadoop.io.Text;
 
 
 /**
 /**
  * Represents a directive from the {@link org.apache.hadoop.mapred.JobTracker} 
  * Represents a directive from the {@link org.apache.hadoop.mapred.JobTracker} 
@@ -31,27 +30,29 @@ import org.apache.hadoop.io.Text;
  * 
  * 
  */
  */
 class KillJobAction extends TaskTrackerAction {
 class KillJobAction extends TaskTrackerAction {
-  String jobId;
+  JobID jobId;
 
 
   public KillJobAction() {
   public KillJobAction() {
     super(ActionType.KILL_JOB);
     super(ActionType.KILL_JOB);
   }
   }
 
 
-  public KillJobAction(String taskId) {
+  public KillJobAction(JobID jobId) {
     super(ActionType.KILL_JOB);
     super(ActionType.KILL_JOB);
-    this.jobId = taskId;
+    this.jobId = jobId;
   }
   }
   
   
-  public String getJobId() {
+  public JobID getJobID() {
     return jobId;
     return jobId;
   }
   }
   
   
+  @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    Text.writeString(out, jobId);
+    jobId.write(out);
   }
   }
 
 
+  @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    jobId = Text.readString(in);
+    jobId = JobID.read(in);
   }
   }
 
 
 }
 }

+ 7 - 6
src/java/org/apache/hadoop/mapred/KillTaskAction.java

@@ -22,7 +22,6 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.hadoop.io.Text;
 
 
 /**
 /**
  * Represents a directive from the {@link org.apache.hadoop.mapred.JobTracker} 
  * Represents a directive from the {@link org.apache.hadoop.mapred.JobTracker} 
@@ -30,26 +29,28 @@ import org.apache.hadoop.io.Text;
  * 
  * 
  */
  */
 class KillTaskAction extends TaskTrackerAction {
 class KillTaskAction extends TaskTrackerAction {
-  String taskId;
+  TaskAttemptID taskId;
   
   
   public KillTaskAction() {
   public KillTaskAction() {
     super(ActionType.KILL_TASK);
     super(ActionType.KILL_TASK);
   }
   }
   
   
-  public KillTaskAction(String taskId) {
+  public KillTaskAction(TaskAttemptID taskId) {
     super(ActionType.KILL_TASK);
     super(ActionType.KILL_TASK);
     this.taskId = taskId;
     this.taskId = taskId;
   }
   }
 
 
-  public String getTaskId() {
+  public TaskAttemptID getTaskID() {
     return taskId;
     return taskId;
   }
   }
   
   
+  @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    Text.writeString(out, taskId);
+    taskId.write(out);
   }
   }
 
 
+  @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    taskId = Text.readString(in);
+    taskId = TaskAttemptID.read(in);
   }
   }
 }
 }

+ 38 - 53
src/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -19,10 +19,8 @@
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
-import java.util.Random;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -39,13 +37,15 @@ class LocalJobRunner implements JobSubmissionProtocol {
     LogFactory.getLog("org.apache.hadoop.mapred.LocalJobRunner");
     LogFactory.getLog("org.apache.hadoop.mapred.LocalJobRunner");
 
 
   private FileSystem fs;
   private FileSystem fs;
-  private HashMap<String, Job> jobs = new HashMap<String, Job>();
+  private HashMap<JobID, Job> jobs = new HashMap<JobID, Job>();
   private JobConf conf;
   private JobConf conf;
   private int map_tasks = 0;
   private int map_tasks = 0;
   private int reduce_tasks = 0;
   private int reduce_tasks = 0;
 
 
   private JobTrackerMetrics myMetrics = null;
   private JobTrackerMetrics myMetrics = null;
 
 
+  private static final String jobDir =  "localRunner/";
+  
   public long getProtocolVersion(String protocol, long clientVersion) {
   public long getProtocolVersion(String protocol, long clientVersion) {
     return JobSubmissionProtocol.versionID;
     return JobSubmissionProtocol.versionID;
   }
   }
@@ -53,12 +53,11 @@ class LocalJobRunner implements JobSubmissionProtocol {
   private class Job extends Thread
   private class Job extends Thread
     implements TaskUmbilicalProtocol {
     implements TaskUmbilicalProtocol {
     private Path file;
     private Path file;
-    private String id;
+    private JobID id;
     private JobConf job;
     private JobConf job;
-    private Random random = new Random();
 
 
     private JobStatus status;
     private JobStatus status;
-    private ArrayList<String> mapIds = new ArrayList<String>();
+    private ArrayList<TaskAttemptID> mapIds = new ArrayList<TaskAttemptID>();
     private MapOutputFile mapoutputFile;
     private MapOutputFile mapoutputFile;
     private JobProfile profile;
     private JobProfile profile;
     private Path localFile;
     private Path localFile;
@@ -75,13 +74,13 @@ class LocalJobRunner implements JobSubmissionProtocol {
       return TaskUmbilicalProtocol.versionID;
       return TaskUmbilicalProtocol.versionID;
     }
     }
     
     
-    public Job(String jobid, JobConf conf) throws IOException {
+    public Job(JobID jobid, JobConf conf) throws IOException {
       this.file = new Path(conf.getSystemDir(), jobid + "/job.xml");
       this.file = new Path(conf.getSystemDir(), jobid + "/job.xml");
       this.id = jobid;
       this.id = jobid;
       this.mapoutputFile = new MapOutputFile(jobid);
       this.mapoutputFile = new MapOutputFile(jobid);
       this.mapoutputFile.setConf(conf);
       this.mapoutputFile.setConf(conf);
 
 
-      this.localFile = new JobConf(conf).getLocalPath("localRunner/"+id+".xml");
+      this.localFile = new JobConf(conf).getLocalPath(jobDir+id+".xml");
       this.localFs = FileSystem.getLocal(conf);
       this.localFs = FileSystem.getLocal(conf);
 
 
       fs.copyToLocalFile(file, localFile);
       fs.copyToLocalFile(file, localFile);
@@ -105,7 +104,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
         // split input into minimum number of splits
         // split input into minimum number of splits
         InputSplit[] splits;
         InputSplit[] splits;
         splits = job.getInputFormat().getSplits(job, 1);
         splits = job.getInputFormat().getSplits(job, 1);
-        String jobId = profile.getJobId();
+        JobID jobId = profile.getJobID();
         
         
         int numReduceTasks = job.getNumReduceTasks();
         int numReduceTasks = job.getNumReduceTasks();
         if (numReduceTasks > 1 || numReduceTasks < 0) {
         if (numReduceTasks > 1 || numReduceTasks < 0) {
@@ -127,13 +126,13 @@ class LocalJobRunner implements JobSubmissionProtocol {
 
 
         DataOutputBuffer buffer = new DataOutputBuffer();
         DataOutputBuffer buffer = new DataOutputBuffer();
         for (int i = 0; i < splits.length; i++) {
         for (int i = 0; i < splits.length; i++) {
-          String mapId = jobId + "_map_" + idFormat.format(i);
+          TaskAttemptID mapId = new TaskAttemptID(new TaskID(jobId, true, i), 0);  
           mapIds.add(mapId);
           mapIds.add(mapId);
           buffer.reset();
           buffer.reset();
           splits[i].write(buffer);
           splits[i].write(buffer);
           BytesWritable split = new BytesWritable();
           BytesWritable split = new BytesWritable();
           split.set(buffer.getData(), 0, buffer.getLength());
           split.set(buffer.getData(), 0, buffer.getLength());
-          MapTask map = new MapTask(jobId, file.toString(), "tip_m_" + mapId, 
+          MapTask map = new MapTask(file.toString(),  
                                     mapId, i,
                                     mapId, i,
                                     splits[i].getClass().getName(),
                                     splits[i].getClass().getName(),
                                     split);
                                     split);
@@ -160,12 +159,12 @@ class LocalJobRunner implements JobSubmissionProtocol {
           map_tasks -= 1;
           map_tasks -= 1;
           updateCounters(map);
           updateCounters(map);
         }
         }
-        String reduceId = "reduce_" + newId();
+        TaskAttemptID reduceId = new TaskAttemptID(new TaskID(jobId, false, 0), 0);
         try {
         try {
           if (numReduceTasks > 0) {
           if (numReduceTasks > 0) {
             // move map output to reduce input  
             // move map output to reduce input  
             for (int i = 0; i < mapIds.size(); i++) {
             for (int i = 0; i < mapIds.size(); i++) {
-              String mapId = mapIds.get(i);
+              TaskAttemptID mapId = mapIds.get(i);
               Path mapOut = this.mapoutputFile.getOutputFile(mapId);
               Path mapOut = this.mapoutputFile.getOutputFile(mapId);
               Path reduceIn = this.mapoutputFile.getInputFileForWrite(i,reduceId,
               Path reduceIn = this.mapoutputFile.getInputFileForWrite(i,reduceId,
                   localFs.getLength(mapOut));
                   localFs.getLength(mapOut));
@@ -178,8 +177,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
             }
             }
 
 
             {
             {
-              ReduceTask reduce = new ReduceTask(jobId, file.toString(), 
-                                                 "tip_r_0001",
+              ReduceTask reduce = new ReduceTask(file.toString(), 
                                                  reduceId, 0, mapIds.size());
                                                  reduceId, 0, mapIds.size());
               JobConf localConf = new JobConf(job);
               JobConf localConf = new JobConf(job);
               if (outputFs != null) {
               if (outputFs != null) {
@@ -206,8 +204,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
             }
             }
           }
           }
         } finally {
         } finally {
-          for (int i = 0; i < mapIds.size(); i++) {
-            String mapId = mapIds.get(i);
+          for (TaskAttemptID mapId: mapIds) {
             this.mapoutputFile.removeAll(mapId);
             this.mapoutputFile.removeAll(mapId);
           }
           }
           if (numReduceTasks == 1) {
           if (numReduceTasks == 1) {
@@ -244,16 +241,12 @@ class LocalJobRunner implements JobSubmissionProtocol {
         }
         }
       }
       }
     }
     }
-    
-    private String newId() {
-      return Integer.toString(random.nextInt(Integer.MAX_VALUE), 36);
-    }
 
 
     // TaskUmbilicalProtocol methods
     // TaskUmbilicalProtocol methods
 
 
-    public Task getTask(String taskid) { return null; }
+    public Task getTask(TaskAttemptID taskid) { return null; }
 
 
-    public boolean statusUpdate(String taskId, TaskStatus taskStatus) 
+    public boolean statusUpdate(TaskAttemptID taskId, TaskStatus taskStatus) 
     throws IOException, InterruptedException {
     throws IOException, InterruptedException {
       LOG.info(taskStatus.getStateString());
       LOG.info(taskStatus.getStateString());
       float taskIndex = mapIds.indexOf(taskId);
       float taskIndex = mapIds.indexOf(taskId);
@@ -279,15 +272,15 @@ class LocalJobRunner implements JobSubmissionProtocol {
       completedTaskCounters.incrAllCounters(task.getCounters());
       completedTaskCounters.incrAllCounters(task.getCounters());
     }
     }
 
 
-    public void reportDiagnosticInfo(String taskid, String trace) {
+    public void reportDiagnosticInfo(TaskAttemptID taskid, String trace) {
       // Ignore for now
       // Ignore for now
     }
     }
 
 
-    public boolean ping(String taskid) throws IOException {
+    public boolean ping(TaskAttemptID taskid) throws IOException {
       return true;
       return true;
     }
     }
 
 
-    public void done(String taskId, boolean shouldPromote) throws IOException {
+    public void done(TaskAttemptID taskId, boolean shouldPromote) throws IOException {
       int taskIndex = mapIds.indexOf(taskId);
       int taskIndex = mapIds.indexOf(taskId);
       if (taskIndex >= 0) {                       // mapping
       if (taskIndex >= 0) {                       // mapping
         status.setMapProgress(1.0f);
         status.setMapProgress(1.0f);
@@ -296,17 +289,17 @@ class LocalJobRunner implements JobSubmissionProtocol {
       }
       }
     }
     }
 
 
-    public synchronized void fsError(String taskId, String message) 
+    public synchronized void fsError(TaskAttemptID taskId, String message) 
     throws IOException {
     throws IOException {
       LOG.fatal("FSError: "+ message + "from task: " + taskId);
       LOG.fatal("FSError: "+ message + "from task: " + taskId);
     }
     }
 
 
-    public void shuffleError(String taskId, String message) throws IOException {
+    public void shuffleError(TaskAttemptID taskId, String message) throws IOException {
       LOG.fatal("shuffleError: "+ message + "from task: " + taskId);
       LOG.fatal("shuffleError: "+ message + "from task: " + taskId);
     }
     }
     
     
-    public TaskCompletionEvent[] getMapCompletionEvents(
-                                                        String jobId, int fromEventId, int maxLocs) throws IOException {
+    public TaskCompletionEvent[] getMapCompletionEvents(JobID jobId
+        , int fromEventId, int maxLocs) throws IOException {
       return TaskCompletionEvent.EMPTY_ARRAY;
       return TaskCompletionEvent.EMPTY_ARRAY;
     }
     }
     
     
@@ -321,25 +314,25 @@ class LocalJobRunner implements JobSubmissionProtocol {
   // JobSubmissionProtocol methods
   // JobSubmissionProtocol methods
 
 
   private static int jobid = 0;
   private static int jobid = 0;
-  public synchronized String getNewJobId() {
-    return "job_local_" + Integer.toString(++jobid);
+  public synchronized JobID getNewJobId() {
+    return new JobID("local", ++jobid);
   }
   }
 
 
-  public JobStatus submitJob(String jobid) throws IOException {
+  public JobStatus submitJob(JobID jobid) throws IOException {
     return new Job(jobid, this.conf).status;
     return new Job(jobid, this.conf).status;
   }
   }
 
 
-  public void killJob(String id) {
+  public void killJob(JobID id) {
     jobs.get(id).stop();
     jobs.get(id).stop();
   }
   }
 
 
   /** Throws {@link UnsupportedOperationException} */
   /** Throws {@link UnsupportedOperationException} */
-  public boolean killTask(String taskId, boolean shouldFail) throws IOException {
+  public boolean killTask(TaskAttemptID taskId, boolean shouldFail) throws IOException {
     throw new UnsupportedOperationException("Killing tasks in " +
     throw new UnsupportedOperationException("Killing tasks in " +
     "LocalJobRunner is not supported");
     "LocalJobRunner is not supported");
   }
   }
 
 
-  public JobProfile getJobProfile(String id) {
+  public JobProfile getJobProfile(JobID id) {
     Job job = jobs.get(id);
     Job job = jobs.get(id);
     if(job != null)
     if(job != null)
       return job.getProfile();
       return job.getProfile();
@@ -347,14 +340,14 @@ class LocalJobRunner implements JobSubmissionProtocol {
       return null;
       return null;
   }
   }
 
 
-  public TaskReport[] getMapTaskReports(String id) {
+  public TaskReport[] getMapTaskReports(JobID id) {
     return new TaskReport[0];
     return new TaskReport[0];
   }
   }
-  public TaskReport[] getReduceTaskReports(String id) {
+  public TaskReport[] getReduceTaskReports(JobID id) {
     return new TaskReport[0];
     return new TaskReport[0];
   }
   }
 
 
-  public JobStatus getJobStatus(String id) {
+  public JobStatus getJobStatus(JobID id) {
     Job job = jobs.get(id);
     Job job = jobs.get(id);
     if(job != null)
     if(job != null)
       return job.status;
       return job.status;
@@ -362,7 +355,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
       return null;
       return null;
   }
   }
   
   
-  public Counters getJobCounters(String id) {
+  public Counters getJobCounters(JobID id) {
     Job job = jobs.get(id);
     Job job = jobs.get(id);
     return job.currentCounters;
     return job.currentCounters;
   }
   }
@@ -377,28 +370,20 @@ class LocalJobRunner implements JobSubmissionProtocol {
   }
   }
 
 
   public JobStatus[] jobsToComplete() {return null;}
   public JobStatus[] jobsToComplete() {return null;}
-  
-  public JobStatus[] getAllJobs() {return null;}
 
 
-  public TaskCompletionEvent[] getTaskCompletionEvents(
-                                                       String jobid, int fromEventId, int maxEvents) throws IOException{
+  public TaskCompletionEvent[] getTaskCompletionEvents(JobID jobid
+      , int fromEventId, int maxEvents) throws IOException {
     return TaskCompletionEvent.EMPTY_ARRAY;
     return TaskCompletionEvent.EMPTY_ARRAY;
   }
   }
   
   
-  /**
-   * Used for formatting the id numbers
-   */
-  private static NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setMinimumIntegerDigits(4);
-    idFormat.setGroupingUsed(false);
-  }
+  public JobStatus[] getAllJobs() {return null;}
+
   
   
   /**
   /**
    * Returns the diagnostic information for a particular task in the given job.
    * Returns the diagnostic information for a particular task in the given job.
    * To be implemented
    * To be implemented
    */
    */
-  public String[] getTaskDiagnostics(String jobid, String tipId, String taskid)
+  public String[] getTaskDiagnostics(TaskAttemptID taskid)
   		throws IOException{
   		throws IOException{
 	  return new String [0];
 	  return new String [0];
   }
   }

+ 15 - 15
src/java/org/apache/hadoop/mapred/MapOutputFile.java

@@ -20,9 +20,9 @@ package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.Path;
 
 
 /**
 /**
  * Manipulate the working area for the transient store for maps and reduces.
  * Manipulate the working area for the transient store for maps and reduces.
@@ -35,7 +35,7 @@ class MapOutputFile {
   MapOutputFile() {
   MapOutputFile() {
   }
   }
 
 
-  MapOutputFile(String jobId) {
+  MapOutputFile(JobID jobId) {
     this.jobDir = TaskTracker.getJobCacheSubdir() + Path.SEPARATOR + jobId;
     this.jobDir = TaskTracker.getJobCacheSubdir() + Path.SEPARATOR + jobId;
   }
   }
 
 
@@ -45,7 +45,7 @@ class MapOutputFile {
   /** Return the path to local map output file created earlier
   /** Return the path to local map output file created earlier
    * @param mapTaskId a map task id
    * @param mapTaskId a map task id
    */
    */
-  public Path getOutputFile(String mapTaskId)
+  public Path getOutputFile(TaskAttemptID mapTaskId)
     throws IOException {
     throws IOException {
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
                                         mapTaskId + Path.SEPARATOR +
                                         mapTaskId + Path.SEPARATOR +
@@ -56,7 +56,7 @@ class MapOutputFile {
    * @param mapTaskId a map task id
    * @param mapTaskId a map task id
    * @param size the size of the file
    * @param size the size of the file
    */
    */
-  public Path getOutputFileForWrite(String mapTaskId, long size)
+  public Path getOutputFileForWrite(TaskAttemptID mapTaskId, long size)
     throws IOException {
     throws IOException {
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
                                           mapTaskId + Path.SEPARATOR +
                                           mapTaskId + Path.SEPARATOR +
@@ -66,7 +66,7 @@ class MapOutputFile {
   /** Return the path to a local map output index file created earlier
   /** Return the path to a local map output index file created earlier
    * @param mapTaskId a map task id
    * @param mapTaskId a map task id
    */
    */
-  public Path getOutputIndexFile(String mapTaskId)
+  public Path getOutputIndexFile(TaskAttemptID mapTaskId)
     throws IOException {
     throws IOException {
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
                                         mapTaskId + Path.SEPARATOR +
                                         mapTaskId + Path.SEPARATOR +
@@ -77,7 +77,7 @@ class MapOutputFile {
    * @param mapTaskId a map task id
    * @param mapTaskId a map task id
    * @param size the size of the file
    * @param size the size of the file
    */
    */
-  public Path getOutputIndexFileForWrite(String mapTaskId, long size)
+  public Path getOutputIndexFileForWrite(TaskAttemptID mapTaskId, long size)
     throws IOException {
     throws IOException {
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
                                           mapTaskId + Path.SEPARATOR +
                                           mapTaskId + Path.SEPARATOR +
@@ -89,7 +89,7 @@ class MapOutputFile {
    * @param mapTaskId a map task id
    * @param mapTaskId a map task id
    * @param spillNumber the number
    * @param spillNumber the number
    */
    */
-  public Path getSpillFile(String mapTaskId, int spillNumber)
+  public Path getSpillFile(TaskAttemptID mapTaskId, int spillNumber)
     throws IOException {
     throws IOException {
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
                                         mapTaskId + Path.SEPARATOR +
                                         mapTaskId + Path.SEPARATOR +
@@ -102,7 +102,7 @@ class MapOutputFile {
    * @param spillNumber the number
    * @param spillNumber the number
    * @param size the size of the file
    * @param size the size of the file
    */
    */
-  public Path getSpillFileForWrite(String mapTaskId, int spillNumber, 
+  public Path getSpillFileForWrite(TaskAttemptID mapTaskId, int spillNumber, 
          long size) throws IOException {
          long size) throws IOException {
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
                                           mapTaskId + Path.SEPARATOR +
                                           mapTaskId + Path.SEPARATOR +
@@ -114,7 +114,7 @@ class MapOutputFile {
    * @param mapTaskId a map task id
    * @param mapTaskId a map task id
    * @param spillNumber the number
    * @param spillNumber the number
    */
    */
-  public Path getSpillIndexFile(String mapTaskId, int spillNumber)
+  public Path getSpillIndexFile(TaskAttemptID mapTaskId, int spillNumber)
     throws IOException {
     throws IOException {
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
                                         mapTaskId + Path.SEPARATOR +
                                         mapTaskId + Path.SEPARATOR +
@@ -127,7 +127,7 @@ class MapOutputFile {
    * @param spillNumber the number
    * @param spillNumber the number
    * @param size the size of the file
    * @param size the size of the file
    */
    */
-  public Path getSpillIndexFileForWrite(String mapTaskId, int spillNumber,
+  public Path getSpillIndexFileForWrite(TaskAttemptID mapTaskId, int spillNumber,
          long size) throws IOException {
          long size) throws IOException {
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
                                           mapTaskId + Path.SEPARATOR +
                                           mapTaskId + Path.SEPARATOR +
@@ -139,7 +139,7 @@ class MapOutputFile {
    * @param mapTaskId a map task id
    * @param mapTaskId a map task id
    * @param reduceTaskId a reduce task id
    * @param reduceTaskId a reduce task id
    */
    */
-  public Path getInputFile(int mapId, String reduceTaskId)
+  public Path getInputFile(int mapId, TaskAttemptID reduceTaskId)
     throws IOException {
     throws IOException {
     // TODO *oom* should use a format here
     // TODO *oom* should use a format here
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathToRead(jobDir + Path.SEPARATOR +
@@ -153,7 +153,7 @@ class MapOutputFile {
    * @param reduceTaskId a reduce task id
    * @param reduceTaskId a reduce task id
    * @param size the size of the file
    * @param size the size of the file
    */
    */
-  public Path getInputFileForWrite(int mapId, String reduceTaskId, long size)
+  public Path getInputFileForWrite(int mapId, TaskAttemptID reduceTaskId, long size)
     throws IOException {
     throws IOException {
     // TODO *oom* should use a format here
     // TODO *oom* should use a format here
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
     return lDirAlloc.getLocalPathForWrite(jobDir + Path.SEPARATOR +
@@ -163,7 +163,7 @@ class MapOutputFile {
   }
   }
 
 
   /** Removes all of the files related to a task. */
   /** Removes all of the files related to a task. */
-  public void removeAll(String taskId) throws IOException {
+  public void removeAll(TaskAttemptID taskId) throws IOException {
     conf.deleteLocalFiles(jobDir + Path.SEPARATOR +
     conf.deleteLocalFiles(jobDir + Path.SEPARATOR +
                           taskId + Path.SEPARATOR + "output");
                           taskId + Path.SEPARATOR + "output");
   }
   }
@@ -176,7 +176,7 @@ class MapOutputFile {
     }
     }
   }
   }
   
   
-  public void setJobId(String jobId) {
+  public void setJobId(JobID jobId) {
     this.jobDir = TaskTracker.getJobCacheSubdir() + Path.SEPARATOR + jobId;
     this.jobDir = TaskTracker.getJobCacheSubdir() + Path.SEPARATOR + jobId;
   }
   }
 
 

+ 23 - 19
src/java/org/apache/hadoop/mapred/MapOutputLocation.java

@@ -18,19 +18,25 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
+import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URL;
+import java.net.URLConnection;
 
 
-import java.io.*;
-import java.net.*;
-
-import org.apache.hadoop.fs.InMemoryFileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.InMemoryFileSystem;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
 import org.apache.hadoop.mapred.ReduceTask.ReduceCopier.ShuffleClientMetrics;
 import org.apache.hadoop.mapred.ReduceTask.ReduceCopier.ShuffleClientMetrics;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.conf.*;
 
 
 /** The location of a map output file, as passed to a reduce task via the
 /** The location of a map output file, as passed to a reduce task via the
  * {@link InterTrackerProtocol}. */ 
  * {@link InterTrackerProtocol}. */ 
@@ -44,11 +50,11 @@ class MapOutputLocation implements Writable, MRConstants {
        });
        });
   }
   }
 
 
-  private String mapTaskId;
+  private TaskAttemptID mapTaskId;
   private int mapId;
   private int mapId;
   private String host;
   private String host;
   private int port;
   private int port;
-  private String jobId;
+  
   // basic/unit connection timeout (in milliseconds)
   // basic/unit connection timeout (in milliseconds)
   private final static int UNIT_CONNECT_TIMEOUT = 30 * 1000;
   private final static int UNIT_CONNECT_TIMEOUT = 30 * 1000;
   // default read timeout (in milliseconds)
   // default read timeout (in milliseconds)
@@ -59,9 +65,8 @@ class MapOutputLocation implements Writable, MRConstants {
   }
   }
 
 
   /** Construct a location. */
   /** Construct a location. */
-  public MapOutputLocation(String jobId, String mapTaskId, int mapId, 
+  public MapOutputLocation(TaskAttemptID mapTaskId, int mapId, 
                            String host, int port) {
                            String host, int port) {
-    this.jobId = jobId;
     this.mapTaskId = mapTaskId;
     this.mapTaskId = mapTaskId;
     this.mapId = mapId;
     this.mapId = mapId;
     this.host = host;
     this.host = host;
@@ -69,7 +74,7 @@ class MapOutputLocation implements Writable, MRConstants {
   }
   }
 
 
   /** The map task id. */
   /** The map task id. */
-  public String getMapTaskId() { return mapTaskId; }
+  public TaskAttemptID getMapTaskID() { return mapTaskId; }
   
   
   /**
   /**
    * Get the map's id number.
    * Get the map's id number.
@@ -86,23 +91,22 @@ class MapOutputLocation implements Writable, MRConstants {
   public int getPort() { return port; }
   public int getPort() { return port; }
 
 
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    out.writeUTF(jobId);
-    out.writeUTF(mapTaskId);
+    mapTaskId.write(out);
     out.writeInt(mapId);
     out.writeInt(mapId);
-    out.writeUTF(host);
+    Text.writeString(out, host);
     out.writeInt(port);
     out.writeInt(port);
   }
   }
 
 
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    this.jobId = in.readUTF();
-    this.mapTaskId = in.readUTF();
+    this.mapTaskId = TaskAttemptID.read(in);
     this.mapId = in.readInt();
     this.mapId = in.readInt();
-    this.host = in.readUTF();
+    this.host = Text.readString(in);
     this.port = in.readInt();
     this.port = in.readInt();
   }
   }
 
 
+  @Override
   public String toString() {
   public String toString() {
-    return "http://" + host + ":" + port + "/mapOutput?job=" + jobId +
+    return "http://" + host + ":" + port + "/mapOutput?job=" + mapTaskId.getJobID() +
            "&map=" + mapTaskId;
            "&map=" + mapTaskId;
   }
   }
   
   

+ 33 - 21
src/java/org/apache/hadoop/mapred/MapTask.java

@@ -18,18 +18,23 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
+import static org.apache.hadoop.mapred.Task.Counter.COMBINE_INPUT_RECORDS;
+import static org.apache.hadoop.mapred.Task.Counter.COMBINE_OUTPUT_RECORDS;
+import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_BYTES;
+import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_RECORDS;
+import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_BYTES;
+import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_RECORDS;
+
 import java.io.DataInput;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -39,30 +44,26 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.SequenceFile.Sorter.RawKeyValueIterator;
-import org.apache.hadoop.io.SequenceFile.Sorter.SegmentDescriptor;
 import org.apache.hadoop.io.SequenceFile.Sorter;
 import org.apache.hadoop.io.SequenceFile.Sorter;
 import org.apache.hadoop.io.SequenceFile.ValueBytes;
 import org.apache.hadoop.io.SequenceFile.ValueBytes;
 import org.apache.hadoop.io.SequenceFile.Writer;
 import org.apache.hadoop.io.SequenceFile.Writer;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.SequenceFile.Sorter.RawKeyValueIterator;
+import org.apache.hadoop.io.SequenceFile.Sorter.SegmentDescriptor;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionOutputStream;
 import org.apache.hadoop.io.compress.CompressionOutputStream;
 import org.apache.hadoop.io.compress.Compressor;
 import org.apache.hadoop.io.compress.Compressor;
 import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.io.serializer.Serializer;
 import org.apache.hadoop.io.serializer.Serializer;
 import org.apache.hadoop.mapred.ReduceTask.ValuesIterator;
 import org.apache.hadoop.mapred.ReduceTask.ValuesIterator;
 import org.apache.hadoop.util.IndexedSortable;
 import org.apache.hadoop.util.IndexedSortable;
-import org.apache.hadoop.util.IndexedSorter;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.QuickSort;
 import org.apache.hadoop.util.QuickSort;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 
 
-import static org.apache.hadoop.mapred.Task.Counter.*;
-
 /** A Map task. */
 /** A Map task. */
 class MapTask extends Task {
 class MapTask extends Task {
 
 
@@ -81,18 +82,20 @@ class MapTask extends Task {
     super();
     super();
   }
   }
 
 
-  public MapTask(String jobId, String jobFile, String tipId, String taskId, 
+  public MapTask(String jobFile, TaskAttemptID taskId, 
                  int partition, String splitClass, BytesWritable split
                  int partition, String splitClass, BytesWritable split
                  ) throws IOException {
                  ) throws IOException {
-    super(jobId, jobFile, tipId, taskId, partition);
+    super(jobFile, taskId, partition);
     this.splitClass = splitClass;
     this.splitClass = splitClass;
     this.split.set(split);
     this.split.set(split);
   }
   }
 
 
+  @Override
   public boolean isMapTask() {
   public boolean isMapTask() {
     return true;
     return true;
   }
   }
 
 
+  @Override
   public void localizeConfiguration(JobConf conf) throws IOException {
   public void localizeConfiguration(JobConf conf) throws IOException {
     super.localizeConfiguration(conf);
     super.localizeConfiguration(conf);
     Path localSplit = new Path(new Path(getJobFile()).getParent(), 
     Path localSplit = new Path(new Path(getJobFile()).getParent(), 
@@ -104,22 +107,26 @@ class MapTask extends Task {
     out.close();
     out.close();
   }
   }
   
   
+  @Override
   public TaskRunner createRunner(TaskTracker tracker) {
   public TaskRunner createRunner(TaskTracker tracker) {
     return new MapTaskRunner(this, tracker, this.conf);
     return new MapTaskRunner(this, tracker, this.conf);
   }
   }
 
 
+  @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     super.write(out);
     super.write(out);
     Text.writeString(out, splitClass);
     Text.writeString(out, splitClass);
     split.write(out);
     split.write(out);
   }
   }
   
   
+  @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     super.readFields(in);
     splitClass = Text.readString(in);
     splitClass = Text.readString(in);
     split.readFields(in);
     split.readFields(in);
   }
   }
 
 
+  @Override
   InputSplit getInputSplit() throws UnsupportedOperationException {
   InputSplit getInputSplit() throws UnsupportedOperationException {
     return instantiatedSplit;
     return instantiatedSplit;
   }
   }
@@ -169,6 +176,7 @@ class MapTask extends Task {
     }
     }
   };
   };
 
 
+  @Override
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public void run(final JobConf job, final TaskUmbilicalProtocol umbilical)
   public void run(final JobConf job, final TaskUmbilicalProtocol umbilical)
     throws IOException {
     throws IOException {
@@ -570,6 +578,7 @@ class MapTask extends Task {
     public class Buffer extends OutputStream {
     public class Buffer extends OutputStream {
       private final byte[] scratch = new byte[1];
       private final byte[] scratch = new byte[1];
 
 
+      @Override
       public synchronized void write(int v)
       public synchronized void write(int v)
           throws IOException {
           throws IOException {
         scratch[0] = (byte)v;
         scratch[0] = (byte)v;
@@ -583,6 +592,7 @@ class MapTask extends Task {
        * @throws MapBufferTooSmallException if record is too large to
        * @throws MapBufferTooSmallException if record is too large to
        *    deserialize into the collection buffer.
        *    deserialize into the collection buffer.
        */
        */
+      @Override
       public synchronized void write(byte b[], int off, int len)
       public synchronized void write(byte b[], int off, int len)
           throws IOException {
           throws IOException {
         boolean kvfull = false;
         boolean kvfull = false;
@@ -699,6 +709,7 @@ class MapTask extends Task {
 
 
     protected class SpillThread extends Thread {
     protected class SpillThread extends Thread {
 
 
+      @Override
       public void run() {
       public void run() {
         try {
         try {
           sortAndSpill();
           sortAndSpill();
@@ -728,12 +739,12 @@ class MapTask extends Task {
       FSDataOutputStream indexOut = null;
       FSDataOutputStream indexOut = null;
       try {
       try {
         // create spill file
         // create spill file
-        Path filename = mapOutputFile.getSpillFileForWrite(getTaskId(),
+        Path filename = mapOutputFile.getSpillFileForWrite(getTaskID(),
                                       numSpills, size);
                                       numSpills, size);
         out = localFs.create(filename);
         out = localFs.create(filename);
         // create spill index
         // create spill index
         Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
         Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
-                             getTaskId(), numSpills, partitions * 16);
+                             getTaskID(), numSpills, partitions * 16);
         indexOut = localFs.create(indexFilename);
         indexOut = localFs.create(indexFilename);
         final int endPosition = (kvend > kvstart)
         final int endPosition = (kvend > kvstart)
           ? kvend
           ? kvend
@@ -811,12 +822,12 @@ class MapTask extends Task {
       final int partition = partitioner.getPartition(key, value, partitions);
       final int partition = partitioner.getPartition(key, value, partitions);
       try {
       try {
         // create spill file
         // create spill file
-        Path filename = mapOutputFile.getSpillFileForWrite(getTaskId(),
+        Path filename = mapOutputFile.getSpillFileForWrite(getTaskID(),
                                       numSpills, size);
                                       numSpills, size);
         out = localFs.create(filename);
         out = localFs.create(filename);
         // create spill index
         // create spill index
         Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
         Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
-                             getTaskId(), numSpills, partitions * 16);
+                             getTaskID(), numSpills, partitions * 16);
         indexOut = localFs.create(indexFilename);
         indexOut = localFs.create(indexFilename);
         // we don't run the combiner for a single record
         // we don't run the combiner for a single record
         for (int i = 0; i < partitions; ++i) {
         for (int i = 0; i < partitions; ++i) {
@@ -950,6 +961,7 @@ class MapTask extends Task {
         super(in, comparator, keyClass, valClass, conf, reporter);
         super(in, comparator, keyClass, valClass, conf, reporter);
       }
       }
 
 
+      @Override
       public VALUE next() {
       public VALUE next() {
         combineInputCounter.increment(1);
         combineInputCounter.increment(1);
         return super.next();
         return super.next();
@@ -965,8 +977,8 @@ class MapTask extends Task {
       FileSystem localFs = FileSystem.getLocal(job);
       FileSystem localFs = FileSystem.getLocal(job);
       
       
       for(int i = 0; i < numSpills; i++) {
       for(int i = 0; i < numSpills; i++) {
-        filename[i] = mapOutputFile.getSpillFile(getTaskId(), i);
-        indexFileName[i] = mapOutputFile.getSpillIndexFile(getTaskId(), i);
+        filename[i] = mapOutputFile.getSpillFile(getTaskID(), i);
+        indexFileName[i] = mapOutputFile.getSpillIndexFile(getTaskID(), i);
         finalOutFileSize += localFs.getLength(filename[i]);
         finalOutFileSize += localFs.getLength(filename[i]);
       }
       }
       //make correction in the length to include the sequence file header
       //make correction in the length to include the sequence file header
@@ -975,10 +987,10 @@ class MapTask extends Task {
       
       
       finalIndexFileSize = partitions * 16;
       finalIndexFileSize = partitions * 16;
       
       
-      Path finalOutputFile = mapOutputFile.getOutputFileForWrite(getTaskId(), 
+      Path finalOutputFile = mapOutputFile.getOutputFileForWrite(getTaskID(), 
                              finalOutFileSize);
                              finalOutFileSize);
       Path finalIndexFile = mapOutputFile.getOutputIndexFileForWrite(
       Path finalIndexFile = mapOutputFile.getOutputIndexFileForWrite(
-                            getTaskId(), finalIndexFileSize);
+                            getTaskID(), finalIndexFileSize);
       
       
       if (numSpills == 1) { //the spill is the final output
       if (numSpills == 1) { //the spill is the final output
         localFs.rename(filename[0], finalOutputFile);
         localFs.rename(filename[0], finalOutputFile);
@@ -1033,7 +1045,7 @@ class MapTask extends Task {
             segmentList.add(i, s);
             segmentList.add(i, s);
           }
           }
           segmentStart = finalOut.getPos();
           segmentStart = finalOut.getPos();
-          RawKeyValueIterator kvIter = sorter.merge(segmentList, new Path(getTaskId())); 
+          RawKeyValueIterator kvIter = sorter.merge(segmentList, new Path(getTaskID().toString())); 
           SequenceFile.Writer writer = SequenceFile.createWriter(job, finalOut, 
           SequenceFile.Writer writer = SequenceFile.createWriter(job, finalOut, 
                                                                  job.getMapOutputKeyClass(), job.getMapOutputValueClass(), 
                                                                  job.getMapOutputKeyClass(), job.getMapOutputValueClass(), 
                                                                  compressionType, codec);
                                                                  compressionType, codec);

+ 2 - 2
src/java/org/apache/hadoop/mapred/MapTaskRunner.java

@@ -32,13 +32,13 @@ class MapTaskRunner extends TaskRunner {
       return false;
       return false;
     }
     }
     
     
-    mapOutputFile.removeAll(getTask().getTaskId());
+    mapOutputFile.removeAll(getTask().getTaskID());
     return true;
     return true;
   }
   }
 
 
   /** Delete all of the temporary map output files. */
   /** Delete all of the temporary map output files. */
   public void close() throws IOException {
   public void close() throws IOException {
     LOG.info(getTask()+" done; removing files.");
     LOG.info(getTask()+" done; removing files.");
-    mapOutputFile.removeAll(getTask().getTaskId());
+    mapOutputFile.removeAll(getTask().getTaskID());
   }
   }
 }
 }

+ 7 - 1
src/java/org/apache/hadoop/mapred/MapTaskStatus.java

@@ -18,33 +18,39 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
+
 public class MapTaskStatus extends TaskStatus {
 public class MapTaskStatus extends TaskStatus {
 
 
   public MapTaskStatus() {}
   public MapTaskStatus() {}
 
 
-  public MapTaskStatus(String taskid, float progress,
+  public MapTaskStatus(TaskAttemptID taskid, float progress,
           State runState, String diagnosticInfo, String stateString,
           State runState, String diagnosticInfo, String stateString,
           String taskTracker, Phase phase, Counters counters) {
           String taskTracker, Phase phase, Counters counters) {
     super(taskid, progress, runState, diagnosticInfo, stateString,
     super(taskid, progress, runState, diagnosticInfo, stateString,
           taskTracker, phase, counters);
           taskTracker, phase, counters);
   }
   }
 
 
+  @Override
   public boolean getIsMap() {
   public boolean getIsMap() {
     return true;
     return true;
   }
   }
 
 
+  @Override
   public long getShuffleFinishTime() {
   public long getShuffleFinishTime() {
     throw new UnsupportedOperationException("getShuffleFinishTime() not supported for MapTask");
     throw new UnsupportedOperationException("getShuffleFinishTime() not supported for MapTask");
   }
   }
 
 
+  @Override
   void setShuffleFinishTime(long shuffleFinishTime) {
   void setShuffleFinishTime(long shuffleFinishTime) {
     throw new UnsupportedOperationException("setShuffleFinishTime() not supported for MapTask");
     throw new UnsupportedOperationException("setShuffleFinishTime() not supported for MapTask");
   }
   }
 
 
+  @Override
   public long getSortFinishTime() {
   public long getSortFinishTime() {
     throw new UnsupportedOperationException("getSortFinishTime() not supported for MapTask");
     throw new UnsupportedOperationException("getSortFinishTime() not supported for MapTask");
   }
   }
 
 
+  @Override
   void setSortFinishTime(long sortFinishTime) {
   void setSortFinishTime(long sortFinishTime) {
     throw new UnsupportedOperationException("setSortFinishTime() not supported for MapTask");
     throw new UnsupportedOperationException("setSortFinishTime() not supported for MapTask");
   }
   }

+ 75 - 64
src/java/org/apache/hadoop/mapred/ReduceTask.java

@@ -18,6 +18,10 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
+import static org.apache.hadoop.mapred.Task.Counter.REDUCE_INPUT_GROUPS;
+import static org.apache.hadoop.mapred.Task.Counter.REDUCE_INPUT_RECORDS;
+import static org.apache.hadoop.mapred.Task.Counter.REDUCE_OUTPUT_RECORDS;
+
 import java.io.DataInput;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.File;
 import java.io.File;
@@ -28,6 +32,7 @@ import java.net.URLClassLoader;
 import java.text.DecimalFormat;
 import java.text.DecimalFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Hashtable;
 import java.util.Hashtable;
@@ -38,19 +43,18 @@ import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.Random;
 import java.util.Set;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.SortedSet;
-import java.util.Comparator;
 import java.util.TreeSet;
 import java.util.TreeSet;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumFileSystem;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.InMemoryFileSystem;
 import org.apache.hadoop.fs.InMemoryFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.ChecksumFileSystem;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.InputBuffer;
 import org.apache.hadoop.io.InputBuffer;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
@@ -70,8 +74,6 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
-import static org.apache.hadoop.mapred.Task.Counter.*;
-
 /** A Reduce task. */
 /** A Reduce task. */
 class ReduceTask extends Task {
 class ReduceTask extends Task {
 
 
@@ -129,16 +131,18 @@ class ReduceTask extends Task {
     super();
     super();
   }
   }
 
 
-  public ReduceTask(String jobId, String jobFile, String tipId, String taskId,
+  public ReduceTask(String jobFile, TaskAttemptID taskId,
                     int partition, int numMaps) {
                     int partition, int numMaps) {
-    super(jobId, jobFile, tipId, taskId, partition);
+    super(jobFile, taskId, partition);
     this.numMaps = numMaps;
     this.numMaps = numMaps;
   }
   }
 
 
+  @Override
   public TaskRunner createRunner(TaskTracker tracker) throws IOException {
   public TaskRunner createRunner(TaskTracker tracker) throws IOException {
     return new ReduceTaskRunner(this, tracker, this.conf);
     return new ReduceTaskRunner(this, tracker, this.conf);
   }
   }
 
 
+  @Override
   public boolean isMapTask() {
   public boolean isMapTask() {
     return false;
     return false;
   }
   }
@@ -148,17 +152,20 @@ class ReduceTask extends Task {
   /**
   /**
    * Localize the given JobConf to be specific for this task.
    * Localize the given JobConf to be specific for this task.
    */
    */
+  @Override
   public void localizeConfiguration(JobConf conf) throws IOException {
   public void localizeConfiguration(JobConf conf) throws IOException {
     super.localizeConfiguration(conf);
     super.localizeConfiguration(conf);
     conf.setNumMapTasks(numMaps);
     conf.setNumMapTasks(numMaps);
   }
   }
 
 
+  @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     super.write(out);
     super.write(out);
 
 
     out.writeInt(numMaps);                        // write the number of maps
     out.writeInt(numMaps);                        // write the number of maps
   }
   }
 
 
+  @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     super.readFields(in);
 
 
@@ -172,7 +179,7 @@ class ReduceTask extends Task {
     if (isLocal) {
     if (isLocal) {
       // for local jobs
       // for local jobs
       for(int i = 0; i < numMaps; ++i) {
       for(int i = 0; i < numMaps; ++i) {
-        fileList.add(mapOutputFile.getInputFile(i, getTaskId()));
+        fileList.add(mapOutputFile.getInputFile(i, getTaskID()));
       }
       }
     } else {
     } else {
       // for non local jobs
       // for non local jobs
@@ -306,12 +313,14 @@ class ReduceTask extends Task {
       reducePhase.set(super.in.getProgress().get()); // update progress
       reducePhase.set(super.in.getProgress().get()); // update progress
       reporter.progress();
       reporter.progress();
     }
     }
+    @Override
     public VALUE next() {
     public VALUE next() {
       reduceInputValueCounter.increment(1);
       reduceInputValueCounter.increment(1);
       return super.next();
       return super.next();
     }
     }
   }
   }
 
 
+  @Override
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public void run(JobConf job, final TaskUmbilicalProtocol umbilical)
   public void run(JobConf job, final TaskUmbilicalProtocol umbilical)
     throws IOException {
     throws IOException {
@@ -326,7 +335,7 @@ class ReduceTask extends Task {
     if (!job.get("mapred.job.tracker", "local").equals("local")) {
     if (!job.get("mapred.job.tracker", "local").equals("local")) {
       reduceCopier = new ReduceCopier(umbilical, job);
       reduceCopier = new ReduceCopier(umbilical, job);
       if (!reduceCopier.fetchOutputs()) {
       if (!reduceCopier.fetchOutputs()) {
-        throw new IOException(getTaskId() + "The reduce copier failed");
+        throw new IOException(getTaskID() + "The reduce copier failed");
       }
       }
       isLocal = false;
       isLocal = false;
     }
     }
@@ -336,7 +345,7 @@ class ReduceTask extends Task {
     // get the input files for the reducer to merge
     // get the input files for the reducer to merge
     Path[] mapFiles = getMapFiles(lfs, isLocal);
     Path[] mapFiles = getMapFiles(lfs, isLocal);
     
     
-    Path tempDir = new Path(getTaskId()); 
+    Path tempDir = new Path(getTaskID().toString()); 
 
 
     SequenceFile.Sorter.RawKeyValueIterator rIter;
     SequenceFile.Sorter.RawKeyValueIterator rIter;
  
  
@@ -538,8 +547,8 @@ class ReduceTask extends Task {
     /** 
     /** 
      * The set of obsolete map taskids.
      * The set of obsolete map taskids.
      */
      */
-    private Set <String> obsoleteMapIds = 
-      Collections.synchronizedSet(new TreeSet<String>());
+    private Set <TaskAttemptID> obsoleteMapIds = 
+      Collections.synchronizedSet(new TreeSet<TaskAttemptID>());
     
     
     private Random random = null;
     private Random random = null;
     
     
@@ -589,8 +598,8 @@ class ReduceTask extends Task {
     /**
     /**
      * A map of taskId -> no. of failed fetches
      * A map of taskId -> no. of failed fetches
      */
      */
-    Map<String, Integer> mapTaskToFailedFetchesMap = 
-      new HashMap<String, Integer>();    
+    Map<TaskAttemptID, Integer> mapTaskToFailedFetchesMap = 
+      new HashMap<TaskAttemptID, Integer>();    
 
 
     /**
     /**
      * Initial backoff interval (milliseconds)
      * Initial backoff interval (milliseconds)
@@ -615,8 +624,8 @@ class ReduceTask extends Task {
           MetricsUtil.createRecord(metricsContext, "shuffleInput");
           MetricsUtil.createRecord(metricsContext, "shuffleInput");
         this.shuffleMetrics.setTag("user", conf.getUser());
         this.shuffleMetrics.setTag("user", conf.getUser());
         this.shuffleMetrics.setTag("jobName", conf.getJobName());
         this.shuffleMetrics.setTag("jobName", conf.getJobName());
-        this.shuffleMetrics.setTag("jobId", ReduceTask.this.getJobId());
-        this.shuffleMetrics.setTag("taskId", getTaskId());
+        this.shuffleMetrics.setTag("jobId", ReduceTask.this.getJobID().toString());
+        this.shuffleMetrics.setTag("taskId", getTaskID().toString());
         this.shuffleMetrics.setTag("sessionId", conf.getSessionId());
         this.shuffleMetrics.setTag("sessionId", conf.getSessionId());
         metricsContext.registerUpdater(this);
         metricsContext.registerUpdater(this);
       }
       }
@@ -702,7 +711,7 @@ class ReduceTask extends Task {
       private Reporter reporter;
       private Reporter reporter;
       
       
       public MapOutputCopier(Reporter reporter) {
       public MapOutputCopier(Reporter reporter) {
-        setName("MapOutputCopier " + reduceTask.getTaskId() + "." + id);
+        setName("MapOutputCopier " + reduceTask.getTaskID() + "." + id);
         LOG.debug(getName() + " created");
         LOG.debug(getName() + " created");
         this.reporter = reporter;
         this.reporter = reporter;
       }
       }
@@ -745,6 +754,7 @@ class ReduceTask extends Task {
       /** Loop forever and fetch map outputs as they become available.
       /** Loop forever and fetch map outputs as they become available.
        * The thread exits when it is interrupted by {@link ReduceTaskRunner}
        * The thread exits when it is interrupted by {@link ReduceTaskRunner}
        */
        */
+      @Override
       public void run() {
       public void run() {
         while (true) {        
         while (true) {        
           try {
           try {
@@ -764,8 +774,8 @@ class ReduceTask extends Task {
               size = copyOutput(loc);
               size = copyOutput(loc);
               shuffleClientMetrics.successFetch();
               shuffleClientMetrics.successFetch();
             } catch (IOException e) {
             } catch (IOException e) {
-              LOG.warn(reduceTask.getTaskId() + " copy failed: " +
-                       loc.getMapTaskId() + " from " + loc.getHost());
+              LOG.warn(reduceTask.getTaskID() + " copy failed: " +
+                       loc.getMapTaskID() + " from " + loc.getHost());
               LOG.warn(StringUtils.stringifyException(e));
               LOG.warn(StringUtils.stringifyException(e));
               shuffleClientMetrics.failedFetch();
               shuffleClientMetrics.failedFetch();
               
               
@@ -794,18 +804,18 @@ class ReduceTask extends Task {
                               ) throws IOException, InterruptedException {
                               ) throws IOException, InterruptedException {
         // check if we still need to copy the output from this location
         // check if we still need to copy the output from this location
         if (!neededOutputs.contains(loc.getMapId()) || 
         if (!neededOutputs.contains(loc.getMapId()) || 
-            obsoleteMapIds.contains(loc.getMapTaskId())) {
+            obsoleteMapIds.contains(loc.getMapTaskID())) {
           return CopyResult.OBSOLETE;
           return CopyResult.OBSOLETE;
         } 
         } 
  
  
-        String reduceId = reduceTask.getTaskId();
-        LOG.info(reduceId + " Copying " + loc.getMapTaskId() +
+        TaskAttemptID reduceId = reduceTask.getTaskID();
+        LOG.info(reduceId + " Copying " + loc.getMapTaskID() +
                  " output from " + loc.getHost() + ".");
                  " output from " + loc.getHost() + ".");
         // a temp filename. If this file gets created in ramfs, we're fine,
         // a temp filename. If this file gets created in ramfs, we're fine,
         // else, we will check the localFS to find a suitable final location
         // else, we will check the localFS to find a suitable final location
         // for this path
         // for this path
         Path filename = new Path("/" + TaskTracker.getJobCacheSubdir() +
         Path filename = new Path("/" + TaskTracker.getJobCacheSubdir() +
-                                 Path.SEPARATOR + getJobId() +
+                                 Path.SEPARATOR + getTaskID().getJobID() +
                                  Path.SEPARATOR + reduceId +
                                  Path.SEPARATOR + reduceId +
                                  Path.SEPARATOR + "output" + "/map_" +
                                  Path.SEPARATOR + "output" + "/map_" +
                                  loc.getMapId() + ".out");
                                  loc.getMapId() + ".out");
@@ -850,7 +860,7 @@ class ReduceTask extends Task {
                                   tmpFilename);
                                   tmpFilename);
           }
           }
           
           
-          LOG.info(reduceId + " done copying " + loc.getMapTaskId() +
+          LOG.info(reduceId + " done copying " + loc.getMapTaskID() +
                    " output from " + loc.getHost() + ".");
                    " output from " + loc.getHost() + ".");
           //Create a thread to do merges. Synchronize access/update to 
           //Create a thread to do merges. Synchronize access/update to 
           //mergeInProgress
           //mergeInProgress
@@ -955,7 +965,7 @@ class ReduceTask extends Task {
       //append a unique string in the uri for the inmem fs name
       //append a unique string in the uri for the inmem fs name
       URI uri = URI.create("ramfs://mapoutput" + reduceTask.hashCode());
       URI uri = URI.create("ramfs://mapoutput" + reduceTask.hashCode());
       inMemFileSys = (InMemoryFileSystem)FileSystem.get(uri, conf);
       inMemFileSys = (InMemoryFileSystem)FileSystem.get(uri, conf);
-      LOG.info(reduceTask.getTaskId() + " Created an InMemoryFileSystem, uri: "
+      LOG.info(reduceTask.getTaskID() + " Created an InMemoryFileSystem, uri: "
                + uri);
                + uri);
       ramfsMergeOutputSize = (long)(MAX_INMEM_FILESYS_USE * 
       ramfsMergeOutputSize = (long)(MAX_INMEM_FILESYS_USE * 
                                     inMemFileSys.getFSSize());
                                     inMemFileSys.getFSSize());
@@ -1030,7 +1040,7 @@ class ReduceTask extends Task {
         // loop until we get all required outputs
         // loop until we get all required outputs
         while (!neededOutputs.isEmpty() && mergeThrowable == null) {
         while (!neededOutputs.isEmpty() && mergeThrowable == null) {
           
           
-          LOG.info(reduceTask.getTaskId() + " Need another " 
+          LOG.info(reduceTask.getTaskID() + " Need another " 
                    + neededOutputs.size() + " map output(s) where " 
                    + neededOutputs.size() + " map output(s) where " 
                    + numInFlight + " is already in progress");
                    + numInFlight + " is already in progress");
           
           
@@ -1055,7 +1065,7 @@ class ReduceTask extends Task {
               getMapCompletionEvents(fromEventId, knownOutputs);
               getMapCompletionEvents(fromEventId, knownOutputs);
 
 
             
             
-              LOG.info(reduceTask.getTaskId() + ": " +  
+              LOG.info(reduceTask.getTaskID() + ": " +  
                      "Got " + (knownOutputs.size()-currentNumKnownMaps) + 
                      "Got " + (knownOutputs.size()-currentNumKnownMaps) + 
                      " new map-outputs & " + 
                      " new map-outputs & " + 
                      (obsoleteMapIds.size()-currentNumObsoleteMapIds) + 
                      (obsoleteMapIds.size()-currentNumObsoleteMapIds) + 
@@ -1071,7 +1081,7 @@ class ReduceTask extends Task {
             retryFetches.clear();
             retryFetches.clear();
           }
           }
           catch (IOException ie) {
           catch (IOException ie) {
-            LOG.warn(reduceTask.getTaskId() +
+            LOG.warn(reduceTask.getTaskID() +
                     " Problem locating map outputs: " +
                     " Problem locating map outputs: " +
                     StringUtils.stringifyException(ie));
                     StringUtils.stringifyException(ie));
           }
           }
@@ -1080,7 +1090,7 @@ class ReduceTask extends Task {
           int numKnown = knownOutputs.size(), numScheduled = 0;
           int numKnown = knownOutputs.size(), numScheduled = 0;
           int numSlow = 0, numDups = 0;
           int numSlow = 0, numDups = 0;
           
           
-          LOG.info(reduceTask.getTaskId() + " Got " + numKnown + 
+          LOG.info(reduceTask.getTaskID() + " Got " + numKnown + 
                    " known map output location(s); scheduling...");
                    " known map output location(s); scheduling...");
           
           
           synchronized (scheduledCopies) {
           synchronized (scheduledCopies) {
@@ -1093,10 +1103,10 @@ class ReduceTask extends Task {
             currentTime = System.currentTimeMillis();
             currentTime = System.currentTimeMillis();
             while (locIt.hasNext()) {
             while (locIt.hasNext()) {
               
               
-              MapOutputLocation loc = (MapOutputLocation)locIt.next();
+              MapOutputLocation loc = locIt.next();
               
               
               // Do not schedule fetches from OBSOLETE maps
               // Do not schedule fetches from OBSOLETE maps
-              if (obsoleteMapIds.contains(loc.getMapTaskId())) {
+              if (obsoleteMapIds.contains(loc.getMapTaskID())) {
                 locIt.remove();
                 locIt.remove();
                 continue;
                 continue;
               }
               }
@@ -1120,7 +1130,7 @@ class ReduceTask extends Task {
             }
             }
             scheduledCopies.notifyAll();
             scheduledCopies.notifyAll();
           }
           }
-          LOG.info(reduceTask.getTaskId() + " Scheduled " + numScheduled +
+          LOG.info(reduceTask.getTaskID() + " Scheduled " + numScheduled +
                    " of " + numKnown + " known outputs (" + numSlow +
                    " of " + numKnown + " known outputs (" + numSlow +
                    " slow hosts and " + numDups + " dup hosts)");
                    " slow hosts and " + numDups + " dup hosts)");
           
           
@@ -1153,7 +1163,7 @@ class ReduceTask extends Task {
           } catch (InterruptedException e) { } // IGNORE
           } catch (InterruptedException e) { } // IGNORE
           
           
           while (numInFlight > 0 && mergeThrowable == null) {
           while (numInFlight > 0 && mergeThrowable == null) {
-            LOG.debug(reduceTask.getTaskId() + " numInFlight = " + 
+            LOG.debug(reduceTask.getTaskID() + " numInFlight = " + 
                       numInFlight);
                       numInFlight);
             CopyResult cr = getCopyResult();
             CopyResult cr = getCopyResult();
             
             
@@ -1178,15 +1188,15 @@ class ReduceTask extends Task {
                 fetchFailedMaps.remove(cr.getLocation().getMapId());
                 fetchFailedMaps.remove(cr.getLocation().getMapId());
               } else if (cr.isObsolete()) {
               } else if (cr.isObsolete()) {
                 //ignore
                 //ignore
-                LOG.info(reduceTask.getTaskId() + 
+                LOG.info(reduceTask.getTaskID() + 
                          " Ignoring obsolete copy result for Map Task: " + 
                          " Ignoring obsolete copy result for Map Task: " + 
-                         cr.getLocation().getMapTaskId() + " from host: " + 
+                         cr.getLocation().getMapTaskID() + " from host: " + 
                          cr.getHost());
                          cr.getHost());
               } else {
               } else {
                 retryFetches.add(cr.getLocation());
                 retryFetches.add(cr.getLocation());
                 
                 
                 // note the failed-fetch
                 // note the failed-fetch
-                String mapTaskId = cr.getLocation().getMapTaskId();
+                TaskAttemptID mapTaskId = cr.getLocation().getMapTaskID();
                 Integer mapId = cr.getLocation().getMapId();
                 Integer mapId = cr.getLocation().getMapId();
                 
                 
                 totalFailures++;
                 totalFailures++;
@@ -1195,7 +1205,7 @@ class ReduceTask extends Task {
                 noFailedFetches = 
                 noFailedFetches = 
                   (noFailedFetches == null) ? 1 : (noFailedFetches + 1);
                   (noFailedFetches == null) ? 1 : (noFailedFetches + 1);
                 mapTaskToFailedFetchesMap.put(mapTaskId, noFailedFetches);
                 mapTaskToFailedFetchesMap.put(mapTaskId, noFailedFetches);
-                LOG.info("Task " + getTaskId() + ": Failed fetch #" + 
+                LOG.info("Task " + getTaskID() + ": Failed fetch #" + 
                          noFailedFetches + " from " + mapTaskId);
                          noFailedFetches + " from " + mapTaskId);
                 
                 
                 // did the fetch fail too many times?
                 // did the fetch fail too many times?
@@ -1254,8 +1264,8 @@ class ReduceTask extends Task {
                       && (!reducerProgressedEnough || reducerStalled)) { 
                       && (!reducerProgressedEnough || reducerStalled)) { 
                     LOG.fatal("Shuffle failed with too many fetch failures " + 
                     LOG.fatal("Shuffle failed with too many fetch failures " + 
                               "and insufficient progress!" +
                               "and insufficient progress!" +
-                              "Killing task " + getTaskId() + ".");
-                    umbilical.shuffleError(getTaskId(), 
+                              "Killing task " + getTaskID() + ".");
+                    umbilical.shuffleError(getTaskID(), 
                                            "Exceeded MAX_FAILED_UNIQUE_FETCHES;"
                                            "Exceeded MAX_FAILED_UNIQUE_FETCHES;"
                                            + " bailing-out.");
                                            + " bailing-out.");
                   }
                   }
@@ -1269,7 +1279,7 @@ class ReduceTask extends Task {
                                        * (1 << (noFailedFetches - 1)) 
                                        * (1 << (noFailedFetches - 1)) 
                                      : (this.maxBackoff * 1000 / 2);
                                      : (this.maxBackoff * 1000 / 2);
                 penaltyBox.put(cr.getHost(), currentTime + currentBackOff);
                 penaltyBox.put(cr.getHost(), currentTime + currentBackOff);
-                LOG.warn(reduceTask.getTaskId() + " adding host " +
+                LOG.warn(reduceTask.getTaskID() + " adding host " +
                          cr.getHost() + " to penalty box, next contact in " +
                          cr.getHost() + " to penalty box, next contact in " +
                          (currentBackOff/1000) + " seconds");
                          (currentBackOff/1000) + " seconds");
                 
                 
@@ -1280,7 +1290,7 @@ class ReduceTask extends Task {
                 // polling the tasktracker a few more times
                 // polling the tasktracker a few more times
                 Iterator<MapOutputLocation> locIt = knownOutputs.iterator();
                 Iterator<MapOutputLocation> locIt = knownOutputs.iterator();
                 while (locIt.hasNext()) {
                 while (locIt.hasNext()) {
-                  MapOutputLocation loc = (MapOutputLocation)locIt.next();
+                  MapOutputLocation loc = locIt.next();
                   if (cr.getHost().equals(loc.getHost())) {
                   if (cr.getHost().equals(loc.getHost())) {
                     retryFetches.add(loc);
                     retryFetches.add(loc);
                     locIt.remove();
                     locIt.remove();
@@ -1324,7 +1334,7 @@ class ReduceTask extends Task {
             while (mergeInProgress) {
             while (mergeInProgress) {
               Thread.sleep(200);
               Thread.sleep(200);
             }
             }
-            LOG.info(reduceTask.getTaskId() + 
+            LOG.info(reduceTask.getTaskID() + 
                      " Copying of all map outputs complete. " + 
                      " Copying of all map outputs complete. " + 
                      "Initiating the last merge on the remaining files in " + 
                      "Initiating the last merge on the remaining files in " + 
                      inMemFileSys.getUri());
                      inMemFileSys.getUri());
@@ -1336,7 +1346,7 @@ class ReduceTask extends Task {
             //initiate merge
             //initiate merge
             Path[] inMemClosedFiles = inMemFileSys.getFiles(MAP_OUTPUT_FILTER);
             Path[] inMemClosedFiles = inMemFileSys.getFiles(MAP_OUTPUT_FILTER);
             if (inMemClosedFiles.length == 0) {
             if (inMemClosedFiles.length == 0) {
-              LOG.info(reduceTask.getTaskId() + "Nothing to merge from " + 
+              LOG.info(reduceTask.getTaskID() + "Nothing to merge from " + 
                        inMemFileSys.getUri());
                        inMemFileSys.getUri());
               return neededOutputs.isEmpty();
               return neededOutputs.isEmpty();
             }
             }
@@ -1349,7 +1359,7 @@ class ReduceTask extends Task {
             //in the merge method)
             //in the merge method)
             int mapId = extractMapIdFromPathName(inMemClosedFiles[0]);
             int mapId = extractMapIdFromPathName(inMemClosedFiles[0]);
             Path outputPath = mapOutputFile.getInputFileForWrite(mapId, 
             Path outputPath = mapOutputFile.getInputFileForWrite(mapId, 
-                             reduceTask.getTaskId(), ramfsMergeOutputSize);
+                             reduceTask.getTaskID(), ramfsMergeOutputSize);
             SequenceFile.Writer writer = sorter.cloneFileAttributes(
             SequenceFile.Writer writer = sorter.cloneFileAttributes(
                                                                     inMemFileSys.makeQualified(inMemClosedFiles[0]), 
                                                                     inMemFileSys.makeQualified(inMemClosedFiles[0]), 
                                                                     localFileSys.makeQualified(outputPath), null);
                                                                     localFileSys.makeQualified(outputPath), null);
@@ -1358,7 +1368,7 @@ class ReduceTask extends Task {
             try {
             try {
               rIter = sorter.merge(inMemClosedFiles, true, 
               rIter = sorter.merge(inMemClosedFiles, true, 
                                    inMemClosedFiles.length, 
                                    inMemClosedFiles.length, 
-                                   new Path(reduceTask.getTaskId()));
+                                   new Path(reduceTask.getTaskID().toString()));
             } catch (Exception e) { 
             } catch (Exception e) { 
               //make sure that we delete the ondisk file that we created earlier
               //make sure that we delete the ondisk file that we created earlier
               //when we invoked cloneFileAttributes
               //when we invoked cloneFileAttributes
@@ -1368,7 +1378,7 @@ class ReduceTask extends Task {
             }
             }
             sorter.writeFile(rIter, writer);
             sorter.writeFile(rIter, writer);
             writer.close();
             writer.close();
-            LOG.info(reduceTask.getTaskId() +
+            LOG.info(reduceTask.getTaskID() +
                      " Merge of the " +inMemClosedFiles.length +
                      " Merge of the " +inMemClosedFiles.length +
                      " files in InMemoryFileSystem complete." +
                      " files in InMemoryFileSystem complete." +
                      " Local file is " + outputPath);
                      " Local file is " + outputPath);
@@ -1378,7 +1388,7 @@ class ReduceTask extends Task {
               mapOutputFilesOnDisk.add(status);
               mapOutputFilesOnDisk.add(status);
             }
             }
           } catch (Throwable t) {
           } catch (Throwable t) {
-            LOG.warn(reduceTask.getTaskId() +
+            LOG.warn(reduceTask.getTaskID() +
                      " Final merge of the inmemory files threw an exception: " + 
                      " Final merge of the inmemory files threw an exception: " + 
                      StringUtils.stringifyException(t));
                      StringUtils.stringifyException(t));
             // check if the last merge generated an error
             // check if the last merge generated an error
@@ -1452,7 +1462,7 @@ class ReduceTask extends Task {
       }
       }
       
       
       TaskCompletionEvent events[] = 
       TaskCompletionEvent events[] = 
-        umbilical.getMapCompletionEvents(reduceTask.getJobId(), 
+        umbilical.getMapCompletionEvents(reduceTask.getJobID(), 
                                          fromEventId.get(), probe_sample_size);
                                          fromEventId.get(), probe_sample_size);
       
       
       // Note the last successful poll time-stamp
       // Note the last successful poll time-stamp
@@ -1474,7 +1484,7 @@ class ReduceTask extends Task {
             URI u = URI.create(event.getTaskTrackerHttp());
             URI u = URI.create(event.getTaskTrackerHttp());
             String host = u.getHost();
             String host = u.getHost();
             int port = u.getPort();
             int port = u.getPort();
-            String taskId = event.getTaskId();
+            TaskAttemptID taskId = event.getTaskID();
             int mId = event.idWithinJob();
             int mId = event.idWithinJob();
             int duration = event.getTaskRunTime();
             int duration = event.getTaskRunTime();
             if (duration > maxMapRuntime) {
             if (duration > maxMapRuntime) {
@@ -1483,24 +1493,23 @@ class ReduceTask extends Task {
               maxFetchRetriesPerMap = 
               maxFetchRetriesPerMap = 
                   getClosestPowerOf2((maxMapRuntime / BACKOFF_INIT) + 1);
                   getClosestPowerOf2((maxMapRuntime / BACKOFF_INIT) + 1);
             }
             }
-            knownOutputs.add(new MapOutputLocation(reduceTask.getJobId(),
-                             taskId, mId, host, port));
+            knownOutputs.add(new MapOutputLocation(taskId, mId, host, port));
           }
           }
           break;
           break;
           case FAILED:
           case FAILED:
           case KILLED:
           case KILLED:
           case OBSOLETE:
           case OBSOLETE:
           {
           {
-            obsoleteMapIds.add(event.getTaskId());
+            obsoleteMapIds.add(event.getTaskID());
             LOG.info("Ignoring obsolete output of " + event.getTaskStatus() + 
             LOG.info("Ignoring obsolete output of " + event.getTaskStatus() + 
-                     " map-task: '" + event.getTaskId() + "'");
+                     " map-task: '" + event.getTaskID() + "'");
           }
           }
           break;
           break;
           case TIPFAILED:
           case TIPFAILED:
           {
           {
             neededOutputs.remove(event.idWithinJob());
             neededOutputs.remove(event.idWithinJob());
             LOG.info("Ignoring output of failed map TIP: '" +  
             LOG.info("Ignoring output of failed map TIP: '" +  
-            		 event.getTaskId() + "'");
+            		 event.getTaskID() + "'");
           }
           }
           break;
           break;
         }
         }
@@ -1522,13 +1531,14 @@ class ReduceTask extends Task {
         this.sorter = sorter;
         this.sorter = sorter;
       }
       }
 
 
+      @Override
       public void run() {
       public void run() {
         try {
         try {
           Path[] mapFiles = new Path[ioSortFactor];
           Path[] mapFiles = new Path[ioSortFactor];
           long approxOutputSize = 0;
           long approxOutputSize = 0;
           int bytesPerSum = 
           int bytesPerSum = 
             reduceTask.getConf().getInt("io.bytes.per.checksum", 512);
             reduceTask.getConf().getInt("io.bytes.per.checksum", 512);
-          LOG.info(reduceTask.getTaskId() 
+          LOG.info(reduceTask.getTaskID() 
                    + " Merging map output files on disk");
                    + " Merging map output files on disk");
           // 1. Prepare the list of files to be merged. This list is prepared
           // 1. Prepare the list of files to be merged. This list is prepared
           // using a list of map output files on disk. Currently we merge
           // using a list of map output files on disk. Currently we merge
@@ -1554,7 +1564,7 @@ class ReduceTask extends Task {
           SequenceFile.Writer writer =
           SequenceFile.Writer writer =
             sorter.cloneFileAttributes(mapFiles[0], outputPath, null);
             sorter.cloneFileAttributes(mapFiles[0], outputPath, null);
           SequenceFile.Sorter.RawKeyValueIterator iter  = null;
           SequenceFile.Sorter.RawKeyValueIterator iter  = null;
-          Path tmpDir = new Path(reduceTask.getTaskId());
+          Path tmpDir = new Path(reduceTask.getTaskID().toString());
           try {
           try {
             iter = sorter.merge(mapFiles, true, ioSortFactor, tmpDir);
             iter = sorter.merge(mapFiles, true, ioSortFactor, tmpDir);
           } catch (Exception e) {
           } catch (Exception e) {
@@ -1569,10 +1579,10 @@ class ReduceTask extends Task {
             mapOutputFilesOnDisk.add(localFileSys.getFileStatus(outputPath));
             mapOutputFilesOnDisk.add(localFileSys.getFileStatus(outputPath));
           }
           }
           
           
-          LOG.info(reduceTask.getTaskId()
+          LOG.info(reduceTask.getTaskID()
                    + " Finished merging map output files on disk.");
                    + " Finished merging map output files on disk.");
         } catch (Throwable t) {
         } catch (Throwable t) {
-          LOG.warn(reduceTask.getTaskId()
+          LOG.warn(reduceTask.getTaskID()
                    + " Merging of the local FS files threw an exception: "
                    + " Merging of the local FS files threw an exception: "
                    + StringUtils.stringifyException(t));
                    + StringUtils.stringifyException(t));
           if (mergeThrowable == null) {
           if (mergeThrowable == null) {
@@ -1595,8 +1605,9 @@ class ReduceTask extends Task {
         this.localFileSys = localFileSys;
         this.localFileSys = localFileSys;
         this.sorter = sorter;
         this.sorter = sorter;
       }
       }
+      @Override
       public void run() {
       public void run() {
-        LOG.info(reduceTask.getTaskId() + " Thread started: " + getName());
+        LOG.info(reduceTask.getTaskID() + " Thread started: " + getName());
         try {
         try {
           Path[] inMemClosedFiles;
           Path[] inMemClosedFiles;
           //initiate merge
           //initiate merge
@@ -1620,7 +1631,7 @@ class ReduceTask extends Task {
             int mapId = extractMapIdFromPathName(inMemClosedFiles[0]);
             int mapId = extractMapIdFromPathName(inMemClosedFiles[0]);
             
             
             Path outputPath = mapOutputFile.getInputFileForWrite(mapId, 
             Path outputPath = mapOutputFile.getInputFileForWrite(mapId, 
-                              reduceTask.getTaskId(), ramfsMergeOutputSize);
+                              reduceTask.getTaskID(), ramfsMergeOutputSize);
 
 
             SequenceFile.Writer writer = sorter.cloneFileAttributes(
             SequenceFile.Writer writer = sorter.cloneFileAttributes(
                                                                     inMemFileSys.makeQualified(inMemClosedFiles[0]), 
                                                                     inMemFileSys.makeQualified(inMemClosedFiles[0]), 
@@ -1628,7 +1639,7 @@ class ReduceTask extends Task {
             SequenceFile.Sorter.RawKeyValueIterator rIter;
             SequenceFile.Sorter.RawKeyValueIterator rIter;
             try {
             try {
               rIter = sorter.merge(inMemClosedFiles, true, 
               rIter = sorter.merge(inMemClosedFiles, true, 
-                                   inMemClosedFiles.length, new Path(reduceTask.getTaskId()));
+                                   inMemClosedFiles.length, new Path(reduceTask.getTaskID().toString()));
             } catch (Exception e) { 
             } catch (Exception e) { 
               //make sure that we delete the ondisk file that we created 
               //make sure that we delete the ondisk file that we created 
               //earlier when we invoked cloneFileAttributes
               //earlier when we invoked cloneFileAttributes
@@ -1638,7 +1649,7 @@ class ReduceTask extends Task {
             }
             }
             sorter.writeFile(rIter, writer);
             sorter.writeFile(rIter, writer);
             writer.close();
             writer.close();
-            LOG.info(reduceTask.getTaskId() + 
+            LOG.info(reduceTask.getTaskID() + 
                      " Merge of the " +inMemClosedFiles.length +
                      " Merge of the " +inMemClosedFiles.length +
                      " files in InMemoryFileSystem complete." +
                      " files in InMemoryFileSystem complete." +
                      " Local file is " + outputPath);
                      " Local file is " + outputPath);
@@ -1649,11 +1660,11 @@ class ReduceTask extends Task {
             }
             }
           }
           }
           else {
           else {
-            LOG.info(reduceTask.getTaskId() + " Nothing to merge from " + 
+            LOG.info(reduceTask.getTaskID() + " Nothing to merge from " + 
                      inMemFileSys.getUri());
                      inMemFileSys.getUri());
           }
           }
         } catch (Throwable t) {
         } catch (Throwable t) {
-          LOG.warn(reduceTask.getTaskId() +
+          LOG.warn(reduceTask.getTaskID() +
                    " Intermediate Merge of the inmemory files threw an exception: "
                    " Intermediate Merge of the inmemory files threw an exception: "
                    + StringUtils.stringifyException(t));
                    + StringUtils.stringifyException(t));
           ReduceCopier.this.mergeThrowable = t;
           ReduceCopier.this.mergeThrowable = t;

+ 2 - 2
src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java

@@ -35,7 +35,7 @@ class ReduceTaskRunner extends TaskRunner {
     }
     }
     
     
     // cleanup from failures
     // cleanup from failures
-    mapOutputFile.removeAll(getTask().getTaskId());
+    mapOutputFile.removeAll(getTask().getTaskID());
     return true;
     return true;
   }
   }
   
   
@@ -44,6 +44,6 @@ class ReduceTaskRunner extends TaskRunner {
   public void close() throws IOException {
   public void close() throws IOException {
     LOG.info(getTask()+" done; removing files.");
     LOG.info(getTask()+" done; removing files.");
     getTask().getProgress().setStatus("closed");
     getTask().getProgress().setStatus("closed");
-    mapOutputFile.removeAll(getTask().getTaskId());
+    mapOutputFile.removeAll(getTask().getTaskID());
   }
   }
 }
 }

+ 23 - 11
src/java/org/apache/hadoop/mapred/ReduceTaskStatus.java

@@ -24,34 +24,36 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 
 
-import org.apache.hadoop.io.Text;
 
 
 
 
 public class ReduceTaskStatus extends TaskStatus {
 public class ReduceTaskStatus extends TaskStatus {
 
 
   private long shuffleFinishTime; 
   private long shuffleFinishTime; 
   private long sortFinishTime; 
   private long sortFinishTime; 
-  private List<String> failedFetchTasks = new ArrayList<String>(1);
+  private List<TaskAttemptID> failedFetchTasks = new ArrayList<TaskAttemptID>(1);
   
   
   public ReduceTaskStatus() {}
   public ReduceTaskStatus() {}
 
 
-  public ReduceTaskStatus(String taskid, float progress, State runState,
+  public ReduceTaskStatus(TaskAttemptID taskid, float progress, State runState,
           String diagnosticInfo, String stateString, String taskTracker,
           String diagnosticInfo, String stateString, String taskTracker,
           Phase phase, Counters counters) {
           Phase phase, Counters counters) {
     super(taskid, progress, runState, diagnosticInfo, stateString, taskTracker,
     super(taskid, progress, runState, diagnosticInfo, stateString, taskTracker,
             phase, counters);
             phase, counters);
   }
   }
 
 
+  @Override
   public Object clone() {
   public Object clone() {
     ReduceTaskStatus myClone = (ReduceTaskStatus)super.clone();
     ReduceTaskStatus myClone = (ReduceTaskStatus)super.clone();
-    myClone.failedFetchTasks = new ArrayList<String>(failedFetchTasks);
+    myClone.failedFetchTasks = new ArrayList<TaskAttemptID>(failedFetchTasks);
     return myClone;
     return myClone;
   }
   }
 
 
+  @Override
   public boolean getIsMap() {
   public boolean getIsMap() {
     return false;
     return false;
   }
   }
 
 
+  @Override
   void setFinishTime(long finishTime) {
   void setFinishTime(long finishTime) {
     if (shuffleFinishTime == 0) {
     if (shuffleFinishTime == 0) {
       this.shuffleFinishTime = finishTime; 
       this.shuffleFinishTime = finishTime; 
@@ -62,18 +64,22 @@ public class ReduceTaskStatus extends TaskStatus {
     super.setFinishTime(finishTime);
     super.setFinishTime(finishTime);
   }
   }
 
 
+  @Override
   public long getShuffleFinishTime() {
   public long getShuffleFinishTime() {
     return shuffleFinishTime;
     return shuffleFinishTime;
   }
   }
 
 
+  @Override
   void setShuffleFinishTime(long shuffleFinishTime) {
   void setShuffleFinishTime(long shuffleFinishTime) {
     this.shuffleFinishTime = shuffleFinishTime;
     this.shuffleFinishTime = shuffleFinishTime;
   }
   }
 
 
+  @Override
   public long getSortFinishTime() {
   public long getSortFinishTime() {
     return sortFinishTime;
     return sortFinishTime;
   }
   }
 
 
+  @Override
   void setSortFinishTime(long sortFinishTime) {
   void setSortFinishTime(long sortFinishTime) {
     this.sortFinishTime = sortFinishTime;
     this.sortFinishTime = sortFinishTime;
     if (0 == this.shuffleFinishTime){
     if (0 == this.shuffleFinishTime){
@@ -81,14 +87,17 @@ public class ReduceTaskStatus extends TaskStatus {
     }
     }
   }
   }
 
 
-  public List<String> getFetchFailedMaps() {
+  @Override
+  public List<TaskAttemptID> getFetchFailedMaps() {
     return failedFetchTasks;
     return failedFetchTasks;
   }
   }
   
   
-  void addFetchFailedMap(String mapTaskId) {
+  @Override
+  void addFetchFailedMap(TaskAttemptID mapTaskId) {
     failedFetchTasks.add(mapTaskId);
     failedFetchTasks.add(mapTaskId);
   }
   }
   
   
+  @Override
   synchronized void statusUpdate(TaskStatus status) {
   synchronized void statusUpdate(TaskStatus status) {
     super.statusUpdate(status);
     super.statusUpdate(status);
     
     
@@ -100,7 +109,7 @@ public class ReduceTaskStatus extends TaskStatus {
       sortFinishTime = status.getSortFinishTime();
       sortFinishTime = status.getSortFinishTime();
     }
     }
     
     
-    List<String> newFetchFailedMaps = status.getFetchFailedMaps();
+    List<TaskAttemptID> newFetchFailedMaps = status.getFetchFailedMaps();
     if (failedFetchTasks == null) {
     if (failedFetchTasks == null) {
       failedFetchTasks = newFetchFailedMaps;
       failedFetchTasks = newFetchFailedMaps;
     } else if (newFetchFailedMaps != null){
     } else if (newFetchFailedMaps != null){
@@ -108,29 +117,32 @@ public class ReduceTaskStatus extends TaskStatus {
     }
     }
   }
   }
 
 
+  @Override
   synchronized void clearStatus() {
   synchronized void clearStatus() {
     super.clearStatus();
     super.clearStatus();
     failedFetchTasks.clear();
     failedFetchTasks.clear();
   }
   }
 
 
+  @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     super.readFields(in);
     shuffleFinishTime = in.readLong(); 
     shuffleFinishTime = in.readLong(); 
     sortFinishTime = in.readLong();
     sortFinishTime = in.readLong();
     int noFailedFetchTasks = in.readInt();
     int noFailedFetchTasks = in.readInt();
-    failedFetchTasks = new ArrayList<String>(noFailedFetchTasks);
+    failedFetchTasks = new ArrayList<TaskAttemptID>(noFailedFetchTasks);
     for (int i=0; i < noFailedFetchTasks; ++i) {
     for (int i=0; i < noFailedFetchTasks; ++i) {
-      failedFetchTasks.add(Text.readString(in));
+      failedFetchTasks.add(TaskAttemptID.read(in));
     }
     }
   }
   }
 
 
+  @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     super.write(out);
     super.write(out);
     out.writeLong(shuffleFinishTime);
     out.writeLong(shuffleFinishTime);
     out.writeLong(sortFinishTime);
     out.writeLong(sortFinishTime);
     out.writeInt(failedFetchTasks.size());
     out.writeInt(failedFetchTasks.size());
-    for (String taskId : failedFetchTasks) {
-      Text.writeString(out, taskId);
+    for (TaskAttemptID taskId : failedFetchTasks) {
+      taskId.write(out);
     }
     }
   }
   }
   
   

+ 13 - 2
src/java/org/apache/hadoop/mapred/RunningJob.java

@@ -18,7 +18,8 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import java.io.*;
+import java.io.IOException;
+
 
 
 /** 
 /** 
  * <code>RunningJob</code> is the user-interface to query for details on a 
  * <code>RunningJob</code> is the user-interface to query for details on a 
@@ -36,6 +37,12 @@ public interface RunningJob {
    * 
    * 
    * @return the job identifier.
    * @return the job identifier.
    */
    */
+  public JobID getID();
+  
+  /** @deprecated This method is deprecated and will be removed. Applications should 
+   * rather use {@link #getID()}.
+   */
+  @Deprecated
   public String getJobID();
   public String getJobID();
   
   
   /**
   /**
@@ -128,8 +135,12 @@ public interface RunningJob {
    *                   job failure status.  
    *                   job failure status.  
    * @throws IOException
    * @throws IOException
    */
    */
+  public void killTask(TaskAttemptID taskId, boolean shouldFail) throws IOException;
+  
+  /** @deprecated Applications should rather use {@link #killTask(TaskAttemptID, boolean)}*/
+  @Deprecated
   public void killTask(String taskId, boolean shouldFail) throws IOException;
   public void killTask(String taskId, boolean shouldFail) throws IOException;
-    
+  
   /**
   /**
    * Gets the counters for this job.
    * Gets the counters for this job.
    * 
    * 

+ 3 - 2
src/java/org/apache/hadoop/mapred/StatusHttpServer.java

@@ -290,9 +290,10 @@ public class StatusHttpServer {
       JobTracker tracker = 
       JobTracker tracker = 
         (JobTracker) getServletContext().getAttribute("job.tracker");
         (JobTracker) getServletContext().getAttribute("job.tracker");
       
       
-      String jobId = request.getParameter("jobid");
-      if(jobId == null)
+      String jobIdStr = request.getParameter("jobid");
+      if(jobIdStr == null)
         return;
         return;
+      JobID jobId = JobID.forName(jobIdStr);
       String typeStr = request.getParameter("type");
       String typeStr = request.getParameter("type");
       boolean isMap = false;
       boolean isMap = false;
       if("map".equalsIgnoreCase(typeStr)) {
       if("map".equalsIgnoreCase(typeStr)) {

+ 29 - 39
src/java/org/apache/hadoop/mapred/Task.java

@@ -22,32 +22,30 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
+import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.text.NumberFormat;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.dfs.DistributedFileSystem;
 import org.apache.hadoop.dfs.DistributedFileSystem;
+import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.fs.kfs.KosmosFileSystem;
 import org.apache.hadoop.fs.kfs.KosmosFileSystem;
 import org.apache.hadoop.fs.s3.S3FileSystem;
 import org.apache.hadoop.fs.s3.S3FileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.net.*;
 
 
 /** Base class for tasks. */
 /** Base class for tasks. */
 abstract class Task implements Writable, Configurable {
 abstract class Task implements Writable, Configurable {
@@ -98,9 +96,7 @@ abstract class Task implements Writable, Configurable {
   ////////////////////////////////////////////
   ////////////////////////////////////////////
 
 
   private String jobFile;                         // job configuration file
   private String jobFile;                         // job configuration file
-  private String taskId;                          // unique, includes job id
-  private String jobId;                           // unique jobid
-  private String tipId;
+  private TaskAttemptID taskId;                          // unique, includes job id
   private int partition;                          // id within job
   private int partition;                          // id within job
   TaskStatus taskStatus; 										      // current status of the task
   TaskStatus taskStatus; 										      // current status of the task
   private Path taskOutputPath;                    // task-specific output dir
   private Path taskOutputPath;                    // task-specific output dir
@@ -117,12 +113,10 @@ abstract class Task implements Writable, Configurable {
     taskStatus = TaskStatus.createTaskStatus(isMapTask());
     taskStatus = TaskStatus.createTaskStatus(isMapTask());
   }
   }
 
 
-  public Task(String jobId, String jobFile, String tipId, 
-              String taskId, int partition) {
+  public Task(String jobFile, TaskAttemptID taskId, int partition) {
     this.jobFile = jobFile;
     this.jobFile = jobFile;
     this.taskId = taskId;
     this.taskId = taskId;
-    this.jobId = jobId;
-    this.tipId = tipId; 
+     
     this.partition = partition;
     this.partition = partition;
     this.taskStatus = TaskStatus.createTaskStatus(isMapTask(), this.taskId, 
     this.taskStatus = TaskStatus.createTaskStatus(isMapTask(), this.taskId, 
                                                   0.0f, 
                                                   0.0f, 
@@ -132,7 +126,7 @@ abstract class Task implements Writable, Configurable {
                                                     TaskStatus.Phase.MAP : 
                                                     TaskStatus.Phase.MAP : 
                                                     TaskStatus.Phase.SHUFFLE, 
                                                     TaskStatus.Phase.SHUFFLE, 
                                                   counters);
                                                   counters);
-    this.mapOutputFile.setJobId(jobId);
+    this.mapOutputFile.setJobId(taskId.getJobID());
   }
   }
 
 
   ////////////////////////////////////////////
   ////////////////////////////////////////////
@@ -140,16 +134,15 @@ abstract class Task implements Writable, Configurable {
   ////////////////////////////////////////////
   ////////////////////////////////////////////
   public void setJobFile(String jobFile) { this.jobFile = jobFile; }
   public void setJobFile(String jobFile) { this.jobFile = jobFile; }
   public String getJobFile() { return jobFile; }
   public String getJobFile() { return jobFile; }
-  public String getTaskId() { return taskId; }
-  public String getTipId(){ return tipId; }
+  public TaskAttemptID getTaskID() { return taskId; }
   public Counters getCounters() { return counters; }
   public Counters getCounters() { return counters; }
   
   
   /**
   /**
    * Get the job name for this task.
    * Get the job name for this task.
    * @return the job name
    * @return the job name
    */
    */
-  public String getJobId() {
-    return jobId;
+  public JobID getJobID() {
+    return taskId.getJobID();
   }
   }
   
   
   /**
   /**
@@ -180,10 +173,8 @@ abstract class Task implements Writable, Configurable {
   ////////////////////////////////////////////
   ////////////////////////////////////////////
 
 
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    UTF8.writeString(out, jobFile);
-    UTF8.writeString(out, tipId); 
-    UTF8.writeString(out, taskId);
-    UTF8.writeString(out, jobId);
+    Text.writeString(out, jobFile);
+    taskId.write(out);
     out.writeInt(partition);
     out.writeInt(partition);
     if (taskOutputPath != null) {
     if (taskOutputPath != null) {
       Text.writeString(out, taskOutputPath.toString());
       Text.writeString(out, taskOutputPath.toString());
@@ -193,10 +184,8 @@ abstract class Task implements Writable, Configurable {
     taskStatus.write(out);
     taskStatus.write(out);
   }
   }
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    jobFile = UTF8.readString(in);
-    tipId = UTF8.readString(in);
-    taskId = UTF8.readString(in);
-    jobId = UTF8.readString(in);
+    jobFile = Text.readString(in);
+    taskId = TaskAttemptID.read(in);
     partition = in.readInt();
     partition = in.readInt();
     String outPath = Text.readString(in);
     String outPath = Text.readString(in);
     if (outPath.length() != 0) {
     if (outPath.length() != 0) {
@@ -205,10 +194,11 @@ abstract class Task implements Writable, Configurable {
       taskOutputPath = null;
       taskOutputPath = null;
     }
     }
     taskStatus.readFields(in);
     taskStatus.readFields(in);
-    this.mapOutputFile.setJobId(jobId); 
+    this.mapOutputFile.setJobId(taskId.getJobID()); 
   }
   }
 
 
-  public String toString() { return taskId; }
+  @Override
+  public String toString() { return taskId.toString(); }
 
 
   private Path getTaskOutputPath(JobConf conf) {
   private Path getTaskOutputPath(JobConf conf) {
     Path p = new Path(FileOutputFormat.getOutputPath(conf), 
     Path p = new Path(FileOutputFormat.getOutputPath(conf), 
@@ -226,11 +216,11 @@ abstract class Task implements Writable, Configurable {
    * Localize the given JobConf to be specific for this task.
    * Localize the given JobConf to be specific for this task.
    */
    */
   public void localizeConfiguration(JobConf conf) throws IOException {
   public void localizeConfiguration(JobConf conf) throws IOException {
-    conf.set("mapred.tip.id", tipId); 
-    conf.set("mapred.task.id", taskId);
+    conf.set("mapred.tip.id", taskId.getTaskID().toString()); 
+    conf.set("mapred.task.id", taskId.toString());
     conf.setBoolean("mapred.task.is.map", isMapTask());
     conf.setBoolean("mapred.task.is.map", isMapTask());
     conf.setInt("mapred.task.partition", partition);
     conf.setInt("mapred.task.partition", partition);
-    conf.set("mapred.job.id", jobId);
+    conf.set("mapred.job.id", taskId.getJobID().toString());
     
     
     // The task-specific output path
     // The task-specific output path
     if (FileOutputFormat.getOutputPath(conf) != null) {
     if (FileOutputFormat.getOutputPath(conf) != null) {
@@ -303,7 +293,7 @@ abstract class Task implements Writable, Configurable {
                 Thread.sleep(PROGRESS_INTERVAL);
                 Thread.sleep(PROGRESS_INTERVAL);
               } 
               } 
               catch (InterruptedException e) {
               catch (InterruptedException e) {
-                LOG.debug(getTaskId() + " Progress/ping thread exiting " +
+                LOG.debug(getTaskID() + " Progress/ping thread exiting " +
                                         "since it got interrupted");
                                         "since it got interrupted");
                 break;
                 break;
               }
               }
@@ -345,7 +335,7 @@ abstract class Task implements Writable, Configurable {
       }, "Comm thread for "+taskId);
       }, "Comm thread for "+taskId);
     thread.setDaemon(true);
     thread.setDaemon(true);
     thread.start();
     thread.start();
-    LOG.debug(getTaskId() + " Progress/ping thread started");
+    LOG.debug(getTaskID() + " Progress/ping thread started");
   }
   }
 
 
   
   
@@ -463,7 +453,7 @@ abstract class Task implements Writable, Configurable {
           taskStatus.statusUpdate(taskProgress.get(), taskProgress.toString(), 
           taskStatus.statusUpdate(taskProgress.get(), taskProgress.toString(), 
                                   counters);
                                   counters);
           try {
           try {
-            if (!umbilical.statusUpdate(getTaskId(), taskStatus)) {
+            if (!umbilical.statusUpdate(getTaskID(), taskStatus)) {
               LOG.warn("Parent died.  Exiting "+taskId);
               LOG.warn("Parent died.  Exiting "+taskId);
               System.exit(66);
               System.exit(66);
             }
             }
@@ -496,7 +486,7 @@ abstract class Task implements Writable, Configurable {
           shouldBePromoted = true;
           shouldBePromoted = true;
         }
         }
         umbilical.done(taskId, shouldBePromoted);
         umbilical.done(taskId, shouldBePromoted);
-        LOG.info("Task '" + getTaskId() + "' done.");
+        LOG.info("Task '" + getTaskID() + "' done.");
         return;
         return;
       } catch (IOException ie) {
       } catch (IOException ie) {
         LOG.warn("Failure signalling completion: " + 
         LOG.warn("Failure signalling completion: " + 
@@ -556,10 +546,10 @@ abstract class Task implements Writable, Configurable {
         // Delete the temporary task-specific output directory
         // Delete the temporary task-specific output directory
         if (!fs.delete(taskOutputPath, true)) {
         if (!fs.delete(taskOutputPath, true)) {
           LOG.info("Failed to delete the temporary output directory of task: " + 
           LOG.info("Failed to delete the temporary output directory of task: " + 
-                  getTaskId() + " - " + taskOutputPath);
+                  getTaskID() + " - " + taskOutputPath);
         }
         }
         
         
-        LOG.info("Saved output of task '" + getTaskId() + "' to " + jobOutputPath);
+        LOG.info("Saved output of task '" + getTaskID() + "' to " + jobOutputPath);
       }
       }
     }
     }
   }
   }
@@ -580,11 +570,11 @@ abstract class Task implements Writable, Configurable {
       if (!fs.rename(taskOutput, finalOutputPath)) {
       if (!fs.rename(taskOutput, finalOutputPath)) {
         if (!fs.delete(finalOutputPath, true)) {
         if (!fs.delete(finalOutputPath, true)) {
           throw new IOException("Failed to delete earlier output of task: " + 
           throw new IOException("Failed to delete earlier output of task: " + 
-                  getTaskId());
+                  getTaskID());
         }
         }
         if (!fs.rename(taskOutput, finalOutputPath)) {
         if (!fs.rename(taskOutput, finalOutputPath)) {
           throw new IOException("Failed to save output of task: " + 
           throw new IOException("Failed to save output of task: " + 
-                  getTaskId());
+                  getTaskID());
         }
         }
       }
       }
       LOG.debug("Moved " + taskOutput + " to " + finalOutputPath);
       LOG.debug("Moved " + taskOutput + " to " + finalOutputPath);

+ 29 - 6
src/java/org/apache/hadoop/mapred/TaskCompletionEvent.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.mapred;
 import java.io.DataInput;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
+
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
 
 
@@ -34,7 +35,7 @@ public class TaskCompletionEvent implements Writable{
   private int eventId; 
   private int eventId; 
   private String taskTrackerHttp;
   private String taskTrackerHttp;
   private int taskRunTime; // using int since runtime is the time difference
   private int taskRunTime; // using int since runtime is the time difference
-  private String taskId;
+  private TaskAttemptID taskId;
   Status status; 
   Status status; 
   boolean isMap = false;
   boolean isMap = false;
   private int idWithinJob;
   private int idWithinJob;
@@ -55,7 +56,7 @@ public class TaskCompletionEvent implements Writable{
    * @param taskTrackerHttp task tracker's host:port for http. 
    * @param taskTrackerHttp task tracker's host:port for http. 
    */
    */
   public TaskCompletionEvent(int eventId, 
   public TaskCompletionEvent(int eventId, 
-                             String taskId,
+                             TaskAttemptID taskId,
                              int idWithinJob,
                              int idWithinJob,
                              boolean isMap,
                              boolean isMap,
                              Status status, 
                              Status status, 
@@ -78,10 +79,21 @@ public class TaskCompletionEvent implements Writable{
   /**
   /**
    * Returns task id. 
    * Returns task id. 
    * @return task id
    * @return task id
+   * @deprecated use {@link #getTaskID()} instead.
    */
    */
+  @Deprecated
   public String getTaskId() {
   public String getTaskId() {
+    return taskId.toString();
+  }
+  
+  /**
+   * Returns task id. 
+   * @return task id
+   */
+  public TaskAttemptID getTaskID() {
     return taskId;
     return taskId;
   }
   }
+  
   /**
   /**
    * Returns enum Status.SUCESS or Status.FAILURE.
    * Returns enum Status.SUCESS or Status.FAILURE.
    * @return task tracker status
    * @return task tracker status
@@ -123,11 +135,21 @@ public class TaskCompletionEvent implements Writable{
   /**
   /**
    * Sets task id. 
    * Sets task id. 
    * @param taskId
    * @param taskId
+   * @deprecated use {@link #setTaskID(TaskAttemptID)} instead.
    */
    */
-  public void setTaskId(
-                        String taskId) {
+  @Deprecated
+  public void setTaskId(String taskId) {
+    this.taskId = TaskAttemptID.forName(taskId);
+  }
+  
+  /**
+   * Sets task id. 
+   * @param taskId
+   */
+  public void setTaskID(TaskAttemptID taskId) {
     this.taskId = taskId;
     this.taskId = taskId;
   }
   }
+  
   /**
   /**
    * Set task status. 
    * Set task status. 
    * @param status
    * @param status
@@ -145,6 +167,7 @@ public class TaskCompletionEvent implements Writable{
     this.taskTrackerHttp = taskTrackerHttp;
     this.taskTrackerHttp = taskTrackerHttp;
   }
   }
     
     
+  @Override
   public String toString(){
   public String toString(){
     StringBuffer buf = new StringBuffer(); 
     StringBuffer buf = new StringBuffer(); 
     buf.append("Task Id : "); 
     buf.append("Task Id : "); 
@@ -165,7 +188,7 @@ public class TaskCompletionEvent implements Writable{
   // Writable
   // Writable
   //////////////////////////////////////////////
   //////////////////////////////////////////////
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    WritableUtils.writeString(out, taskId); 
+    taskId.write(out); 
     WritableUtils.writeVInt(out, idWithinJob);
     WritableUtils.writeVInt(out, idWithinJob);
     out.writeBoolean(isMap);
     out.writeBoolean(isMap);
     WritableUtils.writeEnum(out, status); 
     WritableUtils.writeEnum(out, status); 
@@ -174,7 +197,7 @@ public class TaskCompletionEvent implements Writable{
   }
   }
   
   
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    this.taskId = WritableUtils.readString(in); 
+    this.taskId = TaskAttemptID.read(in); 
     this.idWithinJob = WritableUtils.readVInt(in);
     this.idWithinJob = WritableUtils.readVInt(in);
     this.isMap = in.readBoolean();
     this.isMap = in.readBoolean();
     this.status = WritableUtils.readEnum(in, Status.class);
     this.status = WritableUtils.readEnum(in, Status.class);

+ 44 - 95
src/java/org/apache/hadoop/mapred/TaskInProgress.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.mapred;
 
 
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
@@ -29,8 +28,6 @@ import java.util.TreeSet;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.mapred.JobClient.RawSplit;
 import org.apache.hadoop.mapred.JobClient.RawSplit;
 
 
 
 
@@ -53,13 +50,6 @@ class TaskInProgress {
   int maxTaskAttempts = 4;    
   int maxTaskAttempts = 4;    
   static final double SPECULATIVE_GAP = 0.2;
   static final double SPECULATIVE_GAP = 0.2;
   static final long SPECULATIVE_LAG = 60 * 1000;
   static final long SPECULATIVE_LAG = 60 * 1000;
-  static final String MAP_IDENTIFIER = "_m_";
-  static final String REDUCE_IDENTIFIER = "_r_";
-  private static NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setMinimumIntegerDigits(6);
-    idFormat.setGroupingUsed(false);
-  }
 
 
   public static final Log LOG = LogFactory.getLog("org.apache.hadoop.mapred.TaskInProgress");
   public static final Log LOG = LogFactory.getLog("org.apache.hadoop.mapred.TaskInProgress");
 
 
@@ -69,7 +59,7 @@ class TaskInProgress {
   private int numMaps;
   private int numMaps;
   private int partition;
   private int partition;
   private JobTracker jobtracker;
   private JobTracker jobtracker;
-  private String id;
+  private TaskID id;
   private JobInProgress job;
   private JobInProgress job;
 
 
   // Status of the TIP
   // Status of the TIP
@@ -84,36 +74,33 @@ class TaskInProgress {
   private int completes = 0;
   private int completes = 0;
   private boolean failed = false;
   private boolean failed = false;
   private boolean killed = false;
   private boolean killed = false;
-
-  // The 'unique' prefix for taskids of this tip
-  String taskIdPrefix;
-    
+   
   // The 'next' usable taskid of this tip
   // The 'next' usable taskid of this tip
   int nextTaskId = 0;
   int nextTaskId = 0;
     
     
   // The taskid that took this TIP to SUCCESS
   // The taskid that took this TIP to SUCCESS
-  private String successfulTaskId;
+  private TaskAttemptID successfulTaskId;
   
   
   // Map from task Id -> TaskTracker Id, contains tasks that are
   // Map from task Id -> TaskTracker Id, contains tasks that are
   // currently runnings
   // currently runnings
-  private TreeMap<String, String> activeTasks = new TreeMap<String, String>();
+  private TreeMap<TaskAttemptID, String> activeTasks = new TreeMap<TaskAttemptID, String>();
   private JobConf conf;
   private JobConf conf;
-  private Map<String,List<String>> taskDiagnosticData =
-    new TreeMap<String,List<String>>();
+  private Map<TaskAttemptID,List<String>> taskDiagnosticData =
+    new TreeMap<TaskAttemptID,List<String>>();
   /**
   /**
    * Map from taskId -> TaskStatus
    * Map from taskId -> TaskStatus
    */
    */
-  private TreeMap<String,TaskStatus> taskStatuses = 
-    new TreeMap<String,TaskStatus>();
+  private TreeMap<TaskAttemptID,TaskStatus> taskStatuses = 
+    new TreeMap<TaskAttemptID,TaskStatus>();
 
 
   // Map from taskId -> Task
   // Map from taskId -> Task
-  private Map<String, Task> tasks = new TreeMap<String, Task>();
+  private Map<TaskAttemptID, Task> tasks = new TreeMap<TaskAttemptID, Task>();
 
 
   private TreeSet<String> machinesWhereFailed = new TreeSet<String>();
   private TreeSet<String> machinesWhereFailed = new TreeSet<String>();
-  private TreeSet<String> tasksReportedClosed = new TreeSet<String>();
+  private TreeSet<TaskAttemptID> tasksReportedClosed = new TreeSet<TaskAttemptID>();
   
   
   //list of tasks to kill, <taskid> -> <shouldFail> 
   //list of tasks to kill, <taskid> -> <shouldFail> 
-  private TreeMap<String, Boolean> tasksToKill = new TreeMap<String, Boolean>();
+  private TreeMap<TaskAttemptID, Boolean> tasksToKill = new TreeMap<TaskAttemptID, Boolean>();
   
   
   private Counters counters = new Counters();
   private Counters counters = new Counters();
   
   
@@ -121,7 +108,7 @@ class TaskInProgress {
   /**
   /**
    * Constructor for MapTask
    * Constructor for MapTask
    */
    */
-  public TaskInProgress(String jobid, String jobFile, 
+  public TaskInProgress(JobID jobid, String jobFile, 
                         RawSplit rawSplit, 
                         RawSplit rawSplit, 
                         JobTracker jobtracker, JobConf conf, 
                         JobTracker jobtracker, JobConf conf, 
                         JobInProgress job, int partition) {
                         JobInProgress job, int partition) {
@@ -132,13 +119,13 @@ class TaskInProgress {
     this.conf = conf;
     this.conf = conf;
     this.partition = partition;
     this.partition = partition;
     setMaxTaskAttempts();
     setMaxTaskAttempts();
-    init(JobTracker.getJobUniqueString(jobid));
+    init(jobid);
   }
   }
         
         
   /**
   /**
    * Constructor for ReduceTask
    * Constructor for ReduceTask
    */
    */
-  public TaskInProgress(String jobid, String jobFile, 
+  public TaskInProgress(JobID jobid, String jobFile, 
                         int numMaps, 
                         int numMaps, 
                         int partition, JobTracker jobtracker, JobConf conf,
                         int partition, JobTracker jobtracker, JobConf conf,
                         JobInProgress job) {
                         JobInProgress job) {
@@ -149,7 +136,7 @@ class TaskInProgress {
     this.job = job;
     this.job = job;
     this.conf = conf;
     this.conf = conf;
     setMaxTaskAttempts();
     setMaxTaskAttempts();
-    init(JobTracker.getJobUniqueString(jobid));
+    init(jobid);
   }
   }
   
   
   /**
   /**
@@ -162,39 +149,10 @@ class TaskInProgress {
       this.maxTaskAttempts = conf.getMaxReduceAttempts();
       this.maxTaskAttempts = conf.getMaxReduceAttempts();
     }
     }
   }
   }
-  
-  /**
-   * Return true if the tip id represents a map
-   * @param tipId the tip id
-   * @return whether the tip is a map tip or a reduce tip
-   */
-  public static boolean isMapId(String tipId) {
-    if (tipId.contains(MAP_IDENTIFIER))  {
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Make a unique name for this TIP.
-   * @param uniqueBase The unique name of the job
-   * @return The unique string for this tip
-   */
-  private String makeUniqueString(String uniqueBase) {
-    StringBuilder result = new StringBuilder();
-    result.append(uniqueBase);
-    if (isMapTask()) {
-      result.append(MAP_IDENTIFIER);
-    } else {
-      result.append(REDUCE_IDENTIFIER);
-    }
-    result.append(idFormat.format(partition));
-    return result.toString();
-  }
     
     
   /**
   /**
    * Return the index of the tip within the job, so 
    * Return the index of the tip within the job, so 
-   * "tip_200707121733_1313_0002_m_012345" would return 12345;
+   * "task_200707121733_1313_0002_m_012345" would return 12345;
    * @return int the tip index
    * @return int the tip index
    */
    */
   public int idWithinJob() {
   public int idWithinJob() {
@@ -213,10 +171,9 @@ class TaskInProgress {
   /**
   /**
    * Initialization common to Map and Reduce
    * Initialization common to Map and Reduce
    */
    */
-  void init(String jobUniqueString) {
+  void init(JobID jobId) {
     this.startTime = System.currentTimeMillis();
     this.startTime = System.currentTimeMillis();
-    this.taskIdPrefix = makeUniqueString(jobUniqueString);
-    this.id = "tip_" + this.taskIdPrefix;
+    this.id = new TaskID(jobId, isMapTask(), partition);
   }
   }
 
 
   ////////////////////////////////////
   ////////////////////////////////////
@@ -232,7 +189,7 @@ class TaskInProgress {
   /**
   /**
    * Return an ID for this task, not its component taskid-threads
    * Return an ID for this task, not its component taskid-threads
    */
    */
-  public String getTIPId() {
+  public TaskID getTIPId() {
     return this.id;
     return this.id;
   }
   }
   /**
   /**
@@ -247,7 +204,7 @@ class TaskInProgress {
    * @param taskId
    * @param taskId
    * @return
    * @return
    */  
    */  
-  public Task getTaskObject(String taskId) {
+  public Task getTask(TaskAttemptID taskId) {
     return tasks.get(taskId);
     return tasks.get(taskId);
   }
   }
   
   
@@ -259,16 +216,16 @@ class TaskInProgress {
     return !activeTasks.isEmpty();
     return !activeTasks.isEmpty();
   }
   }
     
     
-  private String getSuccessfulTaskid() {
+  private TaskAttemptID getSuccessfulTaskid() {
     return successfulTaskId;
     return successfulTaskId;
   }
   }
   
   
-  private void setSuccessfulTaskid(String successfulTaskId) {
+  private void setSuccessfulTaskid(TaskAttemptID successfulTaskId) {
     this.successfulTaskId = successfulTaskId; 
     this.successfulTaskId = successfulTaskId; 
   }
   }
   
   
   private void resetSuccessfulTaskid() {
   private void resetSuccessfulTaskid() {
-    this.successfulTaskId = ""; 
+    this.successfulTaskId = null; 
   }
   }
   
   
   /**
   /**
@@ -286,9 +243,9 @@ class TaskInProgress {
    * @param taskid taskid of attempt to check for completion
    * @param taskid taskid of attempt to check for completion
    * @return <code>true</code> if taskid is complete, else <code>false</code>
    * @return <code>true</code> if taskid is complete, else <code>false</code>
    */
    */
-  public boolean isComplete(String taskid) {
-    return ((completes > 0) && 
-             getSuccessfulTaskid().equals(taskid));
+  public boolean isComplete(TaskAttemptID taskid) {
+    return ((completes > 0) 
+            && taskid.equals(getSuccessfulTaskid()));
   }
   }
 
 
   /**
   /**
@@ -327,12 +284,13 @@ class TaskInProgress {
   public Counters getCounters() {
   public Counters getCounters() {
     return counters;
     return counters;
   }
   }
+
   /**
   /**
    * Returns whether a component task-thread should be 
    * Returns whether a component task-thread should be 
    * closed because the containing JobInProgress has completed
    * closed because the containing JobInProgress has completed
    * or the task is killed by the user
    * or the task is killed by the user
    */
    */
-  public boolean shouldClose(String taskid) {
+  public boolean shouldClose(TaskAttemptID taskid) {
     /**
     /**
      * If the task hasn't been closed yet, and it belongs to a completed
      * If the task hasn't been closed yet, and it belongs to a completed
      * TaskInProgress close it.
      * TaskInProgress close it.
@@ -381,7 +339,7 @@ class TaskInProgress {
    * @param taskId the id of the required task
    * @param taskId the id of the required task
    * @return the list of diagnostics for that task
    * @return the list of diagnostics for that task
    */
    */
-  synchronized List<String> getDiagnosticInfo(String taskId) {
+  synchronized List<String> getDiagnosticInfo(TaskAttemptID taskId) {
     return taskDiagnosticData.get(taskId);
     return taskDiagnosticData.get(taskId);
   }
   }
     
     
@@ -396,7 +354,7 @@ class TaskInProgress {
    * @param taskId id of the task 
    * @param taskId id of the task 
    * @param diagInfo diagnostic information for the task
    * @param diagInfo diagnostic information for the task
    */
    */
-  public void addDiagnosticInfo(String taskId, String diagInfo) {
+  public void addDiagnosticInfo(TaskAttemptID taskId, String diagInfo) {
     List<String> diagHistory = taskDiagnosticData.get(taskId);
     List<String> diagHistory = taskDiagnosticData.get(taskId);
     if (diagHistory == null) {
     if (diagHistory == null) {
       diagHistory = new ArrayList<String>();
       diagHistory = new ArrayList<String>();
@@ -412,7 +370,7 @@ class TaskInProgress {
    * @return has the task changed its state noticably?
    * @return has the task changed its state noticably?
    */
    */
   synchronized boolean updateStatus(TaskStatus status) {
   synchronized boolean updateStatus(TaskStatus status) {
-    String taskid = status.getTaskId();
+    TaskAttemptID taskid = status.getTaskID();
     String diagInfo = status.getDiagnosticInfo();
     String diagInfo = status.getDiagnosticInfo();
     TaskStatus oldStatus = taskStatuses.get(taskid);
     TaskStatus oldStatus = taskStatuses.get(taskid);
     boolean changed = true;
     boolean changed = true;
@@ -461,7 +419,7 @@ class TaskInProgress {
    * Indicate that one of the taskids in this TaskInProgress
    * Indicate that one of the taskids in this TaskInProgress
    * has failed.
    * has failed.
    */
    */
-  public void incompleteSubTask(String taskid, String trackerName, 
+  public void incompleteSubTask(TaskAttemptID taskid, String trackerName, 
                                 JobStatus jobStatus) {
                                 JobStatus jobStatus) {
     //
     //
     // Note the failure and its location
     // Note the failure and its location
@@ -529,7 +487,7 @@ class TaskInProgress {
    * @param taskId id of the completed task-attempt
    * @param taskId id of the completed task-attempt
    * @param finalTaskState final {@link TaskStatus.State} of the task-attempt
    * @param finalTaskState final {@link TaskStatus.State} of the task-attempt
    */
    */
-  private void completedTask(String taskId, TaskStatus.State finalTaskState) {
+  private void completedTask(TaskAttemptID taskId, TaskStatus.State finalTaskState) {
     TaskStatus status = taskStatuses.get(taskId);
     TaskStatus status = taskStatuses.get(taskId);
     status.setRunState(finalTaskState);
     status.setRunState(finalTaskState);
     activeTasks.remove(taskId);
     activeTasks.remove(taskId);
@@ -540,7 +498,7 @@ class TaskInProgress {
    * TaskInProgress has successfully completed; hence we mark this
    * TaskInProgress has successfully completed; hence we mark this
    * taskid as {@link TaskStatus.State.KILLED}. 
    * taskid as {@link TaskStatus.State.KILLED}. 
    */
    */
-  void alreadyCompletedTask(String taskid) {
+  void alreadyCompletedTask(TaskAttemptID taskid) {
     // 'KILL' the task 
     // 'KILL' the task 
     completedTask(taskid, TaskStatus.State.KILLED);
     completedTask(taskid, TaskStatus.State.KILLED);
     
     
@@ -555,7 +513,7 @@ class TaskInProgress {
    * Indicate that one of the taskids in this TaskInProgress
    * Indicate that one of the taskids in this TaskInProgress
    * has successfully completed!
    * has successfully completed!
    */
    */
-  public void completed(String taskid) {
+  public void completed(TaskAttemptID taskid) {
     //
     //
     // Record that this taskid is complete
     // Record that this taskid is complete
     //
     //
@@ -594,7 +552,7 @@ class TaskInProgress {
    * @param taskid
    * @param taskid
    * @return
    * @return
    */
    */
-  public TaskStatus getTaskStatus(String taskid) {
+  public TaskStatus getTaskStatus(TaskAttemptID taskid) {
     return taskStatuses.get(taskid);
     return taskStatuses.get(taskid);
   }
   }
   /**
   /**
@@ -620,7 +578,7 @@ class TaskInProgress {
   /**
   /**
    * Kill the given task
    * Kill the given task
    */
    */
-  boolean killTask(String taskId, boolean shouldFail) {
+  boolean killTask(TaskAttemptID taskId, boolean shouldFail) {
     TaskStatus st = taskStatuses.get(taskId);
     TaskStatus st = taskStatuses.get(taskId);
     if(st != null && (st.getRunState() == TaskStatus.State.RUNNING
     if(st != null && (st.getRunState() == TaskStatus.State.RUNNING
         || st.getRunState() == TaskStatus.State.COMMIT_PENDING)
         || st.getRunState() == TaskStatus.State.COMMIT_PENDING)
@@ -651,8 +609,8 @@ class TaskInProgress {
       double bestProgress = 0;
       double bestProgress = 0;
       String bestState = "";
       String bestState = "";
       Counters bestCounters = new Counters();
       Counters bestCounters = new Counters();
-      for (Iterator<String> it = taskStatuses.keySet().iterator(); it.hasNext();) {
-        String taskid = it.next();
+      for (Iterator<TaskAttemptID> it = taskStatuses.keySet().iterator(); it.hasNext();) {
+        TaskAttemptID taskid = it.next();
         TaskStatus status = taskStatuses.get(taskid);
         TaskStatus status = taskStatuses.get(taskid);
         if (status.getRunState() == TaskStatus.State.SUCCEEDED) {
         if (status.getRunState() == TaskStatus.State.SUCCEEDED) {
           bestProgress = 1;
           bestProgress = 1;
@@ -723,9 +681,9 @@ class TaskInProgress {
     }
     }
 
 
     // Create the 'taskid'; do not count the 'killed' tasks against the job!
     // Create the 'taskid'; do not count the 'killed' tasks against the job!
-    String taskid = null;
+    TaskAttemptID taskid = null;
     if (nextTaskId < (MAX_TASK_EXECS + maxTaskAttempts + numKilledTasks)) {
     if (nextTaskId < (MAX_TASK_EXECS + maxTaskAttempts + numKilledTasks)) {
-      taskid = "task_" + taskIdPrefix + "_" + nextTaskId;
+      taskid = new TaskAttemptID( id, nextTaskId);
       ++nextTaskId;
       ++nextTaskId;
     } else {
     } else {
       LOG.warn("Exceeded limit of " + (MAX_TASK_EXECS + maxTaskAttempts) +
       LOG.warn("Exceeded limit of " + (MAX_TASK_EXECS + maxTaskAttempts) +
@@ -733,14 +691,12 @@ class TaskInProgress {
               " attempts for the tip '" + getTIPId() + "'");
               " attempts for the tip '" + getTIPId() + "'");
       return null;
       return null;
     }
     }
-        
-    String jobId = job.getProfile().getJobId();
 
 
     if (isMapTask()) {
     if (isMapTask()) {
-      t = new MapTask(jobId, jobFile, this.id, taskid, partition, 
-                      rawSplit.getClassName(), rawSplit.getBytes());
+      t = new MapTask(jobFile, taskid, partition, 
+          rawSplit.getClassName(), rawSplit.getBytes());
     } else {
     } else {
-      t = new ReduceTask(jobId, jobFile, this.id, taskid, partition, numMaps);
+      t = new ReduceTask(jobFile, taskid, partition, numMaps);
     }
     }
     t.setConf(conf);
     t.setConf(conf);
     tasks.put(taskid, t);
     tasks.put(taskid, t);
@@ -799,11 +755,4 @@ class TaskInProgress {
   public int getSuccessEventNumber() {
   public int getSuccessEventNumber() {
     return successEventNumber;
     return successEventNumber;
   }
   }
-  
-  /**
-   * Gets the tip id for the given taskid
-   * */
-  public static String getTipId(String taskId){
-	  return taskId.substring(0, taskId.lastIndexOf('_')).replace("task", "tip");
-  }
 }
 }

+ 15 - 5
src/java/org/apache/hadoop/mapred/TaskLog.java

@@ -18,11 +18,16 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import java.io.*;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 
 
-import org.apache.commons.logging.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 
 
 /**
 /**
@@ -44,8 +49,8 @@ public class TaskLog {
     }
     }
   }
   }
 
 
-  public static File getTaskLogFile(String taskid, LogName filter) {
-    return new File(new File(LOG_DIR, taskid), filter.toString());
+  public static File getTaskLogFile(TaskAttemptID taskid, LogName filter) {
+    return new File(new File(LOG_DIR, taskid.toString()), filter.toString());
   }
   }
   
   
   /**
   /**
@@ -73,6 +78,7 @@ public class TaskLog {
       this.prefix = prefix;
       this.prefix = prefix;
     }
     }
     
     
+    @Override
     public String toString() {
     public String toString() {
       return prefix;
       return prefix;
     }
     }
@@ -124,7 +130,7 @@ public class TaskLog {
      * @param end the offset to read upto (negative is relative to tail)
      * @param end the offset to read upto (negative is relative to tail)
      * @throws IOException
      * @throws IOException
      */
      */
-    public Reader(String taskid, LogName kind, 
+    public Reader(TaskAttemptID taskid, LogName kind, 
                   long start, long end) throws IOException {
                   long start, long end) throws IOException {
       // find the right log file
       // find the right log file
       File filename = getTaskLogFile(taskid, kind);
       File filename = getTaskLogFile(taskid, kind);
@@ -152,6 +158,7 @@ public class TaskLog {
       }
       }
     }
     }
     
     
+    @Override
     public int read() throws IOException {
     public int read() throws IOException {
       int result = -1;
       int result = -1;
       if (bytesRemaining > 0) {
       if (bytesRemaining > 0) {
@@ -161,6 +168,7 @@ public class TaskLog {
       return result;
       return result;
     }
     }
     
     
+    @Override
     public int read(byte[] buffer, int offset, int length) throws IOException {
     public int read(byte[] buffer, int offset, int length) throws IOException {
       length = (int) Math.min(length, bytesRemaining);
       length = (int) Math.min(length, bytesRemaining);
       int bytes = file.read(buffer, offset, length);
       int bytes = file.read(buffer, offset, length);
@@ -170,10 +178,12 @@ public class TaskLog {
       return bytes;
       return bytes;
     }
     }
     
     
+    @Override
     public int available() throws IOException {
     public int available() throws IOException {
       return (int) Math.min(bytesRemaining, file.available());
       return (int) Math.min(bytesRemaining, file.available());
     }
     }
 
 
+    @Override
     public void close() throws IOException {
     public void close() throws IOException {
       file.close();
       file.close();
     }
     }

+ 3 - 2
src/java/org/apache/hadoop/mapred/TaskLogAppender.java

@@ -30,7 +30,8 @@ import org.apache.log4j.spi.LoggingEvent;
  * 
  * 
  */
  */
 public class TaskLogAppender extends FileAppender {
 public class TaskLogAppender extends FileAppender {
-  private String taskId;
+  private String taskId; //taskId should be managed as String rather than TaskID object
+  //so that log4j can configure it from the configuration(log4j.properties). 
   private int maxEvents;
   private int maxEvents;
   private Queue<LoggingEvent> tail = null;
   private Queue<LoggingEvent> tail = null;
 
 
@@ -40,7 +41,7 @@ public class TaskLogAppender extends FileAppender {
       if (maxEvents > 0) {
       if (maxEvents > 0) {
         tail = new LinkedList<LoggingEvent>();
         tail = new LinkedList<LoggingEvent>();
       }
       }
-      setFile(TaskLog.getTaskLogFile(taskId, 
+      setFile(TaskLog.getTaskLogFile(TaskAttemptID.forName(taskId), 
                                      TaskLog.LogName.SYSLOG).toString());
                                      TaskLog.LogName.SYSLOG).toString());
       setAppend(true);
       setAppend(true);
       super.activateOptions();
       super.activateOptions();

+ 11 - 6
src/java/org/apache/hadoop/mapred/TaskLogServlet.java

@@ -18,12 +18,14 @@
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
 import java.io.File;
 import java.io.File;
-import java.io.InputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
 
 
 import javax.servlet.ServletException;
 import javax.servlet.ServletException;
-import javax.servlet.http.*;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
 
 
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
@@ -31,8 +33,9 @@ import org.apache.hadoop.util.StringUtils;
  * A servlet that is run by the TaskTrackers to provide the task logs via http.
  * A servlet that is run by the TaskTrackers to provide the task logs via http.
  */
  */
 public class TaskLogServlet extends HttpServlet {
 public class TaskLogServlet extends HttpServlet {
+  private static final long serialVersionUID = -6615764817774487321L;
   
   
-  private boolean haveTaskLog(String taskId, TaskLog.LogName type) {
+  private boolean haveTaskLog(TaskAttemptID taskId, TaskLog.LogName type) {
     File f = TaskLog.getTaskLogFile(taskId, type);
     File f = TaskLog.getTaskLogFile(taskId, type);
     return f.canRead();
     return f.canRead();
   }
   }
@@ -86,7 +89,7 @@ public class TaskLogServlet extends HttpServlet {
   }
   }
 
 
   private void printTaskLog(HttpServletResponse response,
   private void printTaskLog(HttpServletResponse response,
-                            OutputStream out, String taskId, 
+                            OutputStream out, TaskAttemptID taskId, 
                             long start, long end, boolean plainText, 
                             long start, long end, boolean plainText, 
                             TaskLog.LogName filter) throws IOException {
                             TaskLog.LogName filter) throws IOException {
     if (!plainText) {
     if (!plainText) {
@@ -135,6 +138,7 @@ public class TaskLogServlet extends HttpServlet {
   /**
   /**
    * Get the logs via http.
    * Get the logs via http.
    */
    */
+  @Override
   public void doGet(HttpServletRequest request, 
   public void doGet(HttpServletRequest request, 
                     HttpServletResponse response
                     HttpServletResponse response
                     ) throws ServletException, IOException {
                     ) throws ServletException, IOException {
@@ -143,12 +147,13 @@ public class TaskLogServlet extends HttpServlet {
     boolean plainText = false;
     boolean plainText = false;
     TaskLog.LogName filter = null;
     TaskLog.LogName filter = null;
 
 
-    String taskId = request.getParameter("taskid");
-    if (taskId == null) {
+    String taskIdStr = request.getParameter("taskid");
+    if (taskIdStr == null) {
       response.sendError(HttpServletResponse.SC_BAD_REQUEST, 
       response.sendError(HttpServletResponse.SC_BAD_REQUEST, 
                          "Argument taskid is required");
                          "Argument taskid is required");
       return;
       return;
     }
     }
+    TaskAttemptID taskId = TaskAttemptID.forName(taskIdStr);
     String logFilter = request.getParameter("filter");
     String logFilter = request.getParameter("filter");
     if (logFilter != null) {
     if (logFilter != null) {
       try {
       try {

+ 8 - 5
src/java/org/apache/hadoop/mapred/TaskReport.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.io.WritableUtils;
 
 
 /** A report on the state of a task. */
 /** A report on the state of a task. */
 public class TaskReport implements Writable {
 public class TaskReport implements Writable {
-  private String taskid;
+  private TaskID taskid;
   private float progress;
   private float progress;
   private String state;
   private String state;
   private String[] diagnostics;
   private String[] diagnostics;
@@ -37,7 +37,7 @@ public class TaskReport implements Writable {
 
 
   public TaskReport() {}
   public TaskReport() {}
 
 
-  TaskReport(String taskid, float progress, String state,
+  TaskReport(TaskID taskid, float progress, String state,
              String[] diagnostics, long startTime, long finishTime,
              String[] diagnostics, long startTime, long finishTime,
              Counters counters) {
              Counters counters) {
     this.taskid = taskid;
     this.taskid = taskid;
@@ -49,8 +49,11 @@ public class TaskReport implements Writable {
     this.counters = counters;
     this.counters = counters;
   }
   }
     
     
+  /** @deprecated use {@link #getTaskID()} instead */
+  @Deprecated
+  public String getTaskId() { return taskid.toString(); }
   /** The id of the task. */
   /** The id of the task. */
-  public String getTaskId() { return taskid; }
+  public TaskID getTaskID() { return taskid; }
   /** The amount completed, between zero and one. */
   /** The amount completed, between zero and one. */
   public float getProgress() { return progress; }
   public float getProgress() { return progress; }
   /** The most recent state, reported by a {@link Reporter}. */
   /** The most recent state, reported by a {@link Reporter}. */
@@ -94,7 +97,7 @@ public class TaskReport implements Writable {
   // Writable
   // Writable
   //////////////////////////////////////////////
   //////////////////////////////////////////////
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    Text.writeString(out, taskid);
+    taskid.write(out);
     out.writeFloat(progress);
     out.writeFloat(progress);
     Text.writeString(out, state);
     Text.writeString(out, state);
     out.writeLong(startTime);
     out.writeLong(startTime);
@@ -104,7 +107,7 @@ public class TaskReport implements Writable {
   }
   }
 
 
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    this.taskid = Text.readString(in);
+    this.taskid = TaskID.read(in);
     this.progress = in.readFloat();
     this.progress = in.readFloat();
     this.state = Text.readString(in);
     this.state = Text.readString(in);
     this.startTime = in.readLong(); 
     this.startTime = in.readLong(); 

+ 14 - 13
src/java/org/apache/hadoop/mapred/TaskRunner.java

@@ -56,7 +56,7 @@ abstract class TaskRunner extends Thread {
     this.t = t;
     this.t = t;
     this.tracker = tracker;
     this.tracker = tracker;
     this.conf = conf;
     this.conf = conf;
-    this.mapOutputFile = new MapOutputFile(t.getJobId());
+    this.mapOutputFile = new MapOutputFile(t.getJobID());
     this.mapOutputFile.setConf(conf);
     this.mapOutputFile.setConf(conf);
   }
   }
 
 
@@ -88,12 +88,13 @@ abstract class TaskRunner extends Thread {
     return str.toString();
     return str.toString();
   }
   }
   
   
+  @Override
   public final void run() {
   public final void run() {
     try {
     try {
       
       
       //before preparing the job localize 
       //before preparing the job localize 
       //all the archives
       //all the archives
-      String taskid = t.getTaskId();
+      TaskAttemptID taskid = t.getTaskID();
       LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
       LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
       File jobCacheDir = null;
       File jobCacheDir = null;
       if (conf.getJar() != null) {
       if (conf.getJar() != null) {
@@ -102,8 +103,8 @@ abstract class TaskRunner extends Thread {
       }
       }
       File workDir = new File(lDirAlloc.getLocalPathToRead(
       File workDir = new File(lDirAlloc.getLocalPathToRead(
                                 TaskTracker.getJobCacheSubdir() 
                                 TaskTracker.getJobCacheSubdir() 
-                                + Path.SEPARATOR + t.getJobId() 
-                                + Path.SEPARATOR + t.getTaskId()
+                                + Path.SEPARATOR + t.getJobID() 
+                                + Path.SEPARATOR + t.getTaskID()
                                 + Path.SEPARATOR + "work",
                                 + Path.SEPARATOR + "work",
                                 conf). toString());
                                 conf). toString());
 
 
@@ -293,7 +294,7 @@ abstract class TaskRunner extends Thread {
       //  </property>
       //  </property>
       //
       //
       String javaOpts = conf.get("mapred.child.java.opts", "-Xmx200m");
       String javaOpts = conf.get("mapred.child.java.opts", "-Xmx200m");
-      javaOpts = replaceAll(javaOpts, "@taskid@", taskid);
+      javaOpts = replaceAll(javaOpts, "@taskid@", taskid.toString());
       String [] javaOptsSplit = javaOpts.split(" ");
       String [] javaOptsSplit = javaOpts.split(" ");
       
       
       // Add java.library.path; necessary for loading native libraries.
       // Add java.library.path; necessary for loading native libraries.
@@ -368,7 +369,7 @@ abstract class TaskRunner extends Thread {
       InetSocketAddress address = tracker.getTaskTrackerReportAddress();
       InetSocketAddress address = tracker.getTaskTrackerReportAddress();
       vargs.add(address.getAddress().getHostAddress()); 
       vargs.add(address.getAddress().getHostAddress()); 
       vargs.add(Integer.toString(address.getPort())); 
       vargs.add(Integer.toString(address.getPort())); 
-      vargs.add(taskid);                      // pass task identifier
+      vargs.add(taskid.toString());                      // pass task identifier
 
 
       // set memory limit using ulimit if feasible and necessary ...
       // set memory limit using ulimit if feasible and necessary ...
       String[] ulimitCmd = Shell.getUlimitMemoryCommand(conf);
       String[] ulimitCmd = Shell.getUlimitMemoryCommand(conf);
@@ -393,18 +394,18 @@ abstract class TaskRunner extends Thread {
     } catch (FSError e) {
     } catch (FSError e) {
       LOG.fatal("FSError", e);
       LOG.fatal("FSError", e);
       try {
       try {
-        tracker.fsError(t.getTaskId(), e.getMessage());
+        tracker.fsError(t.getTaskID(), e.getMessage());
       } catch (IOException ie) {
       } catch (IOException ie) {
-        LOG.fatal(t.getTaskId()+" reporting FSError", ie);
+        LOG.fatal(t.getTaskID()+" reporting FSError", ie);
       }
       }
     } catch (Throwable throwable) {
     } catch (Throwable throwable) {
-      LOG.warn(t.getTaskId()+" Child Error", throwable);
+      LOG.warn(t.getTaskID()+" Child Error", throwable);
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
       throwable.printStackTrace(new PrintStream(baos));
       throwable.printStackTrace(new PrintStream(baos));
       try {
       try {
-        tracker.reportDiagnosticInfo(t.getTaskId(), baos.toString());
+        tracker.reportDiagnosticInfo(t.getTaskID(), baos.toString());
       } catch (IOException e) {
       } catch (IOException e) {
-        LOG.warn(t.getTaskId()+" Reporting Diagnostics", e);
+        LOG.warn(t.getTaskID()+" Reporting Diagnostics", e);
       }
       }
     } finally {
     } finally {
       try{
       try{
@@ -423,7 +424,7 @@ abstract class TaskRunner extends Thread {
       }catch(IOException ie){
       }catch(IOException ie){
         LOG.warn("Error releasing caches : Cache files might not have been cleaned up");
         LOG.warn("Error releasing caches : Cache files might not have been cleaned up");
       }
       }
-      tracker.reportTaskFinished(t.getTaskId());
+      tracker.reportTaskFinished(t.getTaskID());
     }
     }
   }
   }
 
 
@@ -461,7 +462,7 @@ abstract class TaskRunner extends Thread {
    * Run the child process
    * Run the child process
    */
    */
   private void runChild(List<String> args, File dir,
   private void runChild(List<String> args, File dir,
-                        String taskid) throws IOException {
+                        TaskAttemptID taskid) throws IOException {
 
 
     try {
     try {
       shexec = new ShellCommandExecutor(args.toArray(new String[0]), dir);
       shexec = new ShellCommandExecutor(args.toArray(new String[0]), dir);

+ 15 - 14
src/java/org/apache/hadoop/mapred/TaskStatus.java

@@ -24,7 +24,7 @@ import java.util.List;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
 /**************************************************
 /**************************************************
@@ -43,7 +43,7 @@ abstract class TaskStatus implements Writable, Cloneable {
   public static enum State {RUNNING, SUCCEEDED, FAILED, UNASSIGNED, KILLED, 
   public static enum State {RUNNING, SUCCEEDED, FAILED, UNASSIGNED, KILLED, 
                             COMMIT_PENDING}
                             COMMIT_PENDING}
     
     
-  private String taskid;
+  private TaskAttemptID taskid;
   private float progress;
   private float progress;
   private State runState;
   private State runState;
   private String diagnosticInfo;
   private String diagnosticInfo;
@@ -59,7 +59,7 @@ abstract class TaskStatus implements Writable, Cloneable {
 
 
   public TaskStatus() {}
   public TaskStatus() {}
 
 
-  public TaskStatus(String taskid, float progress,
+  public TaskStatus(TaskAttemptID taskid, float progress,
                     State runState, String diagnosticInfo,
                     State runState, String diagnosticInfo,
                     String stateString, String taskTracker,
                     String stateString, String taskTracker,
                     Phase phase, Counters counters) {
                     Phase phase, Counters counters) {
@@ -74,7 +74,7 @@ abstract class TaskStatus implements Writable, Cloneable {
     this.includeCounters = true;
     this.includeCounters = true;
   }
   }
   
   
-  public String getTaskId() { return taskid; }
+  public TaskAttemptID getTaskID() { return taskid; }
   public abstract boolean getIsMap();
   public abstract boolean getIsMap();
   public float getProgress() { return progress; }
   public float getProgress() { return progress; }
   public void setProgress(float progress) { this.progress = progress; } 
   public void setProgress(float progress) { this.progress = progress; } 
@@ -211,7 +211,7 @@ abstract class TaskStatus implements Writable, Cloneable {
    * 
    * 
    * @return the list of maps from which output-fetches failed.
    * @return the list of maps from which output-fetches failed.
    */
    */
-  public List<String> getFetchFailedMaps() {
+  public List<TaskAttemptID> getFetchFailedMaps() {
     return null;
     return null;
   }
   }
   
   
@@ -220,7 +220,7 @@ abstract class TaskStatus implements Writable, Cloneable {
    *  
    *  
    * @param mapTaskId map from which fetch failed
    * @param mapTaskId map from which fetch failed
    */
    */
-  synchronized void addFetchFailedMap(String mapTaskId) {}
+  synchronized void addFetchFailedMap(TaskAttemptID mapTaskId) {}
 
 
   /**
   /**
    * Update the status of the task.
    * Update the status of the task.
@@ -271,6 +271,7 @@ abstract class TaskStatus implements Writable, Cloneable {
     diagnosticInfo = "";
     diagnosticInfo = "";
   }
   }
 
 
+  @Override
   public Object clone() {
   public Object clone() {
     try {
     try {
       return super.clone();
       return super.clone();
@@ -284,11 +285,11 @@ abstract class TaskStatus implements Writable, Cloneable {
   // Writable
   // Writable
   //////////////////////////////////////////////
   //////////////////////////////////////////////
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    UTF8.writeString(out, taskid);
+    taskid.write(out);
     out.writeFloat(progress);
     out.writeFloat(progress);
     WritableUtils.writeEnum(out, runState);
     WritableUtils.writeEnum(out, runState);
-    UTF8.writeString(out, diagnosticInfo);
-    UTF8.writeString(out, stateString);
+    Text.writeString(out, diagnosticInfo);
+    Text.writeString(out, stateString);
     WritableUtils.writeEnum(out, phase);
     WritableUtils.writeEnum(out, phase);
     out.writeLong(startTime);
     out.writeLong(startTime);
     out.writeLong(finishTime);
     out.writeLong(finishTime);
@@ -299,11 +300,11 @@ abstract class TaskStatus implements Writable, Cloneable {
   }
   }
 
 
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    this.taskid = UTF8.readString(in);
+    this.taskid = TaskAttemptID.read(in);
     this.progress = in.readFloat();
     this.progress = in.readFloat();
     this.runState = WritableUtils.readEnum(in, State.class);
     this.runState = WritableUtils.readEnum(in, State.class);
-    this.diagnosticInfo = UTF8.readString(in);
-    this.stateString = UTF8.readString(in);
+    this.diagnosticInfo = Text.readString(in);
+    this.stateString = Text.readString(in);
     this.phase = WritableUtils.readEnum(in, Phase.class); 
     this.phase = WritableUtils.readEnum(in, Phase.class); 
     this.startTime = in.readLong(); 
     this.startTime = in.readLong(); 
     this.finishTime = in.readLong(); 
     this.finishTime = in.readLong(); 
@@ -318,7 +319,7 @@ abstract class TaskStatus implements Writable, Cloneable {
   // Factory-like methods to create/read/write appropriate TaskStatus objects
   // Factory-like methods to create/read/write appropriate TaskStatus objects
   //////////////////////////////////////////////////////////////////////////////
   //////////////////////////////////////////////////////////////////////////////
   
   
-  static TaskStatus createTaskStatus(DataInput in, String taskId, float progress,
+  static TaskStatus createTaskStatus(DataInput in, TaskAttemptID taskId, float progress,
                                      State runState, String diagnosticInfo,
                                      State runState, String diagnosticInfo,
                                      String stateString, String taskTracker,
                                      String stateString, String taskTracker,
                                      Phase phase, Counters counters) 
                                      Phase phase, Counters counters) 
@@ -328,7 +329,7 @@ abstract class TaskStatus implements Writable, Cloneable {
                           stateString, taskTracker, phase, counters);
                           stateString, taskTracker, phase, counters);
   }
   }
   
   
-  static TaskStatus createTaskStatus(boolean isMap, String taskId, float progress,
+  static TaskStatus createTaskStatus(boolean isMap, TaskAttemptID taskId, float progress,
                                    State runState, String diagnosticInfo,
                                    State runState, String diagnosticInfo,
                                    String stateString, String taskTracker,
                                    String stateString, String taskTracker,
                                    Phase phase, Counters counters) { 
                                    Phase phase, Counters counters) { 

+ 87 - 79
src/java/org/apache/hadoop/mapred/TaskTracker.java

@@ -26,7 +26,7 @@ import java.io.PrintStream;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
-import java.net.URISyntaxException; 
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
@@ -36,10 +36,10 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Random;
 import java.util.Set;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeMap;
+import java.util.Vector;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
-import java.util.Vector;
 
 
 import javax.servlet.ServletContext;
 import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
 import javax.servlet.ServletException;
@@ -55,10 +55,9 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSError;
 import org.apache.hadoop.fs.FSError;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
@@ -77,6 +76,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.RunJar;
 import org.apache.hadoop.util.RunJar;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.LogManager;
 
 
 /*******************************************************
 /*******************************************************
@@ -124,12 +124,12 @@ public class TaskTracker
     
     
   boolean shuttingDown = false;
   boolean shuttingDown = false;
     
     
-  Map<String, TaskInProgress> tasks = new HashMap<String, TaskInProgress>();
+  Map<TaskAttemptID, TaskInProgress> tasks = new HashMap<TaskAttemptID, TaskInProgress>();
   /**
   /**
    * Map from taskId -> TaskInProgress.
    * Map from taskId -> TaskInProgress.
    */
    */
-  Map<String, TaskInProgress> runningTasks = null;
-  Map<String, RunningJob> runningJobs = null;
+  Map<TaskAttemptID, TaskInProgress> runningTasks = null;
+  Map<JobID, RunningJob> runningJobs = null;
   volatile int mapTotal = 0;
   volatile int mapTotal = 0;
   volatile int reduceTotal = 0;
   volatile int reduceTotal = 0;
   boolean justStarted = true;
   boolean justStarted = true;
@@ -303,10 +303,10 @@ public class TaskTracker
                 TaskInProgress tip;
                 TaskInProgress tip;
                 KillTaskAction killAction = (KillTaskAction) action;
                 KillTaskAction killAction = (KillTaskAction) action;
                 synchronized (TaskTracker.this) {
                 synchronized (TaskTracker.this) {
-                  tip = tasks.get(killAction.getTaskId());
+                  tip = tasks.get(killAction.getTaskID());
                 }
                 }
                 LOG.info("Received KillTaskAction for task: " + 
                 LOG.info("Received KillTaskAction for task: " + 
-                         killAction.getTaskId());
+                         killAction.getTaskID());
                 purgeTask(tip, false);
                 purgeTask(tip, false);
               } else {
               } else {
                 LOG.error("Non-delete action given to cleanup thread: "
                 LOG.error("Non-delete action given to cleanup thread: "
@@ -323,7 +323,7 @@ public class TaskTracker
     taskCleanupThread.start();
     taskCleanupThread.start();
   }
   }
     
     
-  private RunningJob addTaskToJob(String jobId, 
+  private RunningJob addTaskToJob(JobID jobId, 
                                   Path localJobFile,
                                   Path localJobFile,
                                   TaskInProgress tip) {
                                   TaskInProgress tip) {
     synchronized (runningJobs) {
     synchronized (runningJobs) {
@@ -345,7 +345,7 @@ public class TaskTracker
     }
     }
   }
   }
 
 
-  private void removeTaskFromJob(String jobId, TaskInProgress tip) {
+  private void removeTaskFromJob(JobID jobId, TaskInProgress tip) {
     synchronized (runningJobs) {
     synchronized (runningJobs) {
       RunningJob rjob = runningJobs.get(jobId);
       RunningJob rjob = runningJobs.get(jobId);
       if (rjob == null) {
       if (rjob == null) {
@@ -403,8 +403,8 @@ public class TaskTracker
 
 
     // Clear out state tables
     // Clear out state tables
     this.tasks.clear();
     this.tasks.clear();
-    this.runningTasks = new TreeMap<String, TaskInProgress>();
-    this.runningJobs = new TreeMap<String, RunningJob>();
+    this.runningTasks = new TreeMap<TaskAttemptID, TaskInProgress>();
+    this.runningJobs = new TreeMap<JobID, RunningJob>();
     this.mapTotal = 0;
     this.mapTotal = 0;
     this.reduceTotal = 0;
     this.reduceTotal = 0;
     this.acceptNewTasks = true;
     this.acceptNewTasks = true;
@@ -480,9 +480,9 @@ public class TaskTracker
 
 
     private List <FetchStatus> reducesInShuffle() {
     private List <FetchStatus> reducesInShuffle() {
       List <FetchStatus> fList = new ArrayList<FetchStatus>();
       List <FetchStatus> fList = new ArrayList<FetchStatus>();
-      for (Map.Entry <String, RunningJob> item : runningJobs.entrySet()) {
+      for (Map.Entry <JobID, RunningJob> item : runningJobs.entrySet()) {
         RunningJob rjob = item.getValue();
         RunningJob rjob = item.getValue();
-        String jobId = item.getKey();
+        JobID jobId = item.getKey();
         FetchStatus f;
         FetchStatus f;
         synchronized (rjob) {
         synchronized (rjob) {
           f = rjob.getFetchStatus();
           f = rjob.getFetchStatus();
@@ -511,6 +511,7 @@ public class TaskTracker
       return fList;
       return fList;
     }
     }
       
       
+    @Override
     public void run() {
     public void run() {
       LOG.info("Starting thread: " + getName());
       LOG.info("Starting thread: " + getName());
         
         
@@ -577,9 +578,9 @@ public class TaskTracker
     /** This is the cache of map events for a given job */ 
     /** This is the cache of map events for a given job */ 
     private List<TaskCompletionEvent> allMapEvents;
     private List<TaskCompletionEvent> allMapEvents;
     /** What jobid this fetchstatus object is for*/
     /** What jobid this fetchstatus object is for*/
-    private String jobId;
+    private JobID jobId;
      
      
-    public FetchStatus(String jobId, int numMaps) {
+    public FetchStatus(JobID jobId, int numMaps) {
       this.fromEventId = new IntWritable(0);
       this.fromEventId = new IntWritable(0);
       this.jobId = jobId;
       this.jobId = jobId;
       this.allMapEvents = new ArrayList<TaskCompletionEvent>(numMaps);
       this.allMapEvents = new ArrayList<TaskCompletionEvent>(numMaps);
@@ -625,7 +626,8 @@ public class TaskTracker
   private void localizeJob(TaskInProgress tip) throws IOException {
   private void localizeJob(TaskInProgress tip) throws IOException {
     Path localJarFile = null;
     Path localJarFile = null;
     Task t = tip.getTask();
     Task t = tip.getTask();
-    String jobId = t.getJobId();
+    
+    JobID jobId = t.getJobID();
     String jobFile = t.getJobFile();
     String jobFile = t.getJobFile();
     // Get sizes of JobFile and JarFile
     // Get sizes of JobFile and JarFile
     // sizes are -1 if they are not present.
     // sizes are -1 if they are not present.
@@ -758,8 +760,8 @@ public class TaskTracker
     // Kill running tasks.  Do this in a 2nd vector, called 'tasksToClose',
     // Kill running tasks.  Do this in a 2nd vector, called 'tasksToClose',
     // because calling jobHasFinished() may result in an edit to 'tasks'.
     // because calling jobHasFinished() may result in an edit to 'tasks'.
     //
     //
-    TreeMap<String, TaskInProgress> tasksToClose =
-      new TreeMap<String, TaskInProgress>();
+    TreeMap<TaskAttemptID, TaskInProgress> tasksToClose =
+      new TreeMap<TaskAttemptID, TaskInProgress>();
     tasksToClose.putAll(tasks);
     tasksToClose.putAll(tasks);
     for (TaskInProgress tip : tasksToClose.values()) {
     for (TaskInProgress tip : tasksToClose.values()) {
       tip.jobHasFinished(false);
       tip.jobHasFinished(false);
@@ -770,6 +772,7 @@ public class TaskTracker
     // time to shutdown.  (They need to wait a full
     // time to shutdown.  (They need to wait a full
     // RPC timeout, which might be 10-30 seconds.)
     // RPC timeout, which might be 10-30 seconds.)
     new Thread("RPC shutdown") {
     new Thread("RPC shutdown") {
+      @Override
       public void run() {
       public void run() {
         if (taskReportServer != null) {
         if (taskReportServer != null) {
           taskReportServer.stop();
           taskReportServer.stop();
@@ -851,7 +854,7 @@ public class TaskTracker
    * @throws IOException
    * @throws IOException
    */  
    */  
   private List<TaskCompletionEvent> queryJobTracker(IntWritable fromEventId,
   private List<TaskCompletionEvent> queryJobTracker(IntWritable fromEventId,
-                                                    String jobId,
+                                                    JobID jobId,
                                                     InterTrackerProtocol jobClient)
                                                     InterTrackerProtocol jobClient)
     throws IOException {
     throws IOException {
 
 
@@ -1033,7 +1036,7 @@ public class TaskTracker
           } catch (MetricsException me) {
           } catch (MetricsException me) {
             LOG.warn("Caught: " + StringUtils.stringifyException(me));
             LOG.warn("Caught: " + StringUtils.stringifyException(me));
           }
           }
-          runningTasks.remove(taskStatus.getTaskId());
+          runningTasks.remove(taskStatus.getTaskID());
         }
         }
       }
       }
       
       
@@ -1089,9 +1092,9 @@ public class TaskTracker
         long timeSinceLastReport = now - tip.getLastProgressReport();
         long timeSinceLastReport = now - tip.getLastProgressReport();
         if (timeSinceLastReport > jobTaskTimeout && !tip.wasKilled) {
         if (timeSinceLastReport > jobTaskTimeout && !tip.wasKilled) {
           String msg = 
           String msg = 
-            "Task " + tip.getTask().getTaskId() + " failed to report status for " 
+            "Task " + tip.getTask().getTaskID() + " failed to report status for " 
             + (timeSinceLastReport / 1000) + " seconds. Killing!";
             + (timeSinceLastReport / 1000) + " seconds. Killing!";
-          LOG.info(tip.getTask().getTaskId() + ": " + msg);
+          LOG.info(tip.getTask().getTaskID() + ": " + msg);
           ReflectionUtils.logThreadInfo(LOG, "lost task", 30);
           ReflectionUtils.logThreadInfo(LOG, "lost task", 30);
           tip.reportDiagnosticInfo(msg);
           tip.reportDiagnosticInfo(msg);
           myMetrics.timedoutTask();
           myMetrics.timedoutTask();
@@ -1107,7 +1110,7 @@ public class TaskTracker
    * @throws IOException
    * @throws IOException
    */
    */
   private synchronized void purgeJob(KillJobAction action) throws IOException {
   private synchronized void purgeJob(KillJobAction action) throws IOException {
-    String jobId = action.getJobId();
+    JobID jobId = action.getJobID();
     LOG.info("Received 'KillJobAction' for job: " + jobId);
     LOG.info("Received 'KillJobAction' for job: " + jobId);
     RunningJob rjob = null;
     RunningJob rjob = null;
     synchronized (runningJobs) {
     synchronized (runningJobs) {
@@ -1126,7 +1129,7 @@ public class TaskTracker
         // task if the job is done/failed
         // task if the job is done/failed
         if (!rjob.keepJobFiles){
         if (!rjob.keepJobFiles){
           fConf.deleteLocalFiles(SUBDIR + Path.SEPARATOR + JOBCACHE + 
           fConf.deleteLocalFiles(SUBDIR + Path.SEPARATOR + JOBCACHE + 
-                                 Path.SEPARATOR +  rjob.getJobId());
+                                 Path.SEPARATOR +  rjob.getJobID());
         }
         }
         // Remove this job 
         // Remove this job 
         rjob.tasks.clear();
         rjob.tasks.clear();
@@ -1148,11 +1151,11 @@ public class TaskTracker
   private void purgeTask(TaskInProgress tip, boolean wasFailure) 
   private void purgeTask(TaskInProgress tip, boolean wasFailure) 
   throws IOException {
   throws IOException {
     if (tip != null) {
     if (tip != null) {
-      LOG.info("About to purge task: " + tip.getTask().getTaskId());
+      LOG.info("About to purge task: " + tip.getTask().getTaskID());
         
         
       // Remove the task from running jobs, 
       // Remove the task from running jobs, 
       // removing the job if it's the last task
       // removing the job if it's the last task
-      removeTaskFromJob(tip.getTask().getJobId(), tip);
+      removeTaskFromJob(tip.getTask().getJobID(), tip);
       tip.jobHasFinished(wasFailure);
       tip.jobHasFinished(wasFailure);
     }
     }
   }
   }
@@ -1179,7 +1182,7 @@ public class TaskTracker
         if (killMe!=null) {
         if (killMe!=null) {
           String msg = "Tasktracker running out of space." +
           String msg = "Tasktracker running out of space." +
             " Killing task.";
             " Killing task.";
-          LOG.info(killMe.getTask().getTaskId() + ": " + msg);
+          LOG.info(killMe.getTask().getTaskID() + ": " + msg);
           killMe.reportDiagnosticInfo(msg);
           killMe.reportDiagnosticInfo(msg);
           purgeTask(killMe, false);
           purgeTask(killMe, false);
         }
         }
@@ -1258,11 +1261,11 @@ public class TaskTracker
    */
    */
   private void startNewTask(LaunchTaskAction action) {
   private void startNewTask(LaunchTaskAction action) {
     Task t = action.getTask();
     Task t = action.getTask();
-    LOG.info("LaunchTaskAction: " + t.getTaskId());
+    LOG.info("LaunchTaskAction: " + t.getTaskID());
     TaskInProgress tip = new TaskInProgress(t, this.fConf);
     TaskInProgress tip = new TaskInProgress(t, this.fConf);
     synchronized (this) {
     synchronized (this) {
-      tasks.put(t.getTaskId(), tip);
-      runningTasks.put(t.getTaskId(), tip);
+      tasks.put(t.getTaskID(), tip);
+      runningTasks.put(t.getTaskID(), tip);
       boolean isMap = t.isMapTask();
       boolean isMap = t.isMapTask();
       if (isMap) {
       if (isMap) {
         mapTotal++;
         mapTotal++;
@@ -1273,14 +1276,14 @@ public class TaskTracker
     try {
     try {
       localizeJob(tip);
       localizeJob(tip);
     } catch (Throwable e) {
     } catch (Throwable e) {
-      String msg = ("Error initializing " + tip.getTask().getTaskId() + 
+      String msg = ("Error initializing " + tip.getTask().getTaskID() + 
                     ":\n" + StringUtils.stringifyException(e));
                     ":\n" + StringUtils.stringifyException(e));
       LOG.warn(msg);
       LOG.warn(msg);
       tip.reportDiagnosticInfo(msg);
       tip.reportDiagnosticInfo(msg);
       try {
       try {
         tip.kill(true);
         tip.kill(true);
       } catch (IOException ie2) {
       } catch (IOException ie2) {
-        LOG.info("Error cleaning up " + tip.getTask().getTaskId() + ":\n" +
+        LOG.info("Error cleaning up " + tip.getTask().getTaskID() + ":\n" +
                  StringUtils.stringifyException(ie2));          
                  StringUtils.stringifyException(ie2));          
       }
       }
         
         
@@ -1369,7 +1372,7 @@ public class TaskTracker
       this.lastProgressReport = System.currentTimeMillis();
       this.lastProgressReport = System.currentTimeMillis();
       this.defaultJobConf = conf;
       this.defaultJobConf = conf;
       localJobConf = null;
       localJobConf = null;
-      taskStatus = TaskStatus.createTaskStatus(task.isMapTask(), task.getTaskId(), 
+      taskStatus = TaskStatus.createTaskStatus(task.isMapTask(), task.getTaskID(), 
                                                0.0f, 
                                                0.0f, 
                                                TaskStatus.State.UNASSIGNED, 
                                                TaskStatus.State.UNASSIGNED, 
                                                diagnosticInfo.toString(), 
                                                diagnosticInfo.toString(), 
@@ -1382,10 +1385,12 @@ public class TaskTracker
     }
     }
         
         
     private void localizeTask(Task task) throws IOException{
     private void localizeTask(Task task) throws IOException{
+
       Path localTaskDir = 
       Path localTaskDir = 
         lDirAlloc.getLocalPathForWrite((TaskTracker.getJobCacheSubdir() + 
         lDirAlloc.getLocalPathForWrite((TaskTracker.getJobCacheSubdir() + 
-                    Path.SEPARATOR + task.getJobId() + Path.SEPARATOR +
-                    task.getTaskId()), defaultJobConf );
+                    Path.SEPARATOR + task.getJobID() + Path.SEPARATOR +
+                    task.getTaskID()), defaultJobConf );
+      
       FileSystem localFs = FileSystem.getLocal(fConf);
       FileSystem localFs = FileSystem.getLocal(fConf);
       if (!localFs.mkdirs(localTaskDir)) {
       if (!localFs.mkdirs(localTaskDir)) {
         throw new IOException("Mkdirs failed to create " 
         throw new IOException("Mkdirs failed to create " 
@@ -1395,7 +1400,7 @@ public class TaskTracker
       // create symlink for ../work if it already doesnt exist
       // create symlink for ../work if it already doesnt exist
       String workDir = lDirAlloc.getLocalPathToRead(
       String workDir = lDirAlloc.getLocalPathToRead(
                          TaskTracker.getJobCacheSubdir() 
                          TaskTracker.getJobCacheSubdir() 
-                         + Path.SEPARATOR + task.getJobId() 
+                         + Path.SEPARATOR + task.getJobID() 
                          + Path.SEPARATOR  
                          + Path.SEPARATOR  
                          + "work", defaultJobConf).toString();
                          + "work", defaultJobConf).toString();
       String link = localTaskDir.getParent().toString() 
       String link = localTaskDir.getParent().toString() 
@@ -1407,8 +1412,8 @@ public class TaskTracker
       // create the working-directory of the task 
       // create the working-directory of the task 
       Path cwd = lDirAlloc.getLocalPathForWrite(
       Path cwd = lDirAlloc.getLocalPathForWrite(
                          TaskTracker.getJobCacheSubdir() 
                          TaskTracker.getJobCacheSubdir() 
-                         + Path.SEPARATOR + task.getJobId() 
-                         + Path.SEPARATOR + task.getTaskId()
+                         + Path.SEPARATOR + task.getJobID() 
+                         + Path.SEPARATOR + task.getTaskID()
                          + Path.SEPARATOR + "work",
                          + Path.SEPARATOR + "work",
                          defaultJobConf);
                          defaultJobConf);
       if (!localFs.mkdirs(cwd)) {
       if (!localFs.mkdirs(cwd)) {
@@ -1421,7 +1426,7 @@ public class TaskTracker
       localJobConf.set("mapred.local.dir",
       localJobConf.set("mapred.local.dir",
                        fConf.get("mapred.local.dir"));
                        fConf.get("mapred.local.dir"));
             
             
-      localJobConf.set("mapred.task.id", task.getTaskId());
+      localJobConf.set("mapred.task.id", task.getTaskID().toString());
       keepFailedTaskFiles = localJobConf.getKeepFailedTaskFiles();
       keepFailedTaskFiles = localJobConf.getKeepFailedTaskFiles();
 
 
       // create _taskid directory in output path temporary directory.
       // create _taskid directory in output path temporary directory.
@@ -1430,7 +1435,7 @@ public class TaskTracker
         Path jobTmpDir = new Path(outputPath, MRConstants.TEMP_DIR_NAME);
         Path jobTmpDir = new Path(outputPath, MRConstants.TEMP_DIR_NAME);
         FileSystem fs = jobTmpDir.getFileSystem(localJobConf);
         FileSystem fs = jobTmpDir.getFileSystem(localJobConf);
         if (fs.exists(jobTmpDir)) {
         if (fs.exists(jobTmpDir)) {
-          Path taskTmpDir = new Path(jobTmpDir, "_" + task.getTaskId());
+          Path taskTmpDir = new Path(jobTmpDir, "_" + task.getTaskID());
           if (!fs.mkdirs(taskTmpDir)) {
           if (!fs.mkdirs(taskTmpDir)) {
             throw new IOException("Mkdirs failed to create " 
             throw new IOException("Mkdirs failed to create " 
                                  + taskTmpDir.toString());
                                  + taskTmpDir.toString());
@@ -1465,7 +1470,7 @@ public class TaskTracker
       String keepPattern = localJobConf.getKeepTaskFilesPattern();
       String keepPattern = localJobConf.getKeepTaskFilesPattern();
       if (keepPattern != null) {
       if (keepPattern != null) {
         alwaysKeepTaskFiles = 
         alwaysKeepTaskFiles = 
-          Pattern.matches(keepPattern, task.getTaskId());
+          Pattern.matches(keepPattern, task.getTaskID().toString());
       } else {
       } else {
         alwaysKeepTaskFiles = false;
         alwaysKeepTaskFiles = false;
       }
       }
@@ -1520,7 +1525,7 @@ public class TaskTracker
      */
      */
     public synchronized void reportProgress(TaskStatus taskStatus) 
     public synchronized void reportProgress(TaskStatus taskStatus) 
     {
     {
-      LOG.info(task.getTaskId() + " " + taskStatus.getProgress() + 
+      LOG.info(task.getTaskID() + " " + taskStatus.getProgress() + 
           "% " + taskStatus.getStateString());
           "% " + taskStatus.getStateString());
       
       
       if (this.done || 
       if (this.done || 
@@ -1528,7 +1533,7 @@ public class TaskTracker
         //make sure we ignore progress messages after a task has 
         //make sure we ignore progress messages after a task has 
         //invoked TaskUmbilicalProtocol.done() or if the task has been
         //invoked TaskUmbilicalProtocol.done() or if the task has been
         //KILLED/FAILED
         //KILLED/FAILED
-        LOG.info(task.getTaskId() + " Ignoring status-update since " +
+        LOG.info(task.getTaskID() + " Ignoring status-update since " +
                  ((this.done) ? "task is 'done'" : 
                  ((this.done) ? "task is 'done'" : 
                                 ("runState: " + this.taskStatus.getRunState()))
                                 ("runState: " + this.taskStatus.getRunState()))
                  ); 
                  ); 
@@ -1583,7 +1588,7 @@ public class TaskTracker
       this.taskStatus.setFinishTime(System.currentTimeMillis());
       this.taskStatus.setFinishTime(System.currentTimeMillis());
       this.done = true;
       this.done = true;
       
       
-      LOG.info("Task " + task.getTaskId() + " is done.");
+      LOG.info("Task " + task.getTaskID() + " is done.");
     }
     }
 
 
     /**
     /**
@@ -1628,13 +1633,13 @@ public class TaskTracker
               try {
               try {
                 // get task's stdout file 
                 // get task's stdout file 
                 taskStdout = FileUtil.makeShellPath(TaskLog.getTaskLogFile
                 taskStdout = FileUtil.makeShellPath(TaskLog.getTaskLogFile
-                                  (task.getTaskId(), TaskLog.LogName.STDOUT));
+                                  (task.getTaskID(), TaskLog.LogName.STDOUT));
                 // get task's stderr file 
                 // get task's stderr file 
                 taskStderr = FileUtil.makeShellPath(TaskLog.getTaskLogFile
                 taskStderr = FileUtil.makeShellPath(TaskLog.getTaskLogFile
-                                  (task.getTaskId(), TaskLog.LogName.STDERR));
+                                  (task.getTaskID(), TaskLog.LogName.STDERR));
                 // get task's syslog file 
                 // get task's syslog file 
                 taskSyslog = FileUtil.makeShellPath(TaskLog.getTaskLogFile
                 taskSyslog = FileUtil.makeShellPath(TaskLog.getTaskLogFile
-                                  (task.getTaskId(), TaskLog.LogName.SYSLOG));
+                                  (task.getTaskID(), TaskLog.LogName.SYSLOG));
               } catch(IOException e){
               } catch(IOException e){
                 LOG.warn("Exception finding task's stdout/err/syslog files");
                 LOG.warn("Exception finding task's stdout/err/syslog files");
               }
               }
@@ -1642,17 +1647,17 @@ public class TaskTracker
               try {
               try {
                 workDir = new File(lDirAlloc.getLocalPathToRead(
                 workDir = new File(lDirAlloc.getLocalPathToRead(
                                      TaskTracker.getJobCacheSubdir() 
                                      TaskTracker.getJobCacheSubdir() 
-                                     + Path.SEPARATOR + task.getJobId() 
-                                     + Path.SEPARATOR + task.getTaskId()
+                                     + Path.SEPARATOR + task.getJobID() 
+                                     + Path.SEPARATOR + task.getTaskID()
                                      + Path.SEPARATOR + "work",
                                      + Path.SEPARATOR + "work",
                                      localJobConf). toString());
                                      localJobConf). toString());
               } catch (IOException e) {
               } catch (IOException e) {
-                LOG.warn("Working Directory of the task " + task.getTaskId() +
+                LOG.warn("Working Directory of the task " + task.getTaskID() +
                 		 "doesnt exist. Throws expetion " +
                 		 "doesnt exist. Throws expetion " +
                           StringUtils.stringifyException(e));
                           StringUtils.stringifyException(e));
               }
               }
               // Build the command  
               // Build the command  
-              File stdout = TaskLog.getTaskLogFile(task.getTaskId(),
+              File stdout = TaskLog.getTaskLogFile(task.getTaskID(),
                                                    TaskLog.LogName.DEBUGOUT);
                                                    TaskLog.LogName.DEBUGOUT);
               // add pipes program as argument if it exists.
               // add pipes program as argument if it exists.
               String program ="";
               String program ="";
@@ -1850,15 +1855,15 @@ public class TaskTracker
       if (taskStatus.getRunState() == TaskStatus.State.COMMIT_PENDING || 
       if (taskStatus.getRunState() == TaskStatus.State.COMMIT_PENDING || 
           taskStatus.getRunState() == TaskStatus.State.SUCCEEDED) {
           taskStatus.getRunState() == TaskStatus.State.SUCCEEDED) {
         // change status to failure
         // change status to failure
-        LOG.info("Reporting output lost:"+task.getTaskId());
+        LOG.info("Reporting output lost:"+task.getTaskID());
         taskStatus.setRunState(TaskStatus.State.FAILED);
         taskStatus.setRunState(TaskStatus.State.FAILED);
         taskStatus.setProgress(0.0f);
         taskStatus.setProgress(0.0f);
         reportDiagnosticInfo("Map output lost, rescheduling: " + 
         reportDiagnosticInfo("Map output lost, rescheduling: " + 
                              failure);
                              failure);
-        runningTasks.put(task.getTaskId(), this);
+        runningTasks.put(task.getTaskID(), this);
         mapTotal++;
         mapTotal++;
       } else {
       } else {
-        LOG.warn("Output already reported lost:"+task.getTaskId());
+        LOG.warn("Output already reported lost:"+task.getTaskID());
       }
       }
     }
     }
 
 
@@ -1871,7 +1876,7 @@ public class TaskTracker
      * by locking tasktracker first and then locks the tip.
      * by locking tasktracker first and then locks the tip.
      */
      */
     void cleanup() throws IOException {
     void cleanup() throws IOException {
-      String taskId = task.getTaskId();
+      TaskAttemptID taskId = task.getTaskID();
       LOG.debug("Cleaning up " + taskId);
       LOG.debug("Cleaning up " + taskId);
       synchronized (TaskTracker.this) {
       synchronized (TaskTracker.this) {
         tasks.remove(taskId);
         tasks.remove(taskId);
@@ -1890,7 +1895,7 @@ public class TaskTracker
           }
           }
           defaultJobConf.deleteLocalFiles(SUBDIR + Path.SEPARATOR + 
           defaultJobConf.deleteLocalFiles(SUBDIR + Path.SEPARATOR + 
                                           JOBCACHE + Path.SEPARATOR + 
                                           JOBCACHE + Path.SEPARATOR + 
-                                          task.getJobId() + 
+                                          task.getJobID() + 
                                           Path.SEPARATOR + taskId);
                                           Path.SEPARATOR + taskId);
         } catch (Throwable ie) {
         } catch (Throwable ie) {
           LOG.info("Error cleaning up task runner: " + 
           LOG.info("Error cleaning up task runner: " + 
@@ -1899,14 +1904,16 @@ public class TaskTracker
       }
       }
     }
     }
         
         
+    @Override
     public boolean equals(Object obj) {
     public boolean equals(Object obj) {
       return (obj instanceof TaskInProgress) &&
       return (obj instanceof TaskInProgress) &&
-        task.getTaskId().equals
-        (((TaskInProgress) obj).getTask().getTaskId());
+        task.getTaskID().equals
+        (((TaskInProgress) obj).getTask().getTaskID());
     }
     }
         
         
+    @Override
     public int hashCode() {
     public int hashCode() {
-      return task.getTaskId().hashCode();
+      return task.getTaskID().hashCode();
     }
     }
   }
   }
 
 
@@ -1917,7 +1924,7 @@ public class TaskTracker
   /**
   /**
    * Called upon startup by the child process, to fetch Task data.
    * Called upon startup by the child process, to fetch Task data.
    */
    */
-  public synchronized Task getTask(String taskid) throws IOException {
+  public synchronized Task getTask(TaskAttemptID taskid) throws IOException {
     TaskInProgress tip = tasks.get(taskid);
     TaskInProgress tip = tasks.get(taskid);
     if (tip != null) {
     if (tip != null) {
       return tip.getTask();
       return tip.getTask();
@@ -1929,7 +1936,7 @@ public class TaskTracker
   /**
   /**
    * Called periodically to report Task progress, from 0.0 to 1.0.
    * Called periodically to report Task progress, from 0.0 to 1.0.
    */
    */
-  public synchronized boolean statusUpdate(String taskid, 
+  public synchronized boolean statusUpdate(TaskAttemptID taskid, 
                                               TaskStatus taskStatus) 
                                               TaskStatus taskStatus) 
   throws IOException {
   throws IOException {
     TaskInProgress tip = tasks.get(taskid);
     TaskInProgress tip = tasks.get(taskid);
@@ -1946,7 +1953,7 @@ public class TaskTracker
    * 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(String 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);
@@ -1956,14 +1963,14 @@ public class TaskTracker
   }
   }
 
 
   /** 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(String taskid) throws IOException {
+  public synchronized boolean ping(TaskAttemptID taskid) throws IOException {
     return tasks.get(taskid) != null;
     return tasks.get(taskid) != null;
   }
   }
 
 
   /**
   /**
    * The task is done.
    * The task is done.
    */
    */
-  public synchronized void done(String taskid, boolean shouldPromote) 
+  public synchronized void done(TaskAttemptID taskid, boolean shouldPromote) 
   throws IOException {
   throws IOException {
     TaskInProgress tip = tasks.get(taskid);
     TaskInProgress tip = tasks.get(taskid);
     if (tip != null) {
     if (tip != null) {
@@ -1977,7 +1984,7 @@ public class TaskTracker
   /** 
   /** 
    * A reduce-task failed to shuffle the map-outputs. Kill the task.
    * A reduce-task failed to shuffle the map-outputs. Kill the task.
    */  
    */  
-  public synchronized void shuffleError(String taskId, String message) 
+  public synchronized void shuffleError(TaskAttemptID taskId, String message) 
   throws IOException { 
   throws IOException { 
     LOG.fatal("Task: " + taskId + " - Killed due to Shuffle Failure: " + message);
     LOG.fatal("Task: " + taskId + " - Killed due to Shuffle Failure: " + message);
     TaskInProgress tip = runningTasks.get(taskId);
     TaskInProgress tip = runningTasks.get(taskId);
@@ -1988,7 +1995,7 @@ public class TaskTracker
   /** 
   /** 
    * A child task had a local filesystem error. Kill the task.
    * A child task had a local filesystem error. Kill the task.
    */  
    */  
-  public synchronized void fsError(String taskId, String message) 
+  public synchronized void fsError(TaskAttemptID taskId, String message) 
   throws IOException {
   throws IOException {
     LOG.fatal("Task: " + taskId + " - Killed due to FSError: " + message);
     LOG.fatal("Task: " + taskId + " - Killed due to FSError: " + message);
     TaskInProgress tip = runningTasks.get(taskId);
     TaskInProgress tip = runningTasks.get(taskId);
@@ -1996,8 +2003,8 @@ public class TaskTracker
     purgeTask(tip, true);
     purgeTask(tip, true);
   }
   }
 
 
-  public TaskCompletionEvent[] getMapCompletionEvents(
-                                                      String jobId, int fromEventId, int maxLocs) throws IOException {
+  public TaskCompletionEvent[] getMapCompletionEvents(JobID jobId
+      , int fromEventId, int maxLocs) throws IOException {
       
       
     TaskCompletionEvent[]mapEvents = TaskCompletionEvent.EMPTY_ARRAY;
     TaskCompletionEvent[]mapEvents = TaskCompletionEvent.EMPTY_ARRAY;
     RunningJob rjob;
     RunningJob rjob;
@@ -2021,7 +2028,7 @@ public class TaskTracker
   /**
   /**
    * The task is no longer running.  It may not have completed successfully
    * The task is no longer running.  It may not have completed successfully
    */
    */
-  void reportTaskFinished(String taskid) {
+  void reportTaskFinished(TaskAttemptID taskid) {
     TaskInProgress tip;
     TaskInProgress tip;
     synchronized (this) {
     synchronized (this) {
       tip = tasks.get(taskid);
       tip = tasks.get(taskid);
@@ -2040,7 +2047,7 @@ public class TaskTracker
   /**
   /**
    * A completed map task's output has been lost.
    * A completed map task's output has been lost.
    */
    */
-  public synchronized void mapOutputLost(String taskid,
+  public synchronized void mapOutputLost(TaskAttemptID taskid,
                                          String errorMsg) throws IOException {
                                          String errorMsg) throws IOException {
     TaskInProgress tip = tasks.get(taskid);
     TaskInProgress tip = tasks.get(taskid);
     if (tip != null) {
     if (tip != null) {
@@ -2054,14 +2061,14 @@ public class TaskTracker
    *  The datastructure for initializing a job
    *  The datastructure for initializing a job
    */
    */
   static class RunningJob{
   static class RunningJob{
-    private String jobid; 
+    private JobID jobid; 
     private Path jobFile;
     private Path jobFile;
     // keep this for later use
     // keep this for later use
     Set<TaskInProgress> tasks;
     Set<TaskInProgress> tasks;
     boolean localized;
     boolean localized;
     boolean keepJobFiles;
     boolean keepJobFiles;
     FetchStatus f;
     FetchStatus f;
-    RunningJob(String jobid, Path jobFile) {
+    RunningJob(JobID jobid, Path jobFile) {
       this.jobid = jobid;
       this.jobid = jobid;
       localized = false;
       localized = false;
       tasks = new HashSet<TaskInProgress>();
       tasks = new HashSet<TaskInProgress>();
@@ -2073,7 +2080,7 @@ public class TaskTracker
       return jobFile;
       return jobFile;
     }
     }
       
       
-    String getJobId() {
+    JobID getJobID() {
       return jobid;
       return jobid;
     }
     }
       
       
@@ -2099,7 +2106,7 @@ public class TaskTracker
       String host = args[0];
       String host = args[0];
       int port = Integer.parseInt(args[1]);
       int port = Integer.parseInt(args[1]);
       InetSocketAddress address = new InetSocketAddress(host, port);
       InetSocketAddress address = new InetSocketAddress(host, port);
-      String taskid = args[2];
+      TaskAttemptID taskid = TaskAttemptID.forName(args[2]);
       //set a very high idle timeout so that the connection is never closed
       //set a very high idle timeout so that the connection is never closed
       defaultConf.setInt("ipc.client.connection.maxidletime", 60*60*1000);
       defaultConf.setInt("ipc.client.connection.maxidletime", 60*60*1000);
       TaskUmbilicalProtocol umbilical =
       TaskUmbilicalProtocol umbilical =
@@ -2185,7 +2192,7 @@ public class TaskTracker
    */
    */
   synchronized List<TaskStatus> getNonRunningTasks() {
   synchronized List<TaskStatus> getNonRunningTasks() {
     List<TaskStatus> result = new ArrayList<TaskStatus>(tasks.size());
     List<TaskStatus> result = new ArrayList<TaskStatus>(tasks.size());
-    for(Map.Entry<String, TaskInProgress> task: tasks.entrySet()) {
+    for(Map.Entry<TaskAttemptID, TaskInProgress> task: tasks.entrySet()) {
       if (!runningTasks.containsKey(task.getKey())) {
       if (!runningTasks.containsKey(task.getKey())) {
         result.add(task.getValue().getStatus());
         result.add(task.getValue().getStatus());
       }
       }
@@ -2262,6 +2269,7 @@ public class TaskTracker
    */
    */
   public static class MapOutputServlet extends HttpServlet {
   public static class MapOutputServlet extends HttpServlet {
     private static final int MAX_BYTES_TO_READ = 64 * 1024;
     private static final int MAX_BYTES_TO_READ = 64 * 1024;
+    @Override
     public void doGet(HttpServletRequest request, 
     public void doGet(HttpServletRequest request, 
                       HttpServletResponse response
                       HttpServletResponse response
                       ) throws ServletException, IOException {
                       ) throws ServletException, IOException {
@@ -2369,7 +2377,7 @@ public class TaskTracker
                            StringUtils.stringifyException(ie));
                            StringUtils.stringifyException(ie));
         log.warn(errorMsg);
         log.warn(errorMsg);
         if (isInputException) {
         if (isInputException) {
-          tracker.mapOutputLost(mapId, errorMsg);
+          tracker.mapOutputLost(TaskAttemptID.forName(mapId), errorMsg);
         }
         }
         response.sendError(HttpServletResponse.SC_GONE, errorMsg);
         response.sendError(HttpServletResponse.SC_GONE, errorMsg);
         shuffleMetrics.failedOutput();
         shuffleMetrics.failedOutput();

+ 14 - 11
src/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.lang.InterruptedException;
 
 
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.ipc.VersionedProtocol;
 
 
@@ -35,16 +34,20 @@ interface TaskUmbilicalProtocol extends VersionedProtocol {
    * Changed the version to 4, since we have replaced 
    * Changed the version to 4, since we have replaced 
    *         TaskUmbilicalProtocol.progress(String, float, String, 
    *         TaskUmbilicalProtocol.progress(String, float, String, 
    *         org.apache.hadoop.mapred.TaskStatus.Phase, Counters) 
    *         org.apache.hadoop.mapred.TaskStatus.Phase, Counters) 
-   *         with {@link #statusUpdate(String, TaskStatus)}
+   *         with statusUpdate(String, TaskStatus)
+   * 
    * Version 5 changed counters representation for HADOOP-2248
    * Version 5 changed counters representation for HADOOP-2248
    * Version 6 changes the TaskStatus representation for HADOOP-2208
    * Version 6 changes the TaskStatus representation for HADOOP-2208
    * Version 7 changes the done api (via HADOOP-3140). It now expects whether
    * Version 7 changes the done api (via HADOOP-3140). It now expects whether
    *           or not the task's output needs to be promoted.
    *           or not the task's output needs to be promoted.
+   * Version 8 changes {job|tip|task}id's to use their corresponding 
+   * objects rather than strings.
    * */
    * */
-  public static final long versionID = 7L;
+
+  public static final long versionID = 8L;
   
   
   /** Called when a child task process starts, to get its task.*/
   /** Called when a child task process starts, to get its task.*/
-  Task getTask(String taskid) throws IOException;
+  Task getTask(TaskAttemptID taskid) throws IOException;
 
 
   /**
   /**
    * Report child's progress to parent.
    * Report child's progress to parent.
@@ -55,7 +58,7 @@ interface TaskUmbilicalProtocol extends VersionedProtocol {
    * @throws InterruptedException
    * @throws InterruptedException
    * @return True if the task is known
    * @return True if the task is known
    */
    */
-  boolean statusUpdate(String taskId, TaskStatus taskStatus) 
+  boolean statusUpdate(TaskAttemptID taskId, TaskStatus taskStatus) 
   throws IOException, InterruptedException;
   throws IOException, InterruptedException;
   
   
   /** Report error messages back to parent.  Calls should be sparing, since all
   /** Report error messages back to parent.  Calls should be sparing, since all
@@ -63,25 +66,25 @@ interface TaskUmbilicalProtocol extends VersionedProtocol {
    *  @param taskid the id of the task involved
    *  @param taskid the id of the task involved
    *  @param trace the text to report
    *  @param trace the text to report
    */
    */
-  void reportDiagnosticInfo(String taskid, String trace) throws IOException;
+  void reportDiagnosticInfo(TaskAttemptID taskid, String trace) throws IOException;
 
 
   /** Periodically called by child to check if parent is still alive. 
   /** Periodically called by child to check if parent is still alive. 
    * @return True if the task is known
    * @return True if the task is known
    */
    */
-  boolean ping(String taskid) throws IOException;
+  boolean ping(TaskAttemptID taskid) throws IOException;
 
 
   /** Report that the task is successfully completed.  Failure is assumed if
   /** Report that the task is successfully completed.  Failure is assumed if
    * the task process exits without calling this.
    * the task process exits without calling this.
    * @param taskid task's id
    * @param taskid task's id
    * @param shouldBePromoted whether to promote the task's output or not 
    * @param shouldBePromoted whether to promote the task's output or not 
    */
    */
-  void done(String taskid, boolean shouldBePromoted) throws IOException;
+  void done(TaskAttemptID taskid, boolean shouldBePromoted) throws IOException;
 
 
   /** Report that a reduce-task couldn't shuffle map-outputs.*/
   /** Report that a reduce-task couldn't shuffle map-outputs.*/
-  void shuffleError(String taskId, String message) throws IOException;
+  void shuffleError(TaskAttemptID taskId, String message) throws IOException;
   
   
   /** Report that the task encounted a local filesystem error.*/
   /** Report that the task encounted a local filesystem error.*/
-  void fsError(String taskId, String message) throws IOException;
+  void fsError(TaskAttemptID taskId, String message) throws IOException;
 
 
   /** Called by a reduce task to get the map output locations for finished maps.
   /** Called by a reduce task to get the map output locations for finished maps.
    *
    *
@@ -91,7 +94,7 @@ interface TaskUmbilicalProtocol extends VersionedProtocol {
    * @param maxLocs the max number of locations to fetch
    * @param maxLocs the max number of locations to fetch
    * @return an array of TaskCompletionEvent
    * @return an array of TaskCompletionEvent
    */
    */
-  TaskCompletionEvent[] getMapCompletionEvents(String jobId, 
+  TaskCompletionEvent[] getMapCompletionEvents(JobID jobId, 
                                                int fromIndex, int maxLocs) throws IOException;
                                                int fromIndex, int maxLocs) throws IOException;
 
 
 }
 }

+ 38 - 23
src/java/org/apache/hadoop/mapred/jobcontrol/Job.java

@@ -19,18 +19,18 @@
 package org.apache.hadoop.mapred.jobcontrol;
 package org.apache.hadoop.mapred.jobcontrol;
 
 
 
 
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
-import java.util.ArrayList;
-import java.util.List;
-import java.io.IOException;
-
 /** This class encapsulates a MapReduce job and its dependency. It monitors 
 /** This class encapsulates a MapReduce job and its dependency. It monitors 
  *  the states of the depending jobs and updates the state of this job.
  *  the states of the depending jobs and updates the state of this job.
  *  A job stats in the WAITING state. If it does not have any deoending jobs, or
  *  A job stats in the WAITING state. If it does not have any deoending jobs, or
@@ -56,7 +56,7 @@ public class Job {
   private JobConf theJobConf;
   private JobConf theJobConf;
   private int state;
   private int state;
   private String jobID; 		// assigned and used by JobControl class
   private String jobID; 		// assigned and used by JobControl class
-  private String mapredJobID; // the job ID assigned by map/reduce
+  private JobID mapredJobID; // the job ID assigned by map/reduce
   private String jobName;		// external name, assigned/used by client app
   private String jobName;		// external name, assigned/used by client app
   private String message;		// some info for human consumption, 
   private String message;		// some info for human consumption, 
   // e.g. the reason why the job failed
   // e.g. the reason why the job failed
@@ -74,7 +74,7 @@ public class Job {
     this.dependingJobs = dependingJobs;
     this.dependingJobs = dependingJobs;
     this.state = Job.WAITING;
     this.state = Job.WAITING;
     this.jobID = "unassigned";
     this.jobID = "unassigned";
-    this.mapredJobID = "unassigned";
+    this.mapredJobID = null; //not yet assigned 
     this.jobName = "unassigned";
     this.jobName = "unassigned";
     this.message = "just initialized";
     this.message = "just initialized";
     this.jc = new JobClient(jobConf);
     this.jc = new JobClient(jobConf);
@@ -90,12 +90,14 @@ public class Job {
     this(jobConf, null);
     this(jobConf, null);
   }
   }
 	
 	
+  @Override
   public String toString() {
   public String toString() {
     StringBuffer sb = new StringBuffer();
     StringBuffer sb = new StringBuffer();
     sb.append("job name:\t").append(this.jobName).append("\n");
     sb.append("job name:\t").append(this.jobName).append("\n");
     sb.append("job id:\t").append(this.jobID).append("\n");
     sb.append("job id:\t").append(this.jobID).append("\n");
     sb.append("job state:\t").append(this.state).append("\n");
     sb.append("job state:\t").append(this.state).append("\n");
-    sb.append("job mapred id:\t").append(this.mapredJobID).append("\n");
+    sb.append("job mapred id:\t").append(this.mapredJobID==null ? "unassigned" 
+        : this.mapredJobID).append("\n");
     sb.append("job message:\t").append(this.message).append("\n");
     sb.append("job message:\t").append(this.message).append("\n");
 		
 		
     if (this.dependingJobs == null || this.dependingJobs.size() == 0) {
     if (this.dependingJobs == null || this.dependingJobs.size() == 0) {
@@ -104,7 +106,7 @@ public class Job {
       sb.append("job has ").append(this.dependingJobs.size()).append(" dependeng jobs:\n");
       sb.append("job has ").append(this.dependingJobs.size()).append(" dependeng jobs:\n");
       for (int i = 0; i < this.dependingJobs.size(); i++) {
       for (int i = 0; i < this.dependingJobs.size(); i++) {
         sb.append("\t depending job ").append(i).append(":\t");
         sb.append("\t depending job ").append(i).append(":\t");
-        sb.append(((Job) this.dependingJobs.get(i)).getJobName()).append("\n");
+        sb.append((this.dependingJobs.get(i)).getJobName()).append("\n");
       }
       }
     }
     }
     return sb.toString();
     return sb.toString();
@@ -126,7 +128,7 @@ public class Job {
   }
   }
 	
 	
   /**
   /**
-   * @return the job ID of this job
+   * @return the job ID of this job assigned by JobControl
    */
    */
   public String getJobID() {
   public String getJobID() {
     return this.jobID;
     return this.jobID;
@@ -142,19 +144,40 @@ public class Job {
 	
 	
   /**
   /**
    * @return the mapred ID of this job
    * @return the mapred ID of this job
+   * @deprecated use {@link #getAssignedJobID()} instead
    */
    */
+  @Deprecated
   public String getMapredJobID() {
   public String getMapredJobID() {
-    return this.mapredJobID;
+    return this.mapredJobID.toString();
   }
   }
 	
 	
   /**
   /**
    * Set the mapred ID for this job.
    * Set the mapred ID for this job.
    * @param mapredJobID the mapred job ID for this job.
    * @param mapredJobID the mapred job ID for this job.
+   * @deprecated use {@link #setAssignedJobID(JobID)} instead
    */
    */
+  @Deprecated
   public void setMapredJobID(String mapredJobID) {
   public void setMapredJobID(String mapredJobID) {
-    this.jobID = mapredJobID;
+    this.mapredJobID = JobID.forName(mapredJobID);
   }
   }
 	
 	
+  /**
+   * @return the mapred ID of this job as assigned by the 
+   * mapred framework.
+   */
+  public JobID getAssignedJobID() {
+    return this.mapredJobID;
+  }
+  
+  /**
+   * Set the mapred ID for this job as assigned by the 
+   * mapred framework.
+   * @param mapredJobID the mapred job ID for this job.
+   */
+  public void setAssignedJobID(JobID mapredJobID) {
+    this.mapredJobID = mapredJobID;
+  }
+  
   /**
   /**
    * @return the mapred job conf of this job
    * @return the mapred job conf of this job
    */
    */
@@ -304,7 +327,7 @@ public class Job {
     Job pred = null;
     Job pred = null;
     int n = this.dependingJobs.size();
     int n = this.dependingJobs.size();
     for (int i = 0; i < n; i++) {
     for (int i = 0; i < n; i++) {
-      pred = (Job) this.dependingJobs.get(i);
+      pred = this.dependingJobs.get(i);
       int s = pred.checkState();
       int s = pred.checkState();
       if (s == Job.WAITING || s == Job.READY || s == Job.RUNNING) {
       if (s == Job.WAITING || s == Job.READY || s == Job.RUNNING) {
         break; // a pred is still not completed, continue in WAITING
         break; // a pred is still not completed, continue in WAITING
@@ -345,7 +368,7 @@ public class Job {
         }
         }
       }
       }
       RunningJob running = jc.submitJob(theJobConf);
       RunningJob running = jc.submitJob(theJobConf);
-      this.mapredJobID = running.getJobID();
+      this.mapredJobID = running.getID();
       this.state = Job.RUNNING;
       this.state = Job.RUNNING;
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       this.state = Job.FAILED;
       this.state = Job.FAILED;
@@ -353,12 +376,4 @@ public class Job {
     }
     }
   }
   }
 	
 	
-  /**
-   * @param args
-   */
-  public static void main(String[] args) {
-    // TODO Auto-generated method stub
-
-  }
-
 }
 }

+ 2 - 1
src/java/org/apache/hadoop/mapred/pipes/Application.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
 import org.apache.hadoop.mapred.TaskLog;
 import org.apache.hadoop.mapred.TaskLog;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -80,7 +81,7 @@ class Application<K1 extends WritableComparable, V1 extends Writable,
     FileUtil.chmod(executable, "a+x");
     FileUtil.chmod(executable, "a+x");
     cmd.add(executable);
     cmd.add(executable);
     // wrap the command in a stdout/stderr capture
     // wrap the command in a stdout/stderr capture
-    String taskid = conf.get("mapred.task.id");
+    TaskAttemptID taskid = TaskAttemptID.forName(conf.get("mapred.task.id"));
     File stdout = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.STDOUT);
     File stdout = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.STDOUT);
     File stderr = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.STDERR);
     File stderr = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.STDERR);
     long logLength = TaskLog.getTaskLogLength(conf);
     long logLength = TaskLog.getTaskLogLength(conf);

+ 11 - 9
src/test/org/apache/hadoop/mapred/TestJobStatusPersistency.java

@@ -17,15 +17,17 @@
  */
  */
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.util.Properties;
+
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.fs.Path;
-
-import java.io.*;
-import java.util.Properties;
 
 
 public class TestJobStatusPersistency extends ClusterMapReduceTestCase {
 public class TestJobStatusPersistency extends ClusterMapReduceTestCase {
-  private String runJob() throws Exception {
+  private JobID runJob() throws Exception {
     OutputStream os = getFileSystem().create(new Path(getInputDir(), "text.txt"));
     OutputStream os = getFileSystem().create(new Path(getInputDir(), "text.txt"));
     Writer wr = new OutputStreamWriter(os);
     Writer wr = new OutputStreamWriter(os);
     wr.write("hello1\n");
     wr.write("hello1\n");
@@ -53,11 +55,11 @@ public class TestJobStatusPersistency extends ClusterMapReduceTestCase {
 
 
     FileOutputFormat.setOutputPath(conf, getOutputDir());
     FileOutputFormat.setOutputPath(conf, getOutputDir());
 
 
-    return JobClient.runJob(conf).getJobID();
+    return JobClient.runJob(conf).getID();
   }
   }
 
 
   public void testNonPersistency() throws Exception {
   public void testNonPersistency() throws Exception {
-    String jobId = runJob();
+    JobID jobId = runJob();
     JobClient jc = new JobClient(createJobConf());
     JobClient jc = new JobClient(createJobConf());
     RunningJob rj = jc.getJob(jobId);
     RunningJob rj = jc.getJob(jobId);
     assertNotNull(rj);
     assertNotNull(rj);
@@ -74,7 +76,7 @@ public class TestJobStatusPersistency extends ClusterMapReduceTestCase {
     config.setProperty("mapred.job.tracker.persist.jobstatus.hours", "1");
     config.setProperty("mapred.job.tracker.persist.jobstatus.hours", "1");
     stopCluster();
     stopCluster();
     startCluster(false, config);
     startCluster(false, config);
-    String jobId = runJob();
+    JobID jobId = runJob();
     JobClient jc = new JobClient(createJobConf());
     JobClient jc = new JobClient(createJobConf());
     RunningJob rj0 = jc.getJob(jobId);
     RunningJob rj0 = jc.getJob(jobId);
     assertNotNull(rj0);
     assertNotNull(rj0);
@@ -96,7 +98,7 @@ public class TestJobStatusPersistency extends ClusterMapReduceTestCase {
     TaskCompletionEvent[] events1 = rj1.getTaskCompletionEvents(0);
     TaskCompletionEvent[] events1 = rj1.getTaskCompletionEvents(0);
     assertEquals(events0.length, events1.length);    
     assertEquals(events0.length, events1.length);    
     for (int i = 0; i < events0.length; i++) {
     for (int i = 0; i < events0.length; i++) {
-      assertEquals(events0[i].getTaskId(), events1[i].getTaskId());
+      assertEquals(events0[i].getTaskID(), events1[i].getTaskID());
       assertEquals(events0[i].getTaskStatus(), events1[i].getTaskStatus());
       assertEquals(events0[i].getTaskStatus(), events1[i].getTaskStatus());
     }
     }
   }
   }

+ 21 - 11
src/test/org/apache/hadoop/mapred/TestMapRed.java

@@ -17,14 +17,25 @@
  */
  */
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapred.lib.*;
+import java.io.BufferedWriter;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.Random;
+
 import junit.framework.TestCase;
 import junit.framework.TestCase;
-import java.io.*;
-import java.util.*;
 
 
-import static org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
 
 
 /**********************************************************
 /**********************************************************
  * MapredLoadTest generates a bunch of work that exercises
  * MapredLoadTest generates a bunch of work that exercises
@@ -255,15 +266,14 @@ public class TestMapRed extends TestCase {
   private static class MyReduce extends IdentityReducer {
   private static class MyReduce extends IdentityReducer {
     private JobConf conf;
     private JobConf conf;
     private boolean compressInput;
     private boolean compressInput;
-    private String taskId;
-    private String jobId;
+    private TaskAttemptID taskId;
     private boolean first = true;
     private boolean first = true;
       
       
+    @Override
     public void configure(JobConf conf) {
     public void configure(JobConf conf) {
       this.conf = conf;
       this.conf = conf;
       compressInput = conf.getCompressMapOutput();
       compressInput = conf.getCompressMapOutput();
-      taskId = conf.get("mapred.task.id");
-      jobId = conf.get("mapred.job.id");
+      taskId = TaskAttemptID.forName(conf.get("mapred.task.id"));
     }
     }
       
       
     public void reduce(WritableComparable key, Iterator values,
     public void reduce(WritableComparable key, Iterator values,
@@ -271,7 +281,7 @@ public class TestMapRed extends TestCase {
                        ) throws IOException {
                        ) throws IOException {
       if (first) {
       if (first) {
         first = false;
         first = false;
-        MapOutputFile mapOutputFile = new MapOutputFile(jobId);
+        MapOutputFile mapOutputFile = new MapOutputFile(taskId.getJobID());
         mapOutputFile.setConf(conf);
         mapOutputFile.setConf(conf);
         Path input = mapOutputFile.getInputFile(0, taskId);
         Path input = mapOutputFile.getInputFile(0, taskId);
         FileSystem fs = FileSystem.get(conf);
         FileSystem fs = FileSystem.get(conf);

+ 14 - 9
src/test/org/apache/hadoop/mapred/TestMiniMRLocalFS.java

@@ -18,20 +18,25 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapred.MRCaching.TestResult;
-import org.apache.hadoop.util.Progressable;
-
 import java.io.DataInput;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
-import java.io.IOException;
 import java.io.File;
 import java.io.File;
+import java.io.IOException;
 import java.util.Iterator;
 import java.util.Iterator;
+
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapred.MRCaching.TestResult;
+import org.apache.hadoop.util.Progressable;
+
 /**
 /**
  * A JUnit test to test min map-reduce cluster with local file system.
  * A JUnit test to test min map-reduce cluster with local file system.
  */
  */
@@ -64,7 +69,7 @@ public class TestMiniMRLocalFS extends TestCase {
       assertTrue("Failed test archives not matching", ret.isOutputOk);
       assertTrue("Failed test archives not matching", ret.isOutputOk);
       // test the task report fetchers
       // test the task report fetchers
       JobClient client = new JobClient(job);
       JobClient client = new JobClient(job);
-      String jobid = ret.job.getJobID();
+      JobID jobid = ret.job.getID();
       TaskReport[] reports = client.getMapTaskReports(jobid);
       TaskReport[] reports = client.getMapTaskReports(jobid);
       assertEquals("number of maps", 1, reports.length);
       assertEquals("number of maps", 1, reports.length);
       reports = client.getReduceTaskReports(jobid);
       reports = client.getReduceTaskReports(jobid);
@@ -247,7 +252,7 @@ public class TestMiniMRLocalFS extends TestCase {
         // expected result
         // expected result
       }
       }
       while (values.hasNext()) {
       while (values.hasNext()) {
-        Writable value = (Writable) values.next();
+        Writable value = values.next();
         System.out.println("reduce: " + key + ", " + value);
         System.out.println("reduce: " + key + ", " + value);
         output.collect(key, value);
         output.collect(key, value);
       }
       }

+ 13 - 11
src/test/org/apache/hadoop/mapred/TestMiniMRMapRedDebugScript.java

@@ -17,23 +17,25 @@
  */
  */
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import java.io.*;
-import java.util.*;
-import java.net.URISyntaxException;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.net.URI;
 import java.net.URI;
+import java.net.URISyntaxException;
+
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.filecache.DistributedCache;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
-import org.apache.hadoop.filecache.DistributedCache; 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.dfs.MiniDFSCluster;
 
 
 /**
 /**
  * Class to test mapred debug Script
  * Class to test mapred debug Script
@@ -67,7 +69,7 @@ public class TestMiniMRMapRedDebugScript extends TestCase {
    * @return task log as string
    * @return task log as string
    * @throws IOException
    * @throws IOException
    */
    */
-  public static String readTaskLog(TaskLog.LogName  filter, String taskId)
+  public static String readTaskLog(TaskLog.LogName  filter, TaskAttemptID taskId)
   throws IOException {
   throws IOException {
     // string buffer to store task log
     // string buffer to store task log
     StringBuffer result = new StringBuffer();
     StringBuffer result = new StringBuffer();
@@ -161,9 +163,9 @@ public class TestMiniMRMapRedDebugScript extends TestCase {
     	e.printStackTrace();
     	e.printStackTrace();
     }
     }
 
 
-    String jobId = job.getJobID();
+    JobID jobId = job.getID();
     // construct the task id of first map task of failmap
     // construct the task id of first map task of failmap
-    String taskId = "task_" + jobId.substring(4) + "_m_000000_0";
+    TaskAttemptID taskId = new TaskAttemptID(new TaskID(jobId,true, 0), 0);
     // wait for the job to finish.
     // wait for the job to finish.
     while (!job.isComplete()) ;
     while (!job.isComplete()) ;
     
     

+ 14 - 7
src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java

@@ -18,20 +18,27 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import java.io.*;
-import java.util.*;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.examples.WordCount;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.examples.WordCount;
 
 
 /**
 /**
  * A JUnit test to test Mini Map-Reduce Cluster with Mini-DFS.
  * A JUnit test to test Mini Map-Reduce Cluster with Mini-DFS.
@@ -174,7 +181,7 @@ public class TestMiniMRWithDFS extends TestCase {
     LOG.info("runWordCount");
     LOG.info("runWordCount");
     // Run a word count example
     // Run a word count example
     // Keeping tasks that match this pattern
     // Keeping tasks that match this pattern
-    jobConf.setKeepTaskFilesPattern("task_[^_]*_[0-9]*_m_000001_.*");
+    jobConf.setKeepTaskFilesPattern(TaskAttemptID.getTaskAttemptIDsPattern(null, null, true, 1, null));
     TestResult result;
     TestResult result;
     final Path inDir = new Path("./wc/input");
     final Path inDir = new Path("./wc/input");
     final Path outDir = new Path("./wc/output");
     final Path outDir = new Path("./wc/output");
@@ -182,9 +189,9 @@ public class TestMiniMRWithDFS extends TestCase {
     result = launchWordCount(jobConf, inDir, outDir, input, 3, 1);
     result = launchWordCount(jobConf, inDir, outDir, input, 3, 1);
     assertEquals("The\t1\nbrown\t1\nfox\t2\nhas\t1\nmany\t1\n" +
     assertEquals("The\t1\nbrown\t1\nfox\t2\nhas\t1\nmany\t1\n" +
                  "quick\t1\nred\t1\nsilly\t1\nsox\t1\n", result.output);
                  "quick\t1\nred\t1\nsilly\t1\nsox\t1\n", result.output);
-    String jobid = result.job.getJobID();
-    String taskid = "task_" + jobid.substring(4) + "_m_000001_0";
-    checkTaskDirectories(mr, new String[]{jobid}, new String[]{taskid});
+    JobID jobid = result.job.getID();
+    TaskAttemptID taskid = new TaskAttemptID(new TaskID(jobid, true, 1),0);
+    checkTaskDirectories(mr, new String[]{jobid.toString()}, new String[]{taskid.toString()});
     // test with maps=0
     // test with maps=0
     jobConf = mr.createJobConf();
     jobConf = mr.createJobConf();
     input = "owen is oom";
     input = "owen is oom";

+ 1 - 1
src/webapps/job/jobblacklistedtrackers.jsp

@@ -37,7 +37,7 @@
   	  return;
   	  return;
     }
     }
     
     
-    JobInProgress job = (JobInProgress) tracker.getJob(jobId);
+    JobInProgress job = (JobInProgress) tracker.getJob(JobID.forName(jobId));
     if (job == null) {
     if (job == null) {
       out.print("<b>Job " + jobId + " not found.</b><br>\n");
       out.print("<b>Job " + jobId + " not found.</b><br>\n");
       return;
       return;

+ 1 - 1
src/webapps/job/jobconf.jsp

@@ -26,7 +26,7 @@
 <h2>Job Configuration: JobId - <%= jobId %></h2><br>
 <h2>Job Configuration: JobId - <%= jobId %></h2><br>
 
 
 <%
 <%
-  String jobFilePath = tracker.getLocalJobFilePath(jobId);
+  String jobFilePath = JobTracker.getLocalJobFilePath(JobID.forName(jobId));
   FileInputStream jobFile = null;
   FileInputStream jobFile = null;
   try {
   try {
     jobFile = new FileInputStream(jobFilePath);
     jobFile = new FileInputStream(jobFilePath);

+ 8 - 4
src/webapps/job/jobdetails.jsp

@@ -104,6 +104,10 @@
 <%   
 <%   
     String jobId = request.getParameter("jobid"); 
     String jobId = request.getParameter("jobid"); 
     String refreshParam = request.getParameter("refresh");
     String refreshParam = request.getParameter("refresh");
+    if (jobId == null) {
+      out.println("<h2>Missing 'jobid'!</h2>");
+      return;
+    }
     
     
     int refresh = 60; // refresh every 60 seconds by default
     int refresh = 60; // refresh every 60 seconds by default
     if (refreshParam != null) {
     if (refreshParam != null) {
@@ -113,12 +117,12 @@
         catch (NumberFormatException ignored) {
         catch (NumberFormatException ignored) {
         }
         }
     }
     }
-    
-    JobInProgress job = (JobInProgress) tracker.getJob(jobId);
+    JobID jobIdObj = JobID.forName(jobId);
+    JobInProgress job = (JobInProgress) tracker.getJob(jobIdObj);
     
     
     String action = request.getParameter("action");
     String action = request.getParameter("action");
     if("changeprio".equalsIgnoreCase(action)) {
     if("changeprio".equalsIgnoreCase(action)) {
-      tracker.setJobPriority(jobId, 
+      tracker.setJobPriority(jobIdObj, 
                              JobPriority.valueOf(request.getParameter("prio")));
                              JobPriority.valueOf(request.getParameter("prio")));
     }
     }
     
     
@@ -129,7 +133,7 @@
     	    return;
     	    return;
 	    }
 	    }
   	    else if(action != null && action.equalsIgnoreCase("kill")) {
   	    else if(action != null && action.equalsIgnoreCase("kill")) {
-	      tracker.killJob(jobId);
+	      tracker.killJob(jobIdObj);
 	    }
 	    }
     }
     }
 %>
 %>

+ 12 - 8
src/webapps/job/jobfailures.jsp

@@ -16,18 +16,18 @@
 <%! 
 <%! 
   private void printFailedAttempts(JspWriter out,
   private void printFailedAttempts(JspWriter out,
                                    JobTracker tracker,
                                    JobTracker tracker,
-                                   String jobId,
+                                   JobID jobId,
                                    TaskInProgress tip,
                                    TaskInProgress tip,
                                    TaskStatus.State failState) throws IOException {
                                    TaskStatus.State failState) throws IOException {
     TaskStatus[] statuses = tip.getTaskStatuses();
     TaskStatus[] statuses = tip.getTaskStatuses();
-    String tipId = tip.getTIPId();
+    TaskID tipId = tip.getTIPId();
     for(int i=0; i < statuses.length; ++i) {
     for(int i=0; i < statuses.length; ++i) {
       TaskStatus.State taskState = statuses[i].getRunState();
       TaskStatus.State taskState = statuses[i].getRunState();
       if ((failState == null && (taskState == TaskStatus.State.FAILED || 
       if ((failState == null && (taskState == TaskStatus.State.FAILED || 
           taskState == TaskStatus.State.KILLED)) || taskState == failState) {
           taskState == TaskStatus.State.KILLED)) || taskState == failState) {
         String taskTrackerName = statuses[i].getTaskTracker();
         String taskTrackerName = statuses[i].getTaskTracker();
         TaskTrackerStatus taskTracker = tracker.getTaskTracker(taskTrackerName);
         TaskTrackerStatus taskTracker = tracker.getTaskTracker(taskTrackerName);
-        out.print("<tr><td>" + statuses[i].getTaskId() +
+        out.print("<tr><td>" + statuses[i].getTaskID() +
                   "</td><td><a href=\"taskdetails.jsp?jobid="+ jobId + 
                   "</td><td><a href=\"taskdetails.jsp?jobid="+ jobId + 
                   "&tipid=" + tipId + "\">" + tipId +
                   "&tipid=" + tipId + "\">" + tipId +
                   "</a></td>");
                   "</a></td>");
@@ -41,8 +41,7 @@
         out.print("<td>" + taskState + "</td>");
         out.print("<td>" + taskState + "</td>");
         out.print("<td><pre>");
         out.print("<td><pre>");
         String[] failures = 
         String[] failures = 
-                     tracker.getTaskDiagnostics(jobId, tipId, 
-                                                statuses[i].getTaskId());
+                     tracker.getTaskDiagnostics(statuses[i].getTaskID());
         if (failures == null) {
         if (failures == null) {
           out.print("&nbsp;");
           out.print("&nbsp;");
         } else {
         } else {
@@ -59,7 +58,7 @@
         if (taskTracker != null) {
         if (taskTracker != null) {
           String taskLogUrl = "http://" + taskTracker.getHost() + ":" +
           String taskLogUrl = "http://" + taskTracker.getHost() + ":" +
           	taskTracker.getHttpPort() + "/tasklog?taskid=" + 
           	taskTracker.getHttpPort() + "/tasklog?taskid=" + 
-          	statuses[i].getTaskId();
+          	statuses[i].getTaskID();
           String tailFourKBUrl = taskLogUrl + "&start=-4097";
           String tailFourKBUrl = taskLogUrl + "&start=-4097";
           String tailEightKBUrl = taskLogUrl + "&start=-8193";
           String tailEightKBUrl = taskLogUrl + "&start=-8193";
           String entireLogUrl = taskLogUrl;
           String entireLogUrl = taskLogUrl;
@@ -78,7 +77,7 @@
              
              
   private void printFailures(JspWriter out, 
   private void printFailures(JspWriter out, 
                              JobTracker tracker,
                              JobTracker tracker,
-                             String jobId,
+                             JobID jobId,
                              String kind, 
                              String kind, 
                              String cause) throws IOException {
                              String cause) throws IOException {
     JobInProgress job = (JobInProgress) tracker.getJob(jobId);
     JobInProgress job = (JobInProgress) tracker.getJob(jobId);
@@ -140,6 +139,11 @@
 
 
 <%
 <%
     String jobId = request.getParameter("jobid");
     String jobId = request.getParameter("jobid");
+    if (jobId == null) {
+      out.println("<h2>Missing 'jobid'!</h2>");
+      return;
+    }
+    JobID jobIdObj = JobID.forName(jobId);
     String kind = request.getParameter("kind");
     String kind = request.getParameter("kind");
     String cause = request.getParameter("cause");
     String cause = request.getParameter("cause");
 %>
 %>
@@ -151,7 +155,7 @@
 failures on <a href="jobtracker.jsp"><%=trackerName%></a></h1>
 failures on <a href="jobtracker.jsp"><%=trackerName%></a></h1>
 
 
 <% 
 <% 
-    printFailures(out, tracker, jobId, kind, cause); 
+    printFailures(out, tracker, jobIdObj, kind, cause); 
 %>
 %>
 
 
 <hr>
 <hr>

+ 10 - 5
src/webapps/job/jobtasks.jsp

@@ -15,6 +15,10 @@
   String trackerName = 
   String trackerName = 
            StringUtils.simpleHostname(tracker.getJobTrackerMachine());
            StringUtils.simpleHostname(tracker.getJobTrackerMachine());
   String jobid = request.getParameter("jobid");
   String jobid = request.getParameter("jobid");
+  if (jobid == null) {
+    out.println("<h2>Missing 'jobid'!</h2>");
+    return;
+  }
   String type = request.getParameter("type");
   String type = request.getParameter("type");
   String pagenum = request.getParameter("pagenum");
   String pagenum = request.getParameter("pagenum");
   TaskInProgress[] tasks;
   TaskInProgress[] tasks;
@@ -23,7 +27,8 @@
   int pnum = Integer.parseInt(pagenum);
   int pnum = Integer.parseInt(pagenum);
   int next_page = pnum+1;
   int next_page = pnum+1;
   int numperpage = 2000;
   int numperpage = 2000;
-  JobInProgress job = (JobInProgress) tracker.getJob(jobid);
+  JobID jobidObj = JobID.forName(jobid);
+  JobInProgress job = (JobInProgress) tracker.getJob(jobidObj);
   JobProfile profile = (job != null) ? (job.getProfile()) : null;
   JobProfile profile = (job != null) ? (job.getProfile()) : null;
   JobStatus status = (job != null) ? (job.getStatus()) : null;
   JobStatus status = (job != null) ? (job.getStatus()) : null;
   TaskReport[] reports = null;
   TaskReport[] reports = null;
@@ -31,11 +36,11 @@
   int end_index = start_index + numperpage;
   int end_index = start_index + numperpage;
   int report_len = 0;
   int report_len = 0;
   if ("map".equals(type)){
   if ("map".equals(type)){
-     reports = (job != null) ? tracker.getMapTaskReports(jobid) : null;
+     reports = (job != null) ? tracker.getMapTaskReports(jobidObj) : null;
      tasks = (job != null) ? job.getMapTasks() : null;
      tasks = (job != null) ? job.getMapTasks() : null;
     }
     }
   else{
   else{
-    reports = (job != null) ? tracker.getReduceTaskReports(jobid) : null;
+    reports = (job != null) ? tracker.getReduceTaskReports(jobidObj) : null;
     tasks = (job != null) ? job.getReduceTasks() : null;
     tasks = (job != null) ? job.getReduceTasks() : null;
   }
   }
 %>
 %>
@@ -57,7 +62,7 @@
   }
   }
   // Filtering the reports if some filter is specified
   // Filtering the reports if some filter is specified
   if (!"all".equals(state)) {
   if (!"all".equals(state)) {
-    List<String> filteredReportsTaskIds = new ArrayList<String>();
+    List<TaskID> filteredReportsTaskIds = new ArrayList<TaskID>();
     List<TaskReport> filteredReports = new ArrayList<TaskReport>();
     List<TaskReport> filteredReports = new ArrayList<TaskReport>();
     for (int i = 0; i < tasks.length; ++i) {
     for (int i = 0; i < tasks.length; ++i) {
       if (("completed".equals(state) && tasks[i].isComplete()) 
       if (("completed".equals(state) && tasks[i].isComplete()) 
@@ -70,7 +75,7 @@
       }
       }
     }
     }
     for (int i = 0 ; i < reports.length; ++i) {
     for (int i = 0 ; i < reports.length; ++i) {
-      if (filteredReportsTaskIds.contains(reports[i].getTaskId())) {
+      if (filteredReportsTaskIds.contains(reports[i].getTaskID())) {
         filteredReports.add(reports[i]);
         filteredReports.add(reports[i]);
       }
       }
     }
     }

+ 1 - 1
src/webapps/job/jobtracker.jsp

@@ -33,7 +33,7 @@
           JobInProgress job = (JobInProgress) it.next();
           JobInProgress job = (JobInProgress) it.next();
           JobProfile profile = job.getProfile();
           JobProfile profile = job.getProfile();
           JobStatus status = job.getStatus();
           JobStatus status = job.getStatus();
-          String jobid = profile.getJobId();
+          JobID jobid = profile.getJobID();
 
 
           int desiredMaps = job.desiredMaps();
           int desiredMaps = job.desiredMaps();
           int desiredReduces = job.desiredReduces();
           int desiredReduces = job.desiredReduces();

+ 15 - 12
src/webapps/job/taskdetails.jsp

@@ -29,10 +29,14 @@
 <%
 <%
     JobTracker tracker = (JobTracker) application.getAttribute("job.tracker");
     JobTracker tracker = (JobTracker) application.getAttribute("job.tracker");
     String jobid = request.getParameter("jobid");
     String jobid = request.getParameter("jobid");
-    JobInProgress job = (JobInProgress) tracker.getJob(jobid);
     String tipid = request.getParameter("tipid");
     String tipid = request.getParameter("tipid");
     String taskid = request.getParameter("taskid");
     String taskid = request.getParameter("taskid");
-
+    JobID jobidObj = JobID.forName(jobid);
+    TaskID tipidObj = TaskID.forName(tipid);
+    TaskAttemptID taskidObj = TaskAttemptID.forName(taskid);
+    
+    JobInProgress job = (JobInProgress) tracker.getJob(jobidObj);
+    
     boolean privateActions = JspHelper.conf.getBoolean(PRIVATE_ACTIONS_KEY,
     boolean privateActions = JspHelper.conf.getBoolean(PRIVATE_ACTIONS_KEY,
         false);
         false);
     if (privateActions) {
     if (privateActions) {
@@ -46,19 +50,19 @@
           return;
           return;
         }
         }
         else if (action.equalsIgnoreCase("kill-task")) {
         else if (action.equalsIgnoreCase("kill-task")) {
-          tracker.killTask(taskid, false);
+          tracker.killTask(taskidObj, false);
           //redirect again so that refreshing the page will not attempt to rekill the task
           //redirect again so that refreshing the page will not attempt to rekill the task
           response.sendRedirect("/taskdetails.jsp?" + "&subaction=kill-task"
           response.sendRedirect("/taskdetails.jsp?" + "&subaction=kill-task"
               + "&jobid=" + jobid + "&tipid=" + tipid);
               + "&jobid=" + jobid + "&tipid=" + tipid);
         }
         }
         else if (action.equalsIgnoreCase("fail-task")) {
         else if (action.equalsIgnoreCase("fail-task")) {
-          tracker.killTask(taskid, true);
+          tracker.killTask(taskidObj, true);
           response.sendRedirect("/taskdetails.jsp?" + "&subaction=fail-task"
           response.sendRedirect("/taskdetails.jsp?" + "&subaction=fail-task"
               + "&jobid=" + jobid + "&tipid=" + tipid);
               + "&jobid=" + jobid + "&tipid=" + tipid);
         }
         }
       }
       }
     }
     }
-    TaskStatus[] ts = (job != null) ? tracker.getTaskStatuses(jobid, tipid)
+    TaskStatus[] ts = (job != null) ? tracker.getTaskStatuses(tipidObj)
         : null;
         : null;
 %>
 %>
 
 
@@ -97,7 +101,7 @@
       TaskStatus status = ts[i];
       TaskStatus status = ts[i];
       String taskTrackerName = status.getTaskTracker();
       String taskTrackerName = status.getTaskTracker();
       TaskTrackerStatus taskTracker = tracker.getTaskTracker(taskTrackerName);
       TaskTrackerStatus taskTracker = tracker.getTaskTracker(taskTrackerName);
-      out.print("<tr><td>" + status.getTaskId() + "</td>");
+      out.print("<tr><td>" + status.getTaskID() + "</td>");
       String taskAttemptTracker = null;
       String taskAttemptTracker = null;
       if (taskTracker == null) {
       if (taskTracker == null) {
         out.print("<td>" + taskTrackerName + "</td>");
         out.print("<td>" + taskTrackerName + "</td>");
@@ -127,8 +131,7 @@
           .getFinishTime(), status.getStartTime()) + "</td>");
           .getFinishTime(), status.getStartTime()) + "</td>");
 
 
         out.print("<td><pre>");
         out.print("<td><pre>");
-        String [] failures = tracker.getTaskDiagnostics(jobid, tipid,
-          status.getTaskId());
+        String [] failures = tracker.getTaskDiagnostics(status.getTaskID());
         if (failures == null) {
         if (failures == null) {
           out.print("&nbsp;");
           out.print("&nbsp;");
         } else {
         } else {
@@ -145,7 +148,7 @@
           out.print("n/a");
           out.print("n/a");
         } else {
         } else {
           String taskLogUrl = taskAttemptTracker + "/tasklog?taskid="
           String taskLogUrl = taskAttemptTracker + "/tasklog?taskid="
-            + status.getTaskId();
+            + status.getTaskID();
           String tailFourKBUrl = taskLogUrl + "&start=-4097";
           String tailFourKBUrl = taskLogUrl + "&start=-4097";
           String tailEightKBUrl = taskLogUrl + "&start=-8193";
           String tailEightKBUrl = taskLogUrl + "&start=-8193";
           String entireLogUrl = taskLogUrl + "&all=true";
           String entireLogUrl = taskLogUrl + "&all=true";
@@ -154,17 +157,17 @@
           out.print("<a href=\"" + entireLogUrl + "\">All</a><br/>");
           out.print("<a href=\"" + entireLogUrl + "\">All</a><br/>");
         }
         }
         out.print("</td><td>" + "<a href=\"/taskstats.jsp?jobid=" + jobid
         out.print("</td><td>" + "<a href=\"/taskstats.jsp?jobid=" + jobid
-          + "&tipid=" + tipid + "&taskid=" + status.getTaskId() + "\">"
+          + "&tipid=" + tipid + "&taskid=" + status.getTaskID() + "\">"
           + ((status.getCounters() != null) ? status.getCounters().size() : 0) + "</a></td>");
           + ((status.getCounters() != null) ? status.getCounters().size() : 0) + "</a></td>");
         out.print("<td>");
         out.print("<td>");
         if (privateActions
         if (privateActions
           && status.getRunState() == TaskStatus.State.RUNNING) {
           && status.getRunState() == TaskStatus.State.RUNNING) {
         out.print("<a href=\"/taskdetails.jsp?action=confirm"
         out.print("<a href=\"/taskdetails.jsp?action=confirm"
           + "&subaction=kill-task" + "&jobid=" + jobid + "&tipid="
           + "&subaction=kill-task" + "&jobid=" + jobid + "&tipid="
-          + tipid + "&taskid=" + status.getTaskId() + "\" > Kill </a>");
+          + tipid + "&taskid=" + status.getTaskID() + "\" > Kill </a>");
         out.print("<br><a href=\"/taskdetails.jsp?action=confirm"
         out.print("<br><a href=\"/taskdetails.jsp?action=confirm"
           + "&subaction=fail-task" + "&jobid=" + jobid + "&tipid="
           + "&subaction=fail-task" + "&jobid=" + jobid + "&tipid="
-          + tipid + "&taskid=" + status.getTaskId() + "\" > Fail </a>");
+          + tipid + "&taskid=" + status.getTaskID() + "\" > Fail </a>");
         }
         }
         else
         else
           out.print("<pre>&nbsp;</pre>");
           out.print("<pre>&nbsp;</pre>");

+ 8 - 3
src/webapps/job/taskstats.jsp

@@ -15,17 +15,22 @@
   String trackerName = 
   String trackerName = 
            StringUtils.simpleHostname(tracker.getJobTrackerMachine());
            StringUtils.simpleHostname(tracker.getJobTrackerMachine());
   String jobid = request.getParameter("jobid");
   String jobid = request.getParameter("jobid");
-  JobInProgress job = (JobInProgress) tracker.getJob(jobid);
   String tipid = request.getParameter("tipid");
   String tipid = request.getParameter("tipid");
   String taskid = request.getParameter("taskid");
   String taskid = request.getParameter("taskid");
+  JobID jobidObj = JobID.forName(jobid);
+  TaskID tipidObj = TaskID.forName(tipid);
+  TaskAttemptID taskidObj = TaskAttemptID.forName(taskid);
+  
+  JobInProgress job = (JobInProgress) tracker.getJob(jobidObj);
+  
   Format decimal = new DecimalFormat();
   Format decimal = new DecimalFormat();
   Counters counters;
   Counters counters;
   if (taskid == null) {
   if (taskid == null) {
-    counters = tracker.getTipCounters(jobid, tipid);
+    counters = tracker.getTipCounters(tipidObj);
     taskid = tipid; // for page title etc
     taskid = tipid; // for page title etc
   }
   }
   else {
   else {
-    TaskStatus taskStatus = tracker.getTaskStatus(jobid, tipid, taskid);
+    TaskStatus taskStatus = tracker.getTaskStatus(taskidObj);
     counters = taskStatus.getCounters();
     counters = taskStatus.getCounters();
   }
   }
 %>
 %>

+ 2 - 2
src/webapps/task/tasktracker.jsp

@@ -35,7 +35,7 @@
      Iterator itr = tracker.getRunningTaskStatuses().iterator();
      Iterator itr = tracker.getRunningTaskStatuses().iterator();
      while (itr.hasNext()) {
      while (itr.hasNext()) {
        TaskStatus status = (TaskStatus) itr.next();
        TaskStatus status = (TaskStatus) itr.next();
-       out.print("<tr><td>" + status.getTaskId());
+       out.print("<tr><td>" + status.getTaskID());
        out.print("</td><td>" + status.getRunState()); 
        out.print("</td><td>" + status.getRunState()); 
        out.print("</td><td>" + 
        out.print("</td><td>" + 
                  StringUtils.formatPercent(status.getProgress(), 2));
                  StringUtils.formatPercent(status.getProgress(), 2));
@@ -51,7 +51,7 @@
 <tr><td align="center">Task Attempts</td><td>Status</td>
 <tr><td align="center">Task Attempts</td><td>Status</td>
   <%
   <%
     for(TaskStatus status: tracker.getNonRunningTasks()) {
     for(TaskStatus status: tracker.getNonRunningTasks()) {
-      out.print("<tr><td>" + status.getTaskId() + "</td>");
+      out.print("<tr><td>" + status.getTaskID() + "</td>");
       out.print("<td>" + status.getRunState() + "</td></tr>\n");
       out.print("<td>" + status.getRunState() + "</td></tr>\n");
     }
     }
   %>
   %>