diff --git src/core/org/apache/hadoop/conf/Configuration.java src/core/org/apache/hadoop/conf/Configuration.java
index 441173b..3a1a6c7 100644
--- src/core/org/apache/hadoop/conf/Configuration.java
+++ src/core/org/apache/hadoop/conf/Configuration.java
@@ -352,6 +352,17 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     getProps().setProperty(name, value);
   }
   
+  /**
+   * Sets a property if it is currently unset.
+   * @param name the property name
+   * @param value the new value
+   */
+  public void setIfUnset(String name, String value) {
+    if (get(name) == null) {
+      set(name, value);
+    }
+  }
+  
   private synchronized Properties getOverlay() {
     if (overlay==null){
       overlay=new Properties();
@@ -522,6 +533,15 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   }
 
   /**
+   * Set the given property, if it is currently unset.
+   * @param name property name
+   * @param value new value
+   */
+  public void setBooleanIfUnset(String name, boolean value) {
+    setIfUnset(name, Boolean.toString(value));
+  }
+
+  /**
    * A class that represents a set of positive integer ranges. It parses 
    * strings of the form: "2-3,5,7-" where ranges are separated by comma and 
    * the lower/upper bounds are separated by dash. Either the lower or upper 
diff --git src/core/org/apache/hadoop/io/WritableUtils.java src/core/org/apache/hadoop/io/WritableUtils.java
index d87df52..b1341c1 100644
--- src/core/org/apache/hadoop/io/WritableUtils.java
+++ src/core/org/apache/hadoop/io/WritableUtils.java
@@ -197,30 +197,6 @@ public final class WritableUtils  {
   }
 
   /**
-   * A pair of input/output buffers that we use to clone writables.
-   */
-  private static class CopyInCopyOutBuffer {
-    DataOutputBuffer outBuffer = new DataOutputBuffer();
-    DataInputBuffer inBuffer = new DataInputBuffer();
-    /**
-     * Move the data from the output buffer to the input buffer.
-     */
-    void moveData() {
-      inBuffer.reset(outBuffer.getData(), outBuffer.getLength());
-    }
-  }
-  
-  /**
-   * Allocate a buffer for each thread that tries to clone objects.
-   */
-  private static ThreadLocal<CopyInCopyOutBuffer> cloneBuffers
-      = new ThreadLocal<CopyInCopyOutBuffer>() {
-      protected synchronized CopyInCopyOutBuffer initialValue() {
-        return new CopyInCopyOutBuffer();
-      }
-    };
-
-  /**
    * Make a copy of a writable object using serialization to a buffer.
    * @param orig The object to copy
    * @return The copied object
@@ -229,7 +205,7 @@ public final class WritableUtils  {
     try {
       @SuppressWarnings("unchecked") // Unchecked cast from Class to Class<T>
       T newInst = ReflectionUtils.newInstance((Class<T>) orig.getClass(), conf);
-      cloneInto(newInst, orig);
+      ReflectionUtils.cloneInto(conf, newInst, orig);
       return newInst;
     } catch (IOException e) {
       throw new RuntimeException("Error writing/reading clone buffer", e);
@@ -241,14 +217,11 @@ public final class WritableUtils  {
    * @param dst the object to copy from
    * @param src the object to copy into, which is destroyed
    * @throws IOException
+   * @deprecated use ReflectionUtils.cloneInto instead.
    */
+  @Deprecated
   public static void cloneInto(Writable dst, Writable src) throws IOException {
-    CopyInCopyOutBuffer buffer = cloneBuffers.get();
-    buffer.outBuffer.reset();
-    src.write(buffer.outBuffer);
-    buffer.moveData();
-    dst.readFields(buffer.inBuffer);
-    return;
+    ReflectionUtils.cloneWritableInto(dst, src);
   }
 
   /**
diff --git src/core/org/apache/hadoop/util/GenericOptionsParser.java src/core/org/apache/hadoop/util/GenericOptionsParser.java
index 3e43d13..2832378 100644
--- src/core/org/apache/hadoop/util/GenericOptionsParser.java
+++ src/core/org/apache/hadoop/util/GenericOptionsParser.java
@@ -104,9 +104,26 @@ import org.apache.hadoop.fs.Path;
 public class GenericOptionsParser {
 
   private static final Log LOG = LogFactory.getLog(GenericOptionsParser.class);
-
+  private Configuration conf;
   private CommandLine commandLine;
 
+  /**
+   * Create an options parser with the given options to parse the args.
+   * @param opts the options
+   * @param args the command line arguments
+   */
+  public GenericOptionsParser(Options opts, String[] args) {
+    this(new Configuration(), new Options(), args);
+  }
+
+  /**
+   * Create an options parser to parse the args.
+   * @param args the command line arguments
+   */
+  public GenericOptionsParser(String[] args) {
+    this(new Configuration(), new Options(), args);
+  }
+  
   /** 
    * Create a <code>GenericOptionsParser<code> to parse only the generic Hadoop  
    * arguments. 
@@ -134,6 +151,7 @@ public class GenericOptionsParser {
    */
   public GenericOptionsParser(Configuration conf, Options options, String[] args) {
     parseGeneralOptions(options, conf, args);
+    this.conf = conf;
   }
 
   /**
@@ -147,6 +165,14 @@ public class GenericOptionsParser {
   }
 
   /**
+   * Get the modified configuration
+   * @return the configuration that has the modified parameters.
+   */
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  /**
    * Returns the commons-cli <code>CommandLine</code> object 
    * to process the parsed arguments. 
    * 
diff --git src/core/org/apache/hadoop/util/ReflectionUtils.java src/core/org/apache/hadoop/util/ReflectionUtils.java
index 519edbd..4226ddb 100644
--- src/core/org/apache/hadoop/util/ReflectionUtils.java
+++ src/core/org/apache/hadoop/util/ReflectionUtils.java
@@ -27,6 +27,12 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.*;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
 
 /**
  * General reflection utils
@@ -34,7 +40,9 @@ import org.apache.hadoop.conf.*;
 
 public class ReflectionUtils {
     
-  private static final Class[] emptyArray = new Class[]{};
+  private static final Class<?>[] emptyArray = new Class[]{};
+  private static SerializationFactory serialFactory = null;
+
   /** 
    * Cache of constructors for each class. Pins the classes so they
    * can't be garbage collected until ReflectionUtils can be collected.
@@ -217,5 +225,67 @@ public class ReflectionUtils {
   static int getCacheSize() {
     return CONSTRUCTOR_CACHE.size();
   }
+  /**
+   * A pair of input/output buffers that we use to clone writables.
+   */
+  private static class CopyInCopyOutBuffer {
+    DataOutputBuffer outBuffer = new DataOutputBuffer();
+    DataInputBuffer inBuffer = new DataInputBuffer();
+    /**
+     * Move the data from the output buffer to the input buffer.
+     */
+    void moveData() {
+      inBuffer.reset(outBuffer.getData(), outBuffer.getLength());
+    }
+  }
+  
+  /**
+   * Allocate a buffer for each thread that tries to clone objects.
+   */
+  private static ThreadLocal<CopyInCopyOutBuffer> cloneBuffers
+      = new ThreadLocal<CopyInCopyOutBuffer>() {
+      protected synchronized CopyInCopyOutBuffer initialValue() {
+        return new CopyInCopyOutBuffer();
+      }
+    };
+
+  private static SerializationFactory getFactory(Configuration conf) {
+    if (serialFactory == null) {
+      serialFactory = new SerializationFactory(conf);
+    }
+    return serialFactory;
+  }
+  
+  /**
+   * Make a copy of the writable object using serialiation to a buffer
+   * @param dst the object to copy from
+   * @param src the object to copy into, which is destroyed
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> T cloneInto(Configuration conf, 
+                                T dst, T src) throws IOException {
+    CopyInCopyOutBuffer buffer = cloneBuffers.get();
+    buffer.outBuffer.reset();
+    SerializationFactory factory = getFactory(conf);
+    Class<T> cls = (Class<T>) src.getClass();
+    Serializer<T> serializer = factory.getSerializer(cls);
+    serializer.open(buffer.outBuffer);
+    serializer.serialize(src);
+    buffer.moveData();
+    Deserializer<T> deserializer = factory.getDeserializer(cls);
+    deserializer.open(buffer.inBuffer);
+    dst = deserializer.deserialize(dst);
+    return dst;
+  }
 
+  @Deprecated
+  public static void cloneWritableInto(Writable dst, 
+                                       Writable src) throws IOException {
+    CopyInCopyOutBuffer buffer = cloneBuffers.get();
+    buffer.outBuffer.reset();
+    src.write(buffer.outBuffer);
+    buffer.moveData();
+    dst.readFields(buffer.inBuffer);
+  }
 }
diff --git src/examples/org/apache/hadoop/examples/ExampleDriver.java src/examples/org/apache/hadoop/examples/ExampleDriver.java
index 78c0fb8..b177b83 100644
--- src/examples/org/apache/hadoop/examples/ExampleDriver.java
+++ src/examples/org/apache/hadoop/examples/ExampleDriver.java
@@ -36,6 +36,8 @@ public class ExampleDriver {
     try {
       pgd.addClass("wordcount", WordCount.class, 
                    "A map/reduce program that counts the words in the input files.");
+      pgd.addClass("newwordcount", NewWordCount.class, 
+                   "A map/reduce program that counts the words in the input files.");
       pgd.addClass("aggregatewordcount", AggregateWordCount.class, 
                    "An Aggregate based map/reduce program that counts the words in the input files.");
       pgd.addClass("aggregatewordhist", AggregateWordHistogram.class, 
diff --git src/examples/org/apache/hadoop/examples/NewWordCount.java src/examples/org/apache/hadoop/examples/NewWordCount.java
new file mode 100644
index 0000000..e938df3
--- /dev/null
+++ src/examples/org/apache/hadoop/examples/NewWordCount.java
@@ -0,0 +1,69 @@
+package org.apache.hadoop.examples;
+
+import java.io.IOException;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.GenericOptionsParser;
+
+public class NewWordCount {
+
+  public static class TokenizerMapper 
+       extends Mapper<Object, Text, Text, IntWritable>{
+    
+    private final static IntWritable one = new IntWritable(1);
+    private Text word = new Text();
+      
+    public void map(Object key, Text value, Context context
+                    ) throws IOException, InterruptedException {
+      StringTokenizer itr = new StringTokenizer(value.toString());
+      while (itr.hasMoreTokens()) {
+        word.set(itr.nextToken());
+        context.write(word, one);
+      }
+    }
+  }
+  
+  public static class IntSumReducer 
+       extends Reducer<Text,IntWritable,Text,IntWritable> {
+    private IntWritable result = new IntWritable();
+
+    public void reduce(Text key, Iterable<IntWritable> values, 
+                       Context context
+                       ) throws IOException, InterruptedException {
+      int sum = 0;
+      for (IntWritable val : values) {
+        sum += val.get();
+      }
+      result.set(sum);
+      context.write(key, result);
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = new Configuration();
+    String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
+    if (otherArgs.length != 2) {
+      System.err.println("Usage: wordcount <in> <out>");
+      System.exit(2);
+    }
+    Job job = new Job(conf, "word count");
+    job.setJarByClass(NewWordCount.class);
+    job.setMapperClass(TokenizerMapper.class);
+    job.setCombinerClass(IntSumReducer.class);
+    job.setReducerClass(IntSumReducer.class);
+    job.setOutputKeyClass(Text.class);
+    job.setOutputValueClass(IntWritable.class);
+    FileInputFormat.addInputPath(job, new Path(otherArgs[0]));
+    FileOutputFormat.setOutputPath(job, new Path(otherArgs[1]));
+    System.exit(job.waitForCompletion() ? 0 : 1);
+  }
+}
diff --git src/mapred/org/apache/hadoop/mapred/Counters.java src/mapred/org/apache/hadoop/mapred/Counters.java
index c41cb7e..d9d5e91 100644
--- src/mapred/org/apache/hadoop/mapred/Counters.java
+++ src/mapred/org/apache/hadoop/mapred/Counters.java
@@ -65,69 +65,18 @@ public class Counters implements Writable, Iterable<Counters.Group> {
   /**
    * A counter record, comprising its name and value. 
    */
-  public static class Counter implements Writable {
-
-    private String name;
-    private String displayName;
-    private long value;
+  public static class Counter extends org.apache.hadoop.mapreduce.Counter {
     
     Counter() { 
-      value = 0L;
     }
 
     Counter(String name, String displayName, long value) {
-      this.name = name;
-      this.displayName = displayName;
-      this.value = value;
-    }
-    
-    /**
-     * Read the binary representation of the counter
-     */
-    public synchronized void readFields(DataInput in) throws IOException {
-      name = Text.readString(in);
-      if (in.readBoolean()) {
-        displayName = Text.readString(in);
-      } else {
-        displayName = name;
-      }
-      value = WritableUtils.readVLong(in);
-    }
-    
-    /**
-     * Write the binary representation of the counter
-     */
-    public synchronized void write(DataOutput out) throws IOException {
-      Text.writeString(out, name);
-      boolean distinctDisplayName = (! name.equals(displayName));
-      out.writeBoolean(distinctDisplayName);
-      if (distinctDisplayName) {
-        Text.writeString(out, displayName);
-      }
-      WritableUtils.writeVLong(out, value);
+      super(name, displayName);
+      increment(value);
     }
     
-    /**
-     * Get the internal name of the counter.
-     * @return the internal name of the counter
-     */
-    public synchronized String getName() {
-      return name;
-    }
-    
-    /**
-     * Get the name of the counter.
-     * @return the user facing name of the counter
-     */
-    public synchronized String getDisplayName() {
-      return displayName;
-    }
-    
-    /**
-     * Set the display name of the counter.
-     */
-    public synchronized void setDisplayName(String displayName) {
-      this.displayName = displayName;
+    public void setDisplayName(String newName) {
+      super.setDisplayName(newName);
     }
     
     /**
@@ -150,7 +99,7 @@ public class Counters implements Writable, Iterable<Counters.Group> {
       
       // Add the value
       buf.append(UNIT_OPEN);
-      buf.append(this.value);
+      buf.append(this.getValue());
       buf.append(UNIT_CLOSE);
       
       buf.append(COUNTER_CLOSE);
@@ -159,10 +108,9 @@ public class Counters implements Writable, Iterable<Counters.Group> {
     }
     
     // Checks for (content) equality of two (basic) counters
+    @Deprecated
     synchronized boolean contentEquals(Counter c) {
-      return name.equals(c.getName())
-             && displayName.equals(c.getDisplayName())
-             && value == c.getCounter();
+      return this.equals(c);
     }
     
     /**
@@ -170,16 +118,9 @@ public class Counters implements Writable, Iterable<Counters.Group> {
      * @return the current value
      */
     public synchronized long getCounter() {
-      return value;
+      return getValue();
     }
     
-    /**
-     * Increment this counter by the given value
-     * @param incr the value to increase this counter by
-     */
-    public synchronized void increment(long incr) {
-      value += incr;
-    }
   }
   
   /**
@@ -297,8 +238,8 @@ public class Counters implements Writable, Iterable<Counters.Group> {
      */
     public synchronized long getCounter(String counterName) {
       for(Counter counter: subcounters.values()) {
-        if (counter != null && counter.displayName.equals(counterName)) {
-          return counter.value;
+        if (counter != null && counter.getDisplayName().equals(counterName)) {
+          return counter.getValue();
         }
       }
       return 0L;
@@ -459,7 +400,7 @@ public class Counters implements Writable, Iterable<Counters.Group> {
    * @param amount amount by which counter is to be incremented
    */
   public synchronized void incrCounter(Enum key, long amount) {
-    findCounter(key).value += amount;
+    findCounter(key).increment(amount);
   }
   
   /**
@@ -470,7 +411,7 @@ public class Counters implements Writable, Iterable<Counters.Group> {
    * @param amount amount by which counter is to be incremented
    */
   public synchronized void incrCounter(String group, String counter, long amount) {
-    getGroup(group).getCounterForName(counter).value += amount;
+    getGroup(group).getCounterForName(counter).increment(amount);
   }
   
   /**
@@ -478,7 +419,7 @@ public class Counters implements Writable, Iterable<Counters.Group> {
    * does not exist.
    */
   public synchronized long getCounter(Enum key) {
-    return findCounter(key).value;
+    return findCounter(key).getValue();
   }
   
   /**
@@ -492,8 +433,8 @@ public class Counters implements Writable, Iterable<Counters.Group> {
       group.displayName = otherGroup.displayName;
       for (Counter otherCounter : otherGroup) {
         Counter counter = group.getCounterForName(otherCounter.getName());
-        counter.displayName = otherCounter.displayName;
-        counter.value += otherCounter.value;
+        counter.setDisplayName(otherCounter.getDisplayName());
+        counter.increment(otherCounter.getValue());
       }
     }
   }
diff --git src/mapred/org/apache/hadoop/mapred/FileSplit.java src/mapred/org/apache/hadoop/mapred/FileSplit.java
index 6e60eee..0cdc29c 100644
--- src/mapred/org/apache/hadoop/mapred/FileSplit.java
+++ src/mapred/org/apache/hadoop/mapred/FileSplit.java
@@ -21,16 +21,15 @@ package org.apache.hadoop.mapred;
 import java.io.IOException;
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.File;                              // deprecated
 
 import org.apache.hadoop.io.UTF8;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 /** A section of an input file.  Returned by {@link
  * InputFormat#getSplits(JobConf, int)} and passed to
  * {@link InputFormat#getRecordReader(InputSplit,JobConf,Reporter)}. */
-public class FileSplit implements InputSplit {
+public class FileSplit extends org.apache.hadoop.mapreduce.InputSplit 
+                       implements InputSplit {
   private Path file;
   private long start;
   private long length;
diff --git src/mapred/org/apache/hadoop/mapred/ID.java src/mapred/org/apache/hadoop/mapred/ID.java
index 7ea42d9..b01f70b 100644
--- src/mapred/org/apache/hadoop/mapred/ID.java
+++ src/mapred/org/apache/hadoop/mapred/ID.java
@@ -18,12 +18,6 @@
 
 package org.apache.hadoop.mapred;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.WritableComparable;
-
 /**
  * A general identifier, which internally stores the id
  * as an integer. This is the super class of {@link JobID}, 
@@ -33,57 +27,15 @@ import org.apache.hadoop.io.WritableComparable;
  * @see TaskID
  * @see TaskAttemptID
  */
-public abstract class ID implements WritableComparable<ID> {
-  protected static final char SEPARATOR = '_';
-  protected int id;
+@Deprecated
+public abstract class ID extends org.apache.hadoop.mapreduce.ID {
 
   /** constructs an ID object from the given int */
   public ID(int id) {
-    this.id = id;
+    super(id);
   }
 
   protected ID() {
   }
 
-  /** returns the int which represents the identifier */
-  public int getId() {
-    return id;
-  }
-
-  @Override
-  public String toString() {
-    return String.valueOf(id);
-  }
-
-  @Override
-  public int hashCode() {
-    return Integer.valueOf(id).hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o)
-      return true;
-    if(o == null)
-      return false;
-    if (o.getClass() == this.getClass()) {
-      ID that = (ID) o;
-      return this.id == that.id;
-    }
-    else
-      return false;
-  }
-
-  /** Compare IDs by associated numbers */
-  public int compareTo(ID that) {
-    return this.id - that.id;
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    this.id = in.readInt();
-  }
-
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(id);
-  }
 }
diff --git src/mapred/org/apache/hadoop/mapred/IsolationRunner.java src/mapred/org/apache/hadoop/mapred/IsolationRunner.java
index 0fea24c..4478210 100644
--- src/mapred/org/apache/hadoop/mapred/IsolationRunner.java
+++ src/mapred/org/apache/hadoop/mapred/IsolationRunner.java
@@ -159,7 +159,9 @@ public class IsolationRunner {
    * Run a single task
    * @param args the first argument is the task directory
    */
-  public static void main(String[] args) throws IOException {
+  public static void main(String[] args
+                          ) throws ClassNotFoundException, IOException, 
+                                   InterruptedException {
     if (args.length != 1) {
       System.out.println("Usage: IsolationRunner <path>/job.xml");
       System.exit(1);
diff --git src/mapred/org/apache/hadoop/mapred/JVMId.java src/mapred/org/apache/hadoop/mapred/JVMId.java
index 260d8d7..ab76a61 100644
--- src/mapred/org/apache/hadoop/mapred/JVMId.java
+++ src/mapred/org/apache/hadoop/mapred/JVMId.java
@@ -68,7 +68,7 @@ class JVMId extends ID {
   /**Compare TaskInProgressIds by first jobIds, then by tip numbers. Reduces are 
    * defined as greater then maps.*/
   @Override
-  public int compareTo(ID o) {
+  public int compareTo(org.apache.hadoop.mapreduce.ID o) {
     JVMId that = (JVMId)o;
     int jobComp = this.jobId.compareTo(that.jobId);
     if(jobComp == 0) {
diff --git src/mapred/org/apache/hadoop/mapred/JobClient.java src/mapred/org/apache/hadoop/mapred/JobClient.java
index 9bb8678..6858026 100644
--- src/mapred/org/apache/hadoop/mapred/JobClient.java
+++ src/mapred/org/apache/hadoop/mapred/JobClient.java
@@ -21,6 +21,7 @@ import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.DataInput;
 import java.io.DataOutput;
+import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -37,6 +38,7 @@ import java.net.URLConnection;
 import java.net.UnknownHostException;
 import java.util.Arrays;
 import java.util.Comparator;
+import java.util.List;
 import java.util.Random;
 
 import javax.security.auth.login.LoginException;
@@ -58,11 +60,14 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.Counters.Group;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -709,11 +714,35 @@ public class JobClient extends Configured implements MRConstants, Tool  {
    * @return a handle to the {@link RunningJob} which can be used to track the
    *         running-job.
    * @throws FileNotFoundException
+   * @throws IOException
+   */
+  public RunningJob submitJob(JobConf job) throws FileNotFoundException,
+                                                  IOException {
+    try {
+      return submitJobInternal(job);
+    } catch (InterruptedException ie) {
+      throw new IOException("interrupted", ie);
+    } catch (ClassNotFoundException cnfe) {
+      throw new IOException("class not found", cnfe);
+    }
+  }
+
+  /**
+   * Internal method for submitting jobs to the system.
+   * @param job
+   * @return
+   * @throws FileNotFoundException
    * @throws InvalidJobConfException
+   * @throws ClassNotFoundException
+   * @throws InterruptedException
    * @throws IOException
    */
-  public RunningJob submitJob(JobConf job) throws FileNotFoundException, 
-                                  InvalidJobConfException, IOException {
+  public 
+  RunningJob submitJobInternal(JobConf job
+                               ) throws FileNotFoundException, 
+                                        ClassNotFoundException,
+                                        InterruptedException,
+                                        IOException {
     /*
      * configure the command line options correctly on the submitting dfs
      */
@@ -724,12 +753,53 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     Path submitSplitFile = new Path(submitJobDir, "job.split");
     configureCommandLineOptions(job, submitJobDir, submitJarFile);
     Path submitJobFile = new Path(submitJobDir, "job.xml");
+    int reduces = job.getNumReduceTasks();
+    JobContext context = new JobContext(job, jobId);
     
     // Check the output specification
-    job.getOutputFormat().checkOutputSpecs(fs, job);
+    if (reduces == 0 ? job.getUseNewMapper() : job.getUseNewReducer()) {
+      org.apache.hadoop.mapreduce.OutputFormat<?,?> output =
+        ReflectionUtils.newInstance(context.getOutputFormatClass(), job);
+      output.checkOutputSpecs(context);
+    } else {
+      job.getOutputFormat().checkOutputSpecs(fs, job);
+    }
 
     // Create the splits for the job
     LOG.debug("Creating splits at " + fs.makeQualified(submitSplitFile));
+    int maps;
+    if (job.getUseNewMapper()) {
+      maps = writeNewSplits(context, submitSplitFile);
+    } else {
+      maps = writeOldSplits(job, submitSplitFile);
+    }
+    job.set("mapred.job.split.file", submitSplitFile.toString());
+    job.setNumMapTasks(maps);
+        
+    // Write job file to JobTracker's fs        
+    FSDataOutputStream out = 
+      FileSystem.create(fs, submitJobFile,
+                        new FsPermission(JOB_FILE_PERMISSION));
+
+    try {
+      job.writeXml(out);
+    } finally {
+      out.close();
+    }
+
+    //
+    // Now, actually submit the job (using the submit name)
+    //
+    JobStatus status = jobSubmitClient.submitJob(jobId);
+    if (status != null) {
+      return new NetworkedJob(status);
+    } else {
+      throw new IOException("Could not launch job");
+    }
+  }
+
+  private int writeOldSplits(JobConf job, 
+                             Path submitSplitFile) throws IOException {
     InputSplit[] splits = 
       job.getInputFormat().getSplits(job, job.getNumMapTasks());
     // sort the splits into order based on size, so that the biggest
@@ -752,36 +822,91 @@ public class JobClient extends Configured implements MRConstants, Tool  {
         }
       }
     });
-    // write the splits to a file for the job tracker
-    FSDataOutputStream out = FileSystem.create(fs,
-        submitSplitFile, new FsPermission(JOB_FILE_PERMISSION));
+    DataOutputStream out = writeSplitsFileHeader(job, submitSplitFile, splits.length);
+    
     try {
-      writeSplitsFile(splits, out);
+      DataOutputBuffer buffer = new DataOutputBuffer();
+      RawSplit rawSplit = new RawSplit();
+      for(InputSplit split: splits) {
+        rawSplit.setClassName(split.getClass().getName());
+        buffer.reset();
+        split.write(buffer);
+        rawSplit.setDataLength(split.getLength());
+        rawSplit.setBytes(buffer.getData(), 0, buffer.getLength());
+        rawSplit.setLocations(split.getLocations());
+        rawSplit.write(out);
+      }
     } finally {
       out.close();
     }
-    job.set("mapred.job.split.file", submitSplitFile.toString());
-    job.setNumMapTasks(splits.length);
-        
-    // Write job file to JobTracker's fs        
-    out = FileSystem.create(fs, submitJobFile,
-        new FsPermission(JOB_FILE_PERMISSION));
+    return splits.length;
+  }
+
+  private static class NewSplitComparator 
+    implements Comparator<org.apache.hadoop.mapreduce.InputSplit>{
 
+    @Override
+    public int compare(org.apache.hadoop.mapreduce.InputSplit o1,
+                       org.apache.hadoop.mapreduce.InputSplit o2) {
+      try {
+        long len1 = o1.getLength();
+        long len2 = o2.getLength();
+        if (len1 < len2) {
+          return 1;
+        } else if (len1 == len2) {
+          return 0;
+        } else {
+          return -1;
+        }
+      } catch (IOException ie) {
+        throw new RuntimeException("exception in compare", ie);
+      } catch (InterruptedException ie) {
+        throw new RuntimeException("exception in compare", ie);        
+      }
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private <T extends org.apache.hadoop.mapreduce.InputSplit> 
+  int writeNewSplits(JobContext job, Path submitSplitFile
+                     ) throws IOException, InterruptedException, 
+                              ClassNotFoundException {
+    JobConf conf = job.getJobConf();
+    org.apache.hadoop.mapreduce.InputFormat<?,?> input =
+      ReflectionUtils.newInstance(job.getInputFormatClass(), job.getJobConf());
+    
+    List<org.apache.hadoop.mapreduce.InputSplit> splits = input.getSplits(job);
+    T[] array = (T[])
+      splits.toArray(new org.apache.hadoop.mapreduce.InputSplit[splits.size()]);
+
+    // sort the splits into order based on size, so that the biggest
+    // go first
+    Arrays.sort(array, new NewSplitComparator());
+    DataOutputStream out = writeSplitsFileHeader(conf, submitSplitFile, 
+                                                 array.length);
     try {
-      job.writeXml(out);
+      if (array.length != 0) {
+        DataOutputBuffer buffer = new DataOutputBuffer();
+        RawSplit rawSplit = new RawSplit();
+        SerializationFactory factory = new SerializationFactory(conf);
+        Serializer<T> serializer = 
+          factory.getSerializer((Class<T>) array[0].getClass());
+        serializer.open(buffer);
+        for(T split: array) {
+          rawSplit.setClassName(split.getClass().getName());
+          buffer.reset();
+          serializer.serialize(split);
+          rawSplit.setDataLength(split.getLength());
+          rawSplit.setBytes(buffer.getData(), 0, buffer.getLength());
+          rawSplit.setLocations(split.getLocations());
+          rawSplit.write(out);
+        }
+        serializer.close();
+      }
     } finally {
       out.close();
     }
-
-    //
-    // Now, actually submit the job (using the submit name)
-    //
-    JobStatus status = jobSubmitClient.submitJob(jobId);
-    if (status != null) {
-      return new NetworkedJob(status);
-    } else {
-      throw new IOException("Could not launch job");
-    }
+    return array.length;
   }
 
   /** 
@@ -877,7 +1002,21 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     
   private static final int CURRENT_SPLIT_FILE_VERSION = 0;
   private static final byte[] SPLIT_FILE_HEADER = "SPL".getBytes();
-    
+
+  private DataOutputStream writeSplitsFileHeader(Configuration conf,
+                                                 Path filename,
+                                                 int length
+                                                 ) throws IOException {
+    // write the splits to a file for the job tracker
+    FileSystem fs = filename.getFileSystem(conf);
+    FSDataOutputStream out = 
+      FileSystem.create(fs, filename, new FsPermission(JOB_FILE_PERMISSION));
+    out.write(SPLIT_FILE_HEADER);
+    WritableUtils.writeVInt(out, CURRENT_SPLIT_FILE_VERSION);
+    WritableUtils.writeVInt(out, length);
+    return out;
+  }
+
   /** Create the list of input splits and write them out in a file for
    *the JobTracker. The format is:
    * <format version>
@@ -887,21 +1026,8 @@ public class JobClient extends Configured implements MRConstants, Tool  {
    * @param splits the input splits to write out
    * @param out the stream to write to
    */
-  private void writeSplitsFile(InputSplit[] splits, FSDataOutputStream out) throws IOException {
-    out.write(SPLIT_FILE_HEADER);
-    WritableUtils.writeVInt(out, CURRENT_SPLIT_FILE_VERSION);
-    WritableUtils.writeVInt(out, splits.length);
-    DataOutputBuffer buffer = new DataOutputBuffer();
-    RawSplit rawSplit = new RawSplit();
-    for(InputSplit split: splits) {
-      rawSplit.setClassName(split.getClass().getName());
-      buffer.reset();
-      split.write(buffer);
-      rawSplit.setDataLength(split.getLength());
-      rawSplit.setBytes(buffer.getData(), 0, buffer.getLength());
-      rawSplit.setLocations(split.getLocations());
-      rawSplit.write(out);
-    }
+  private void writeOldSplitsFile(InputSplit[] splits, 
+                                  FSDataOutputStream out) throws IOException {
   }
 
   /**
diff --git src/mapred/org/apache/hadoop/mapred/JobConf.java src/mapred/org/apache/hadoop/mapred/JobConf.java
index f0b6356..c5b1d8c 100644
--- src/mapred/org/apache/hadoop/mapred/JobConf.java
+++ src/mapred/org/apache/hadoop/mapred/JobConf.java
@@ -770,6 +770,40 @@ public class JobConf extends Configuration {
   }
 
   /**
+   * Should the framework use the new context-object code for running
+   * the mapper?
+   * @return true, if the new api should be used
+   */
+  public boolean getUseNewMapper() {
+    return getBoolean("mapred.mapper.new-api", false);
+  }
+  /**
+   * Set whether the framework should use the new api for the mapper.
+   * This is the default for jobs submitted with the new Job api.
+   * @return true, if the new api should be used
+   */
+  public void setUseNewMapper(boolean flag) {
+    setBoolean("mapred.mapper.new-api", flag);
+  }
+
+  /**
+   * Should the framework use the new context-object code for running
+   * the reducer?
+   * @return true, if the new api should be used
+   */
+  public boolean getUseNewReducer() {
+    return getBoolean("mapred.reducer.new-api", false);
+  }
+  /**
+   * Set whether the framework should use the new api for the reducer. 
+   * This is the default for jobs submitted with the new Job api.
+   * @return true, if the new api should be used
+   */
+  public void setUseNewReducer(boolean flag) {
+    setBoolean("mapred.reducer.new-api", flag);
+  }
+
+  /**
    * Get the value class for job outputs.
    * 
    * @return the value class for job outputs.
diff --git src/mapred/org/apache/hadoop/mapred/JobContext.java src/mapred/org/apache/hadoop/mapred/JobContext.java
index 74d95c4..fc441a5 100644
--- src/mapred/org/apache/hadoop/mapred/JobContext.java
+++ src/mapred/org/apache/hadoop/mapred/JobContext.java
@@ -19,18 +19,19 @@ package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.util.Progressable;
 
-public class JobContext {
-
-  JobConf job;
+public class JobContext extends org.apache.hadoop.mapreduce.JobContext {
+  private JobConf job;
   private Progressable progress;
 
-  JobContext(JobConf conf, Progressable progress) {
-    job = conf;
+  JobContext(JobConf conf, org.apache.hadoop.mapreduce.JobID jobId, 
+             Progressable progress) {
+    super(conf, jobId);
+    this.job = conf;
     this.progress = progress;
   }
 
-  JobContext(JobConf conf) {
-    this(conf, Reporter.NULL);
+  JobContext(JobConf conf, org.apache.hadoop.mapreduce.JobID jobId) {
+    this(conf, jobId, Reporter.NULL);
   }
   
   /**
diff --git src/mapred/org/apache/hadoop/mapred/JobID.java src/mapred/org/apache/hadoop/mapred/JobID.java
index 23b52dd..d529836 100644
--- src/mapred/org/apache/hadoop/mapred/JobID.java
+++ src/mapred/org/apache/hadoop/mapred/JobID.java
@@ -44,88 +44,32 @@ import org.apache.hadoop.io.Text;
  * @see JobTracker#getNewJobId()
  * @see JobTracker#getStartTime()
  */
-public class JobID extends ID {
-  protected static final String JOB = "job";
-  private Text jtIdentifier = new Text();
-  
-  private static NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setGroupingUsed(false);
-    idFormat.setMinimumIntegerDigits(4);
-  }
-  
+@Deprecated
+public class JobID extends org.apache.hadoop.mapreduce.JobID {
   /**
    * Constructs a JobID object 
    * @param jtIdentifier jobTracker identifier
    * @param id job number
    */
   public JobID(String jtIdentifier, int id) {
-    super(id);
-    this.jtIdentifier.set(jtIdentifier);
+    super(jtIdentifier, id);
   }
   
   public JobID() { }
-  
-  public String getJtIdentifier() {
-    return jtIdentifier.toString();
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (!super.equals(o))
-      return false;
-
-    JobID that = (JobID)o;
-    return this.jtIdentifier.equals(that.jtIdentifier);
-  }
-  
-  /**Compare JobIds by first jtIdentifiers, then by job numbers*/
-  @Override
-  public int compareTo(ID o) {
-    JobID that = (JobID)o;
-    int jtComp = this.jtIdentifier.compareTo(that.jtIdentifier);
-    if(jtComp == 0) {
-      return this.id - that.id;
-    }
-    else return jtComp;
-  }
-  
-  @Override
-  public String toString() {
-    return appendTo(new StringBuilder(JOB)).toString();
-  }
 
   /**
-   * Add the stuff after the "job" prefix to the given builder. This is useful,
-   * because the sub-ids use this substring at the start of their string.
-   * @param builder the builder to append to
-   * @return the builder that was passed in
+   * Downgrade a new JobID to an old one
+   * @param old a new or old JobID
+   * @return either old or a new JobID build to match old
    */
-  protected StringBuilder appendTo(StringBuilder builder) {
-    builder.append(SEPARATOR);
-    builder.append(jtIdentifier);
-    builder.append(SEPARATOR);
-    builder.append(idFormat.format(id));
-    return builder;
-  }
-
-  @Override
-  public int hashCode() {
-    return jtIdentifier.hashCode() + id;
-  }
-  
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    jtIdentifier.readFields(in);
+  public static JobID downgrade(org.apache.hadoop.mapreduce.JobID old) {
+    if (old instanceof JobID) {
+      return (JobID) old;
+    } else {
+      return new JobID(old.getJtIdentifier(), old.getId());
+    }
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    jtIdentifier.write(out);
-  }
-  
   @Deprecated
   public static JobID read(DataInput in) throws IOException {
     JobID jobId = new JobID();
@@ -138,19 +82,7 @@ public class JobID extends ID {
    * @throws IllegalArgumentException if the given string is malformed
    */
   public static JobID forName(String str) throws IllegalArgumentException {
-    if(str == null)
-      return null;
-    try {
-      String[] parts = str.split(Character.toString(SEPARATOR));
-      if(parts.length == 3) {
-        if(parts[0].equals(JOB)) {
-          return new JobID(parts[1], Integer.parseInt(parts[2]));
-        }
-      }
-    }catch (Exception ex) {//fall below
-    }
-    throw new IllegalArgumentException("JobId string : " + str 
-        + " is not properly formed");
+    return (JobID) org.apache.hadoop.mapreduce.JobID.forName(str);
   }
   
   /** 
@@ -187,5 +119,5 @@ public class JobID extends ID {
       .append(jobId != null ? idFormat.format(jobId) : "[0-9]*");
     return builder;
   }
-  
+
 }
diff --git src/mapred/org/apache/hadoop/mapred/JobProfile.java src/mapred/org/apache/hadoop/mapred/JobProfile.java
index 1c265b5..08e0192 100644
--- src/mapred/org/apache/hadoop/mapred/JobProfile.java
+++ src/mapred/org/apache/hadoop/mapred/JobProfile.java
@@ -66,7 +66,8 @@ public class JobProfile implements Writable {
    * @param url link to the web-ui for details of the job.
    * @param name user-specified job name.
    */
-  public JobProfile(String user, JobID jobid, String jobFile, String url,
+  public JobProfile(String user, org.apache.hadoop.mapreduce.JobID jobid, 
+                    String jobFile, String url,
                     String name) {
     this(user, jobid, jobFile, url, name, JobConf.DEFAULT_QUEUE_NAME);
   }
@@ -82,10 +83,11 @@ public class JobProfile implements Writable {
    * @param name user-specified job name.
    * @param queueName name of the queue to which the job is submitted
    */
-  public JobProfile(String user, JobID jobid, String jobFile, String url,
-                      String name, String queueName) {
+  public JobProfile(String user, org.apache.hadoop.mapreduce.JobID jobid, 
+                    String jobFile, String url,
+                    String name, String queueName) {
     this.user = user;
-    this.jobid = jobid;
+    this.jobid = (JobID) jobid;
     this.jobFile = jobFile;
     this.url = url;
     this.name = name;
diff --git src/mapred/org/apache/hadoop/mapred/JobStatus.java src/mapred/org/apache/hadoop/mapred/JobStatus.java
index eae8e34..2df94a8 100644
--- src/mapred/org/apache/hadoop/mapred/JobStatus.java
+++ src/mapred/org/apache/hadoop/mapred/JobStatus.java
@@ -48,7 +48,7 @@ public class JobStatus implements Writable, Cloneable {
   public static final int PREP = 4;
   public static final int KILLED = 5;
 
-  private final JobID jobid;
+  private JobID jobid;
   private float mapProgress;
   private float reduceProgress;
   private float cleanupProgress;
@@ -62,7 +62,6 @@ public class JobStatus implements Writable, Cloneable {
   /**
    */
   public JobStatus() {
-    jobid = new JobID();
   }
 
   /**
@@ -288,7 +287,7 @@ public class JobStatus implements Writable, Cloneable {
   }
 
   public synchronized void readFields(DataInput in) throws IOException {
-    jobid.readFields(in);
+    this.jobid = JobID.read(in);
     this.setupProgress = in.readFloat();
     this.mapProgress = in.readFloat();
     this.reduceProgress = in.readFloat();
diff --git src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
index 74f6321..64fa86e 100644
--- src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
+++ src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
@@ -112,7 +112,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
           numReduceTasks = 1;
           job.setNumReduceTasks(1);
         }
-        JobContext jContext = new JobContext(conf);
+        JobContext jContext = new JobContext(conf, jobId);
         OutputCommitter outputCommitter = job.getOutputCommitter();
         outputCommitter.setupJob(jContext);
         status.setSetupProgress(1.0f);
diff --git src/mapred/org/apache/hadoop/mapred/MapTask.java src/mapred/org/apache/hadoop/mapred/MapTask.java
index 14ba784..6842b31 100644
--- src/mapred/org/apache/hadoop/mapred/MapTask.java
+++ src/mapred/org/apache/hadoop/mapred/MapTask.java
@@ -30,6 +30,8 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.locks.Condition;
@@ -37,7 +39,7 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -50,12 +52,13 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.io.compress.CompressionCodec;
 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.Serializer;
 import org.apache.hadoop.mapred.IFile.Writer;
-import org.apache.hadoop.mapred.IFile.Reader;
 import org.apache.hadoop.mapred.Merger.Segment;
 import org.apache.hadoop.mapred.SortedRanges.SkipRangeIterator;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.IndexedSortable;
 import org.apache.hadoop.util.IndexedSorter;
 import org.apache.hadoop.util.Progress;
@@ -72,7 +75,6 @@ class MapTask extends Task {
 
   private BytesWritable split = new BytesWritable();
   private String splitClass;
-  private InputSplit instantiatedSplit = null;
   private final static int APPROX_HEADER_LENGTH = 150;
 
   private static final Log LOG = LogFactory.getLog(MapTask.class.getName());
@@ -131,11 +133,6 @@ class MapTask extends Task {
     split.readFields(in);
   }
 
-  @Override
-  InputSplit getInputSplit() throws UnsupportedOperationException {
-    return instantiatedSplit;
-  }
-
   /**
    * This class wraps the user's record reader to update the counters and progress
    * as records are read.
@@ -147,14 +144,16 @@ class MapTask extends Task {
     private RecordReader<K,V> rawIn;
     private Counters.Counter inputByteCounter;
     private Counters.Counter inputRecordCounter;
+    private TaskReporter reporter;
     private long beforePos = -1;
     private long afterPos = -1;
     
-    TrackedRecordReader(RecordReader<K,V> raw, Counters counters) 
+    TrackedRecordReader(RecordReader<K,V> raw, TaskReporter reporter) 
       throws IOException{
       rawIn = raw;
-      inputRecordCounter = counters.findCounter(MAP_INPUT_RECORDS);
-      inputByteCounter = counters.findCounter(MAP_INPUT_BYTES);
+      inputRecordCounter = reporter.getCounter(MAP_INPUT_RECORDS);
+      inputByteCounter = reporter.getCounter(MAP_INPUT_BYTES);
+      this.reporter = reporter;
     }
 
     public K createKey() {
@@ -181,7 +180,7 @@ class MapTask extends Task {
      
     protected synchronized boolean moveToNext(K key, V value)
       throws IOException {
-      setProgress(getProgress());
+      reporter.setProgress(getProgress());
       beforePos = getPos();
       boolean ret = rawIn.next(key, value);
       afterPos = getPos();
@@ -193,6 +192,9 @@ class MapTask extends Task {
     public float getProgress() throws IOException {
       return rawIn.getProgress();
     }
+    TaskReporter getTaskReporter() {
+      return reporter;
+    }
   }
 
   /**
@@ -207,11 +209,11 @@ class MapTask extends Task {
     private Counters.Counter skipRecCounter;
     private long recIndex = -1;
     
-    SkippingRecordReader(RecordReader<K,V> raw, Counters counters, 
-        TaskUmbilicalProtocol umbilical) throws IOException{
-      super(raw,counters);
+    SkippingRecordReader(RecordReader<K,V> raw, TaskUmbilicalProtocol umbilical,
+                         TaskReporter reporter) throws IOException{
+      super(raw, reporter);
       this.umbilical = umbilical;
-      this.skipRecCounter = counters.findCounter(Counter.MAP_SKIPPED_RECORDS);
+      this.skipRecCounter = reporter.getCounter(Counter.MAP_SKIPPED_RECORDS);
       this.toWriteSkipRecs = toWriteSkipRecs() &&  
         SkipBadRecords.getSkipOutputPath(conf)!=null;
       skipIt = getSkipRanges().skipRangeIterator();
@@ -261,44 +263,49 @@ class MapTask extends Task {
               skipFile.getFileSystem(conf), conf, skipFile,
               (Class<K>) createKey().getClass(),
               (Class<V>) createValue().getClass(), 
-              CompressionType.BLOCK, getReporter(umbilical));
+              CompressionType.BLOCK, getTaskReporter());
       }
       skipWriter.append(key, value);
     }
   }
 
   @Override
-  @SuppressWarnings("unchecked")
   public void run(final JobConf job, final TaskUmbilicalProtocol umbilical)
-    throws IOException {
-
-    final Reporter reporter = getReporter(umbilical);
+    throws IOException, ClassNotFoundException, InterruptedException {
 
     // start thread that will handle communication with parent
-    startCommunicationThread(umbilical);
+    TaskReporter reporter = new TaskReporter(getProgress(), umbilical);
+    reporter.startCommunicationThread();
+    initialize(job, getJobID(), reporter);
 
-    initialize(job, reporter);
     // check if it is a cleanupJobTask
     if (cleanupJob) {
-      runCleanup(umbilical);
+      runCleanup(umbilical, reporter);
       return;
     }
     if (setupJob) {
-      runSetupJob(umbilical);
+      runSetupJob(umbilical, reporter);
       return;
     }
 
-    int numReduceTasks = conf.getNumReduceTasks();
-    LOG.info("numReduceTasks: " + numReduceTasks);
-    MapOutputCollector collector = null;
-    if (numReduceTasks > 0) {
-      collector = new MapOutputBuffer(umbilical, job, reporter);
-    } else { 
-      collector = new DirectMapOutputCollector(umbilical, job, reporter);
+    if (job.getUseNewMapper()) {
+      runNewMapper(job, split, umbilical, reporter);
+    } else {
+      runOldMapper(job, split, umbilical, reporter);
     }
+    done(umbilical, reporter);
+  }
+
+  @SuppressWarnings("unchecked")
+  private <INKEY,INVALUE,OUTKEY,OUTVALUE>
+  void runOldMapper(final JobConf job,
+                    final BytesWritable rawSplit,
+                    final TaskUmbilicalProtocol umbilical,
+                    TaskReporter reporter) throws IOException {
+    InputSplit inputSplit = null;
     // reinstantiate the split
     try {
-      instantiatedSplit = (InputSplit) 
+      inputSplit = (InputSplit) 
         ReflectionUtils.newInstance(job.getClassByName(splitClass), job);
     } catch (ClassNotFoundException exp) {
       IOException wrap = new IOException("Split class " + splitClass + 
@@ -308,24 +315,28 @@ class MapTask extends Task {
     }
     DataInputBuffer splitBuffer = new DataInputBuffer();
     splitBuffer.reset(split.getBytes(), 0, split.getLength());
-    instantiatedSplit.readFields(splitBuffer);
+    inputSplit.readFields(splitBuffer);
     
-    // if it is a file split, we can give more details
-    if (instantiatedSplit instanceof FileSplit) {
-      FileSplit fileSplit = (FileSplit) instantiatedSplit;
-      job.set("map.input.file", fileSplit.getPath().toString());
-      job.setLong("map.input.start", fileSplit.getStart());
-      job.setLong("map.input.length", fileSplit.getLength());
-    }
-      
-    RecordReader rawIn =                  // open input
-      job.getInputFormat().getRecordReader(instantiatedSplit, job, reporter);
-    RecordReader in = isSkipping() ? 
-        new SkippingRecordReader(rawIn, getCounters(), umbilical) :
-        new TrackedRecordReader(rawIn, getCounters());
+    updateJobWithSplit(job, inputSplit);
+    reporter.setInputSplit(inputSplit);
+
+    RecordReader<INKEY,INVALUE> rawIn =                  // open input
+      job.getInputFormat().getRecordReader(inputSplit, job, reporter);
+    RecordReader<INKEY,INVALUE> in = isSkipping() ? 
+        new SkippingRecordReader<INKEY,INVALUE>(rawIn, umbilical, reporter) :
+        new TrackedRecordReader<INKEY,INVALUE>(rawIn, reporter);
     job.setBoolean("mapred.skip.on", isSkipping());
 
-    MapRunnable runner =
+
+    int numReduceTasks = conf.getNumReduceTasks();
+    LOG.info("numReduceTasks: " + numReduceTasks);
+    MapOutputCollector collector = null;
+    if (numReduceTasks > 0) {
+      collector = new MapOutputBuffer(umbilical, job, reporter);
+    } else { 
+      collector = new DirectMapOutputCollector(umbilical, job, reporter);
+    }
+    MapRunnable<INKEY,INVALUE,OUTKEY,OUTVALUE> runner =
       ReflectionUtils.newInstance(job.getMapRunnerClass(), job);
 
     try {
@@ -336,7 +347,170 @@ class MapTask extends Task {
       in.close();                               // close input
       collector.close();
     }
-    done(umbilical);
+  }
+
+  /**
+   * Update the job with details about the file split
+   * @param job the job configuration to update
+   * @param inputSplit the file split
+   */
+  private void updateJobWithSplit(final JobConf job, InputSplit inputSplit) {
+    if (inputSplit instanceof FileSplit) {
+      FileSplit fileSplit = (FileSplit) inputSplit;
+      job.set("map.input.file", fileSplit.getPath().toString());
+      job.setLong("map.input.start", fileSplit.getStart());
+      job.setLong("map.input.length", fileSplit.getLength());
+    }
+  }
+
+  static class NewTrackingRecordReader<K,V> 
+    extends org.apache.hadoop.mapreduce.RecordReader<K,V> {
+    private final org.apache.hadoop.mapreduce.RecordReader<K,V> real;
+    private final org.apache.hadoop.mapreduce.Counter inputRecordCounter;
+    
+    NewTrackingRecordReader(org.apache.hadoop.mapreduce.RecordReader<K,V> real,
+                            TaskReporter reporter) {
+      this.real = real;
+      this.inputRecordCounter = reporter.getCounter(MAP_INPUT_RECORDS);
+    }
+
+    @Override
+    public void close() throws IOException {
+      real.close();
+    }
+
+    @Override
+    public K getCurrentKey() throws IOException, InterruptedException {
+      return real.getCurrentKey();
+    }
+
+    @Override
+    public V getCurrentValue() throws IOException, InterruptedException {
+      return real.getCurrentValue();
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+      return real.getProgress();
+    }
+
+    @Override
+    public void initialize(org.apache.hadoop.mapreduce.InputSplit split,
+                           org.apache.hadoop.mapreduce.TaskAttemptContext context
+                           ) throws IOException, InterruptedException {
+      real.initialize(split, context);
+    }
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+      boolean result = real.nextKeyValue();
+      if (result) {
+        inputRecordCounter.increment(1);
+      }
+      return result;
+    }
+  }
+
+  private class NewOutputCollector<K,V>
+    extends org.apache.hadoop.mapreduce.RecordWriter<K,V> {
+    private MapOutputCollector<K,V> collector;
+
+    NewOutputCollector(JobConf job, 
+                       TaskUmbilicalProtocol umbilical,
+                       TaskReporter reporter
+                       ) throws IOException {
+      if (job.getNumReduceTasks() > 0) {
+        collector = new MapOutputBuffer<K,V>(umbilical, job, reporter);
+      } else { 
+        collector = new DirectMapOutputCollector<K,V>(umbilical, job, reporter);
+      }      
+    }
+
+    @Override
+    public void write(K key, V value) throws IOException {
+      collector.collect(key, value);
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException {
+      collector.flush();
+      collector.close();
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private <INKEY,INVALUE,OUTKEY,OUTVALUE>
+  void runNewMapper(final JobConf job,
+                    final BytesWritable rawSplit,
+                    final TaskUmbilicalProtocol umbilical,
+                    TaskReporter reporter
+                    ) throws IOException, ClassNotFoundException,
+                             InterruptedException {
+    // make a task context so we can get the classes
+    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
+      new org.apache.hadoop.mapreduce.TaskAttemptContext(job, getTaskID());
+    // make a mapper
+    org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE> mapper =
+      (org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>)
+        ReflectionUtils.newInstance(taskContext.getMapperClass(), job);
+    // make the input format
+    org.apache.hadoop.mapreduce.InputFormat<INKEY,INVALUE> inputFormat =
+      (org.apache.hadoop.mapreduce.InputFormat<INKEY,INVALUE>)
+        ReflectionUtils.newInstance(taskContext.getInputFormatClass(), job);
+    // rebuild the input split
+    org.apache.hadoop.mapreduce.InputSplit split = null;
+    DataInputBuffer splitBuffer = new DataInputBuffer();
+    splitBuffer.reset(rawSplit.getBytes(), 0, rawSplit.getLength());
+    SerializationFactory factory = new SerializationFactory(job);
+    Deserializer<? extends org.apache.hadoop.mapreduce.InputSplit>
+      deserializer = 
+        (Deserializer<? extends org.apache.hadoop.mapreduce.InputSplit>) 
+        factory.getDeserializer(job.getClassByName(splitClass));
+    deserializer.open(splitBuffer);
+    split = deserializer.deserialize(null);
+
+    org.apache.hadoop.mapreduce.RecordReader<INKEY,INVALUE> input =
+      new NewTrackingRecordReader<INKEY,INVALUE>
+          (inputFormat.createRecordReader(split, taskContext), reporter);
+    
+    job.setBoolean("mapred.skip.on", isSkipping());
+    NewOutputCollector output = null;
+    org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context 
+         mapperContext = null;
+    try {
+      Constructor<org.apache.hadoop.mapreduce.Mapper.Context> contextConstructor =
+        org.apache.hadoop.mapreduce.Mapper.Context.class.getConstructor
+        (new Class[]{org.apache.hadoop.mapreduce.Mapper.class,
+            Configuration.class,
+            org.apache.hadoop.mapreduce.TaskAttemptID.class,
+            org.apache.hadoop.mapreduce.RecordReader.class,
+            org.apache.hadoop.mapreduce.RecordWriter.class,
+            org.apache.hadoop.mapreduce.StatusReporter.class,
+            org.apache.hadoop.mapreduce.InputSplit.class});
+
+      // get an output object
+      output = new NewOutputCollector(job, umbilical, reporter); 
+
+      mapperContext = contextConstructor.newInstance(mapper, job, getTaskID(),
+                                                     input, output, 
+                                                     reporter,
+                                                     split);
+
+      input.initialize(split, mapperContext);
+      mapper.run(mapperContext);
+    } catch (NoSuchMethodException e) {
+      throw new IOException("Can't find Context constructor", e);
+    } catch (InstantiationException e) {
+      throw new IOException("Can't create Context", e);
+    } catch (InvocationTargetException e) {
+      throw new IOException("Can't invoke Context constructor", e);
+    } catch (IllegalAccessException e) {
+      throw new IOException("Can't invoke Context constructor", e);
+    } finally {
+      //close
+      input.close();                               // close input
+      output.close(mapperContext);
+    }
   }
 
   interface MapOutputCollector<K, V>
@@ -353,21 +527,20 @@ class MapTask extends Task {
  
     private RecordWriter<K, V> out = null;
 
-    private Reporter reporter = null;
+    private TaskReporter reporter = null;
 
     private final Counters.Counter mapOutputRecordCounter;
 
     @SuppressWarnings("unchecked")
     public DirectMapOutputCollector(TaskUmbilicalProtocol umbilical,
-        JobConf job, Reporter reporter) throws IOException {
+        JobConf job, TaskReporter reporter) throws IOException {
       this.reporter = reporter;
       String finalName = getOutputName(getPartition());
       FileSystem fs = FileSystem.get(job);
 
       out = job.getOutputFormat().getRecordWriter(fs, job, finalName, reporter);
 
-      Counters counters = getCounters();
-      mapOutputRecordCounter = counters.findCounter(MAP_OUTPUT_RECORDS);
+      mapOutputRecordCounter = reporter.getCounter(MAP_OUTPUT_RECORDS);
     }
 
     public void close() throws IOException {
@@ -393,7 +566,7 @@ class MapTask extends Task {
     private final int partitions;
     private final Partitioner<K, V> partitioner;
     private final JobConf job;
-    private final Reporter reporter;
+    private final TaskReporter reporter;
     private final Class<K> keyClass;
     private final Class<V> valClass;
     private final RawComparator<K> comparator;
@@ -454,7 +627,7 @@ class MapTask extends Task {
 
     @SuppressWarnings("unchecked")
     public MapOutputBuffer(TaskUmbilicalProtocol umbilical, JobConf job,
-                           Reporter reporter) throws IOException {
+                           TaskReporter reporter) throws IOException {
       this.job = job;
       this.reporter = reporter;
       localFs = FileSystem.getLocal(job);
@@ -504,11 +677,10 @@ class MapTask extends Task {
       valSerializer = serializationFactory.getSerializer(valClass);
       valSerializer.open(bb);
       // counters
-      Counters counters = getCounters();
-      mapOutputByteCounter = counters.findCounter(MAP_OUTPUT_BYTES);
-      mapOutputRecordCounter = counters.findCounter(MAP_OUTPUT_RECORDS);
-      combineInputCounter = counters.findCounter(COMBINE_INPUT_RECORDS);
-      combineOutputCounter = counters.findCounter(COMBINE_OUTPUT_RECORDS);
+      mapOutputByteCounter = reporter.getCounter(MAP_OUTPUT_BYTES);
+      mapOutputRecordCounter = reporter.getCounter(MAP_OUTPUT_RECORDS);
+      combineInputCounter = reporter.getCounter(COMBINE_INPUT_RECORDS);
+      combineOutputCounter = reporter.getCounter(COMBINE_OUTPUT_RECORDS);
       // compression
       if (job.getCompressMapOutput()) {
         Class<? extends CompressionCodec> codecClass =
diff --git src/mapred/org/apache/hadoop/mapred/RawKeyValueIterator.java src/mapred/org/apache/hadoop/mapred/RawKeyValueIterator.java
index 5c4e26b..e8f0185 100644
--- src/mapred/org/apache/hadoop/mapred/RawKeyValueIterator.java
+++ src/mapred/org/apache/hadoop/mapred/RawKeyValueIterator.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.util.Progress;
  * <code>RawKeyValueIterator</code> is an iterator used to iterate over
  * the raw keys and values during sort/merge of intermediate data. 
  */
-interface RawKeyValueIterator {
+public interface RawKeyValueIterator {
   /** 
    * Gets the current raw key.
    * 
diff --git src/mapred/org/apache/hadoop/mapred/ReduceTask.java src/mapred/org/apache/hadoop/mapred/ReduceTask.java
index 44682b4..de5a2fa 100644
--- src/mapred/org/apache/hadoop/mapred/ReduceTask.java
+++ src/mapred/org/apache/hadoop/mapred/ReduceTask.java
@@ -25,6 +25,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.lang.Math;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.net.URI;
 import java.net.URL;
 import java.net.URLClassLoader;
@@ -72,6 +74,7 @@ import org.apache.hadoop.mapred.IFile.*;
 import org.apache.hadoop.mapred.Merger.Segment;
 import org.apache.hadoop.mapred.SortedRanges.SkipRangeIterator;
 import org.apache.hadoop.mapred.TaskTracker.TaskInProgress;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
@@ -255,17 +258,18 @@ class ReduceTask extends Task {
      private SequenceFile.Writer skipWriter;
      private boolean toWriteSkipRecs;
      private boolean hasNext;
+     private TaskReporter reporter;
      
      public SkippingReduceValuesIterator(RawKeyValueIterator in,
          RawComparator<KEY> comparator, Class<KEY> keyClass,
-         Class<VALUE> valClass, Configuration conf, Progressable reporter,
+         Class<VALUE> valClass, Configuration conf, TaskReporter reporter,
          TaskUmbilicalProtocol umbilical) throws IOException {
        super(in, comparator, keyClass, valClass, conf, reporter);
        this.umbilical = umbilical;
        this.skipGroupCounter = 
-         getCounters().findCounter(Counter.REDUCE_SKIPPED_GROUPS);
+         reporter.getCounter(Counter.REDUCE_SKIPPED_GROUPS);
        this.skipRecCounter = 
-         getCounters().findCounter(Counter.REDUCE_SKIPPED_RECORDS);
+         reporter.getCounter(Counter.REDUCE_SKIPPED_RECORDS);
        this.toWriteSkipRecs = toWriteSkipRecs() &&  
          SkipBadRecords.getSkipOutputPath(conf)!=null;
        this.keyClass = keyClass;
@@ -323,7 +327,7 @@ class ReduceTask extends Task {
          skipWriter = SequenceFile.createWriter(
                skipFile.getFileSystem(conf), conf, skipFile,
                keyClass, valClass, 
-               CompressionType.BLOCK, getReporter(umbilical));
+               CompressionType.BLOCK, reporter);
        }
        skipWriter.append(key, value);
      }
@@ -332,9 +336,8 @@ class ReduceTask extends Task {
   @Override
   @SuppressWarnings("unchecked")
   public void run(JobConf job, final TaskUmbilicalProtocol umbilical)
-    throws IOException {
+    throws IOException, InterruptedException, ClassNotFoundException {
     job.setBoolean("mapred.skip.on", isSkipping());
-    Reducer reducer = ReflectionUtils.newInstance(job.getReducerClass(), job);
 
     if (!cleanupJob && !setupJob) {
       copyPhase = getProgress().addPhase("copy");
@@ -342,17 +345,17 @@ class ReduceTask extends Task {
       reducePhase = getProgress().addPhase("reduce");
     }
     // start thread that will handle communication with parent
-    startCommunicationThread(umbilical);
-    final Reporter reporter = getReporter(umbilical);
-    initialize(job, reporter);
+    TaskReporter reporter = new TaskReporter(getProgress(), umbilical);
+    reporter.startCommunicationThread();
+    initialize(job, getJobID(), reporter);
 
     // check if it is a cleanupJobTask
     if (cleanupJob) {
-      runCleanup(umbilical);
+      runCleanup(umbilical, reporter);
       return;
     }
     if (setupJob) {
-      runSetupJob(umbilical);
+      runSetupJob(umbilical, reporter);
       return;
     }
     
@@ -361,7 +364,7 @@ class ReduceTask extends Task {
 
     boolean isLocal = "local".equals(job.get("mapred.job.tracker", "local"));
     if (!isLocal) {
-      reduceCopier = new ReduceCopier(umbilical, job);
+      reduceCopier = new ReduceCopier(umbilical, job, reporter);
       if (!reduceCopier.fetchOutputs()) {
         if(reduceCopier.mergeThrowable instanceof FSError) {
           LOG.error("Task: " + getTaskID() + " - FSError: " + 
@@ -390,17 +393,42 @@ class ReduceTask extends Task {
     
     sortPhase.complete();                         // sort is complete
     setPhase(TaskStatus.Phase.REDUCE); 
+    Class keyClass = job.getMapOutputKeyClass();
+    Class valueClass = job.getMapOutputValueClass();
+    RawComparator comparator = job.getOutputValueGroupingComparator();
 
+    if (!job.getUseNewReducer()) {
+      runOldReducer(job, umbilical, reporter, rIter, comparator, 
+                    keyClass, valueClass);
+    } else {
+      runNewReducer(job, umbilical, reporter, rIter, comparator, 
+                    keyClass, valueClass);
+    }
+    done(umbilical, reporter);
+  }
+
+  @SuppressWarnings("unchecked")
+  private <INKEY,INVALUE,OUTKEY,OUTVALUE>
+  void runOldReducer(JobConf job,
+                     TaskUmbilicalProtocol umbilical,
+                     final TaskReporter reporter,
+                     RawKeyValueIterator rIter,
+                     RawComparator<INKEY> comparator,
+                     Class<INKEY> keyClass,
+                     Class<INVALUE> valueClass) throws IOException {
+    Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE> reducer = 
+      ReflectionUtils.newInstance(job.getReducerClass(), job);
     // make output collector
     String finalName = getOutputName(getPartition());
 
     FileSystem fs = FileSystem.get(job);
 
-    final RecordWriter out = 
+    final RecordWriter<OUTKEY,OUTVALUE> out = 
       job.getOutputFormat().getRecordWriter(fs, job, finalName, reporter);  
     
-    OutputCollector collector = new OutputCollector() {
-        public void collect(Object key, Object value)
+    OutputCollector<OUTKEY,OUTVALUE> collector = 
+      new OutputCollector<OUTKEY,OUTVALUE>() {
+        public void collect(OUTKEY key, OUTVALUE value)
           throws IOException {
           out.write(key, value);
           reduceOutputCounter.increment(1);
@@ -411,18 +439,16 @@ class ReduceTask extends Task {
     
     // apply reduce function
     try {
-      Class keyClass = job.getMapOutputKeyClass();
-      Class valClass = job.getMapOutputValueClass();
       //increment processed counter only if skipping feature is enabled
       boolean incrProcCount = SkipBadRecords.getReducerMaxSkipGroups(job)>0 &&
         SkipBadRecords.getAutoIncrReducerProcCount(job);
       
-      ReduceValuesIterator values = isSkipping() ? 
-          new SkippingReduceValuesIterator(rIter, 
-              job.getOutputValueGroupingComparator(), keyClass, valClass, 
+      ReduceValuesIterator<INKEY,INVALUE> values = isSkipping() ? 
+          new SkippingReduceValuesIterator<INKEY,INVALUE>(rIter, 
+              comparator, keyClass, valueClass, 
               job, reporter, umbilical) :
-          new ReduceValuesIterator(rIter, 
-          job.getOutputValueGroupingComparator(), keyClass, valClass, 
+          new ReduceValuesIterator<INKEY,INVALUE>(rIter, 
+          job.getOutputValueGroupingComparator(), keyClass, valueClass, 
           job, reporter);
       values.informReduceProgress();
       while (values.more()) {
@@ -451,13 +477,98 @@ class ReduceTask extends Task {
       
       throw ioe;
     }
-    done(umbilical);
+  }
+
+  static class NewTrackingRecordWriter<K,V> 
+      extends org.apache.hadoop.mapreduce.RecordWriter<K,V> {
+    private final org.apache.hadoop.mapreduce.RecordWriter<K,V> real;
+    private final org.apache.hadoop.mapreduce.Counter outputRecordCounter;
+  
+    NewTrackingRecordWriter(org.apache.hadoop.mapreduce.RecordWriter<K,V> real,
+                            org.apache.hadoop.mapreduce.Counter recordCounter) {
+      this.real = real;
+      this.outputRecordCounter = recordCounter;
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException,
+    InterruptedException {
+      real.close(context);
+    }
+
+    @Override
+    public void write(K key, V value) throws IOException, InterruptedException {
+      real.write(key,value);
+      outputRecordCounter.increment(1);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private <INKEY,INVALUE,OUTKEY,OUTVALUE>
+  void runNewReducer(JobConf job,
+                     final TaskUmbilicalProtocol umbilical,
+                     final Reporter reporter,
+                     RawKeyValueIterator rIter,
+                     RawComparator<INKEY> comparator,
+                     Class<INKEY> keyClass,
+                     Class<INVALUE> valueClass
+                     ) throws IOException,InterruptedException, 
+                              ClassNotFoundException {
+    // make a task context so we can get the classes
+    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
+      new org.apache.hadoop.mapreduce.TaskAttemptContext(job, getTaskID());
+    // make a reducer
+    org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE> reducer =
+      (org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>)
+        ReflectionUtils.newInstance(taskContext.getReducerClass(), job);
+    // make the output format
+    org.apache.hadoop.mapreduce.OutputFormat<OUTKEY,OUTVALUE> outputFormat =
+      (org.apache.hadoop.mapreduce.OutputFormat<OUTKEY,OUTVALUE>)
+        ReflectionUtils.newInstance(taskContext.getOutputFormatClass(), job);
+    org.apache.hadoop.mapreduce.RecordWriter<OUTKEY,OUTVALUE> output =
+      outputFormat.getRecordWriter(taskContext);
+    job.setBoolean("mapred.skip.on", isSkipping());
+    org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context 
+         reducerContext = null;
+    try {
+      Constructor<org.apache.hadoop.mapreduce.Reducer.Context> contextConstructor =
+        org.apache.hadoop.mapreduce.Reducer.Context.class.getConstructor
+        (new Class[]{org.apache.hadoop.mapreduce.Reducer.class,
+            Configuration.class,
+            org.apache.hadoop.mapreduce.TaskAttemptID.class,
+            RawKeyValueIterator.class,
+            org.apache.hadoop.mapreduce.RecordWriter.class,
+            org.apache.hadoop.mapreduce.StatusReporter.class,
+            RawComparator.class,
+            Class.class,
+            Class.class});
+
+      reducerContext = contextConstructor.newInstance(reducer, job, 
+                                                      getTaskID(),
+                                                      rIter, output, 
+                                                      reporter, comparator, 
+                                                      keyClass, valueClass);
+
+      reducer.run(reducerContext);
+    } catch (NoSuchMethodException e) {
+      throw new IOException("Can't find Context constructor", e);
+    } catch (InstantiationException e) {
+      throw new IOException("Can't create Context", e);
+    } catch (InvocationTargetException e) {
+      throw new IOException("Can't invoke Context constructor", e);
+    } catch (IllegalAccessException e) {
+      throw new IOException("Can't invoke Context constructor", e);
+    } finally {
+      //close
+      output.close(reducerContext);
+    }
   }
 
   class ReduceCopier<K, V> implements MRConstants {
 
     /** Reference to the umbilical object */
     private TaskUmbilicalProtocol umbilical;
+    private final TaskReporter reporter;
     
     /** Reference to the task object */
     
@@ -1550,10 +1661,11 @@ class ReduceTask extends Task {
       conf.setClassLoader(loader);
     }
     
-    public ReduceCopier(TaskUmbilicalProtocol umbilical, JobConf conf)
-      throws IOException {
+    public ReduceCopier(TaskUmbilicalProtocol umbilical, JobConf conf,
+                        TaskReporter reporter)throws IOException {
       
       configureClasspath(conf);
+      this.reporter = reporter;
       this.shuffleClientMetrics = new ShuffleClientMetrics(conf);
       this.umbilical = umbilical;      
       this.reduceTask = ReduceTask.this;
@@ -1643,8 +1755,6 @@ class ReduceTask extends Task {
       
       copiers = new ArrayList<MapOutputCopier>(numCopiers);
       
-      Reporter reporter = getReporter(umbilical);
-
       // start all the copying threads
       for (int i=0; i < numCopiers; i++) {
         MapOutputCopier copier = new MapOutputCopier(conf, reporter);
@@ -2386,7 +2496,6 @@ class ReduceTask extends Task {
                          codec, null);
             RawKeyValueIterator iter  = null;
             Path tmpDir = new Path(reduceTask.getTaskID().toString());
-            final Reporter reporter = getReporter(umbilical);
             try {
               iter = Merger.merge(conf, rfs,
                                   conf.getMapOutputKeyClass(),
@@ -2426,7 +2535,7 @@ class ReduceTask extends Task {
     }
 
     private class InMemFSMergeThread extends Thread {
-     
+      
       public InMemFSMergeThread() {
         setName("Thread for merging in memory files");
         setDaemon(true);
@@ -2481,7 +2590,6 @@ class ReduceTask extends Task {
                      codec, null);
 
         RawKeyValueIterator rIter = null;
-        final Reporter reporter = getReporter(umbilical);
         try {
           LOG.info("Initiating in-memory merge with " + noInMemorySegments + 
                    " segments...");
diff --git src/mapred/org/apache/hadoop/mapred/Reporter.java src/mapred/org/apache/hadoop/mapred/Reporter.java
index 185121d..f2e5697 100644
--- src/mapred/org/apache/hadoop/mapred/Reporter.java
+++ src/mapred/org/apache/hadoop/mapred/Reporter.java
@@ -47,10 +47,13 @@ public interface Reporter extends Progressable {
       }
       public void progress() {
       }
+      public Counter getCounter(Enum<?> name) {
+        return null;
+      }
       public Counter getCounter(String group, String name) {
         return null;
       }
-      public void incrCounter(Enum key, long amount) {
+      public void incrCounter(Enum<?> key, long amount) {
       }
       public void incrCounter(String group, String counter, long amount) {
       }
@@ -69,6 +72,14 @@ public interface Reporter extends Progressable {
   /**
    * Get the {@link Counter} of the given group with the given name.
    * 
+   * @param name counter name
+   * @return the <code>Counter</code> of the given group/name.
+   */
+  public abstract Counter getCounter(Enum<?> name);
+
+  /**
+   * Get the {@link Counter} of the given group with the given name.
+   * 
    * @param group counter group
    * @param name counter name
    * @return the <code>Counter</code> of the given group/name.
@@ -84,7 +95,7 @@ public interface Reporter extends Progressable {
    * @param amount A non-negative amount by which the counter is to 
    *               be incremented.
    */
-  public abstract void incrCounter(Enum key, long amount);
+  public abstract void incrCounter(Enum<?> key, long amount);
   
   /**
    * Increments the counter identified by the group and counter name
diff --git src/mapred/org/apache/hadoop/mapred/Task.java src/mapred/org/apache/hadoop/mapred/Task.java
index cd2d1ad..c6404e6 100644
--- src/mapred/org/apache/hadoop/mapred/Task.java
+++ src/mapred/org/apache/hadoop/mapred/Task.java
@@ -22,10 +22,8 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.text.NumberFormat;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -108,12 +106,11 @@ abstract class Task implements Writable, Configurable {
   ////////////////////////////////////////////
 
   private String jobFile;                         // job configuration file
-  private final TaskAttemptID taskId;             // unique, includes job id
+  private TaskAttemptID taskId;                   // unique, includes job id
   private int partition;                          // id within job
   TaskStatus taskStatus;                          // current status of the task
   protected boolean cleanupJob = false;
   protected boolean setupJob = false;
-  private Thread pingProgressThread;
   
   //skip ranges based on failed ranges from previous attempts
   private SortedRanges skipRanges = new SortedRanges();
@@ -167,7 +164,7 @@ abstract class Task implements Writable, Configurable {
   public void setJobFile(String jobFile) { this.jobFile = jobFile; }
   public String getJobFile() { return jobFile; }
   public TaskAttemptID getTaskID() { return taskId; }
-  public Counters getCounters() { return counters; }
+  Counters getCounters() { return counters; }
   
   /**
    * Get the job name for this task.
@@ -270,7 +267,7 @@ abstract class Task implements Writable, Configurable {
   }
   public void readFields(DataInput in) throws IOException {
     jobFile = Text.readString(in);
-    taskId.readFields(in);
+    taskId = TaskAttemptID.read(in);
     partition = in.readInt();
     taskStatus.readFields(in);
     this.mapOutputFile.setJobId(taskId.getJobID()); 
@@ -314,7 +311,7 @@ abstract class Task implements Writable, Configurable {
    * @param umbilical for progress reports
    */
   public abstract void run(JobConf job, TaskUmbilicalProtocol umbilical)
-    throws IOException;
+    throws IOException, ClassNotFoundException, InterruptedException;
 
 
   /** Return an approprate thread runner for this task. 
@@ -329,160 +326,184 @@ abstract class Task implements Writable, Configurable {
 
   // Current counters
   private transient Counters counters = new Counters();
-  
-  /**
-   * flag that indicates whether progress update needs to be sent to parent.
-   * If true, it has been set. If false, it has been reset. 
-   * Using AtomicBoolean since we need an atomic read & reset method. 
-   */  
-  private AtomicBoolean progressFlag = new AtomicBoolean(false);
+
   /* flag to track whether task is done */
   private AtomicBoolean taskDone = new AtomicBoolean(false);
-  // getters and setters for flag
-  private void setProgressFlag() {
-    progressFlag.set(true);
-  }
-  private boolean resetProgressFlag() {
-    return progressFlag.getAndSet(false);
-  }
   
   public abstract boolean isMapTask();
 
   public Progress getProgress() { return taskProgress; }
 
-  InputSplit getInputSplit() throws UnsupportedOperationException {
-    throw new UnsupportedOperationException("Input only available on map");
-  }
-
-  /** 
-   * The communication thread handles communication with the parent (Task Tracker). 
-   * It sends progress updates if progress has been made or if the task needs to 
-   * let the parent know that it's alive. It also pings the parent to see if it's alive. 
-   */
-  protected void startCommunicationThread(final TaskUmbilicalProtocol umbilical) {
-    pingProgressThread = new Thread(new Runnable() {
-        public void run() {
-          final int MAX_RETRIES = 3;
-          int remainingRetries = MAX_RETRIES;
-          // get current flag value and reset it as well
-          boolean sendProgress = resetProgressFlag();
-          while (!taskDone.get()) {
-            try {
-              boolean taskFound = true; // whether TT knows about this task
-              // sleep for a bit
-              try {
-                Thread.sleep(PROGRESS_INTERVAL);
-              } 
-              catch (InterruptedException e) {
-                LOG.debug(getTaskID() + " Progress/ping thread exiting " +
-                                        "since it got interrupted");
-                break;
-              }
-              
-              if (sendProgress) {
-                // we need to send progress update
-                updateCounters();
-                if (commitPending) {
-                  taskStatus.statusUpdate(TaskStatus.State.COMMIT_PENDING,
-                                          taskProgress.get(),
-                                          taskProgress.toString(), 
-                                          counters);
-                } else {
-                  taskStatus.statusUpdate(TaskStatus.State.RUNNING,
-                                          taskProgress.get(),
-                                          taskProgress.toString(), 
-                                          counters);
-                }
-                taskFound = umbilical.statusUpdate(taskId, taskStatus);
-                taskStatus.clearStatus();
-              }
-              else {
-                // send ping 
-                taskFound = umbilical.ping(taskId);
-              }
-              
-              // if Task Tracker is not aware of our task ID (probably because it died and 
-              // came back up), kill ourselves
-              if (!taskFound) {
-                LOG.warn("Parent died.  Exiting "+taskId);
-                System.exit(66);
-              }
-              
-              sendProgress = resetProgressFlag(); 
-              remainingRetries = MAX_RETRIES;
-            } 
-            catch (Throwable t) {
-              LOG.info("Communication exception: " + StringUtils.stringifyException(t));
-              remainingRetries -=1;
-              if (remainingRetries == 0) {
-                ReflectionUtils.logThreadInfo(LOG, "Communication exception", 0);
-                LOG.warn("Last retry, killing "+taskId);
-                System.exit(65);
-              }
-            }
-          }
-        }
-      }, "Comm thread for "+taskId);
-    pingProgressThread.setDaemon(true);
-    pingProgressThread.start();
-    LOG.debug(getTaskID() + " Progress/ping thread started");
-  }
-
-  public void initialize(JobConf job, Reporter reporter) 
+  public void initialize(JobConf job, JobID id, Reporter reporter) 
   throws IOException {
-    jobContext = new JobContext(job, reporter);
+    jobContext = new JobContext(job, id, reporter);
     taskContext = new TaskAttemptContext(job, taskId, reporter);
     OutputCommitter committer = conf.getOutputCommitter();
     committer.setupTask(taskContext);
   }
   
-  protected Reporter getReporter(final TaskUmbilicalProtocol umbilical) 
-    throws IOException 
-  {
-    return new Reporter() {
-        public void setStatus(String status) {
-          taskProgress.setStatus(status);
-          // indicate that progress update needs to be sent
-          setProgressFlag();
-        }
-        public void progress() {
-          // indicate that progress update needs to be sent
-          setProgressFlag();
+  protected class TaskReporter 
+      extends org.apache.hadoop.mapreduce.StatusReporter
+      implements Runnable, Reporter {
+    private TaskUmbilicalProtocol umbilical;
+    private InputSplit split = null;
+    private Progress taskProgress;
+    private Thread pingThread = null;
+    /**
+     * flag that indicates whether progress update needs to be sent to parent.
+     * If true, it has been set. If false, it has been reset. 
+     * Using AtomicBoolean since we need an atomic read & reset method. 
+     */  
+    private AtomicBoolean progressFlag = new AtomicBoolean(false);
+    
+    TaskReporter(Progress taskProgress,
+                 TaskUmbilicalProtocol umbilical) {
+      this.umbilical = umbilical;
+      this.taskProgress = taskProgress;
+    }
+    // getters and setters for flag
+    void setProgressFlag() {
+      progressFlag.set(true);
+    }
+    boolean resetProgressFlag() {
+      return progressFlag.getAndSet(false);
+    }
+    public void setStatus(String status) {
+      taskProgress.setStatus(status);
+      // indicate that progress update needs to be sent
+      setProgressFlag();
+    }
+    public void setProgress(float progress) {
+      taskProgress.set(progress);
+      // indicate that progress update needs to be sent
+      setProgressFlag();
+    }
+    public void progress() {
+      // indicate that progress update needs to be sent
+      setProgressFlag();
+    }
+    public Counters.Counter getCounter(String group, String name) {
+      Counters.Counter counter = null;
+      if (counters != null) {
+        counter = counters.findCounter(group, name);
+      }
+      return counter;
+    }
+    public Counters.Counter getCounter(Enum<?> name) {
+      return counters == null ? null : counters.findCounter(name);
+    }
+    public void incrCounter(Enum key, long amount) {
+      if (counters != null) {
+        counters.incrCounter(key, amount);
+      }
+      setProgressFlag();
+    }
+    public void incrCounter(String group, String counter, long amount) {
+      if (counters != null) {
+        counters.incrCounter(group, counter, amount);
+      }
+      if(skipping && SkipBadRecords.COUNTER_GROUP.equals(group) && (
+          SkipBadRecords.COUNTER_MAP_PROCESSED_RECORDS.equals(counter) ||
+          SkipBadRecords.COUNTER_REDUCE_PROCESSED_GROUPS.equals(counter))) {
+        //if application reports the processed records, move the 
+        //currentRecStartIndex to the next.
+        //currentRecStartIndex is the start index which has not yet been 
+        //finished and is still in task's stomach.
+        for(int i=0;i<amount;i++) {
+          currentRecStartIndex = currentRecIndexIterator.next();
         }
-        public Counters.Counter getCounter(String group, String name) {
-          Counters.Counter counter = null;
-          if (counters != null) {
-            counter = counters.findCounter(group, name);
+      }
+      setProgressFlag();
+    }
+    public void setInputSplit(InputSplit split) {
+      this.split = split;
+    }
+    public InputSplit getInputSplit() throws UnsupportedOperationException {
+      if (split == null) {
+        throw new UnsupportedOperationException("Input only available on map");
+      } else {
+        return split;
+      }
+    }    
+    /** 
+     * The communication thread handles communication with the parent (Task Tracker). 
+     * It sends progress updates if progress has been made or if the task needs to 
+     * let the parent know that it's alive. It also pings the parent to see if it's alive. 
+     */
+    public void run() {
+      final int MAX_RETRIES = 3;
+      int remainingRetries = MAX_RETRIES;
+      // get current flag value and reset it as well
+      boolean sendProgress = resetProgressFlag();
+      while (!taskDone.get()) {
+        try {
+          boolean taskFound = true; // whether TT knows about this task
+          // sleep for a bit
+          try {
+            Thread.sleep(PROGRESS_INTERVAL);
+          } 
+          catch (InterruptedException e) {
+            LOG.debug(getTaskID() + " Progress/ping thread exiting " +
+            "since it got interrupted");
+            break;
           }
-          return counter;
-        }
-        public void incrCounter(Enum key, long amount) {
-          if (counters != null) {
-            counters.incrCounter(key, amount);
+
+          if (sendProgress) {
+            // we need to send progress update
+            updateCounters();
+            if (commitPending) {
+              taskStatus.statusUpdate(TaskStatus.State.COMMIT_PENDING,
+                                      taskProgress.get(),
+                                      taskProgress.toString(), 
+                                      counters);
+            } else {
+              taskStatus.statusUpdate(TaskStatus.State.RUNNING,
+                                      taskProgress.get(),
+                                      taskProgress.toString(), 
+                                      counters);
+            }
+            taskFound = umbilical.statusUpdate(taskId, taskStatus);
+            taskStatus.clearStatus();
           }
-          setProgressFlag();
-        }
-        public void incrCounter(String group, String counter, long amount) {
-          if (counters != null) {
-            counters.incrCounter(group, counter, amount);
+          else {
+            // send ping 
+            taskFound = umbilical.ping(taskId);
           }
-          if(skipping && SkipBadRecords.COUNTER_GROUP.equals(group) && (
-              SkipBadRecords.COUNTER_MAP_PROCESSED_RECORDS.equals(counter) ||
-              SkipBadRecords.COUNTER_REDUCE_PROCESSED_GROUPS.equals(counter))) {
-            //if application reports the processed records, move the 
-            //currentRecStartIndex to the next.
-            //currentRecStartIndex is the start index which has not yet been 
-            //finished and is still in task's stomach.
-            for(int i=0;i<amount;i++) {
-              currentRecStartIndex = currentRecIndexIterator.next();
-            }
+
+          // if Task Tracker is not aware of our task ID (probably because it died and 
+          // came back up), kill ourselves
+          if (!taskFound) {
+            LOG.warn("Parent died.  Exiting "+taskId);
+            System.exit(66);
+          }
+
+          sendProgress = resetProgressFlag(); 
+          remainingRetries = MAX_RETRIES;
+        } 
+        catch (Throwable t) {
+          LOG.info("Communication exception: " + StringUtils.stringifyException(t));
+          remainingRetries -=1;
+          if (remainingRetries == 0) {
+            ReflectionUtils.logThreadInfo(LOG, "Communication exception", 0);
+            LOG.warn("Last retry, killing "+taskId);
+            System.exit(65);
           }
-          setProgressFlag();
-        }
-        public InputSplit getInputSplit() throws UnsupportedOperationException {
-          return Task.this.getInputSplit();
         }
-      };
+      }
+    }
+    public void startCommunicationThread() {
+      if (pingThread == null) {
+        pingThread = new Thread(this, "communication thread");
+        pingThread.setDaemon(true);
+        pingThread.start();
+      }
+    }
+    public void stopCommunicationThread() throws InterruptedException {
+      if (pingThread != null) {
+        pingThread.interrupt();
+        pingThread.join();
+      }
+    }
   }
   
   /**
@@ -504,12 +525,6 @@ abstract class Task implements Writable, Configurable {
     umbilical.reportNextRecordRange(taskId, range);
   }
 
-  public void setProgress(float progress) {
-    taskProgress.set(progress);
-    // indicate that progress update needs to be sent
-    setProgressFlag();
-  }
-
   /**
    * An updater that tracks the last number reported for a given file
    * system and only creates the counters when they are needed.
@@ -568,7 +583,9 @@ abstract class Task implements Writable, Configurable {
     }
   }
 
-  public void done(TaskUmbilicalProtocol umbilical) throws IOException {
+  public void done(TaskUmbilicalProtocol umbilical,
+                   TaskReporter reporter
+                   ) throws IOException, InterruptedException {
     LOG.info("Task:" + taskId + " is done."
              + " And is in the process of commiting");
     updateCounters();
@@ -596,13 +613,10 @@ abstract class Task implements Writable, Configurable {
         }
       }
       //wait for commit approval and commit
-      commit(umbilical, outputCommitter);
+      commit(umbilical, reporter, outputCommitter);
     }
     taskDone.set(true);
-    pingProgressThread.interrupt();
-    try {
-      pingProgressThread.join();
-    } catch (InterruptedException ie) {}
+    reporter.stopCommunicationThread();
     sendLastUpdate(umbilical);
     //signal the tasktracker that we are done
     sendDone(umbilical);
@@ -665,6 +679,7 @@ abstract class Task implements Writable, Configurable {
   }
 
   private void commit(TaskUmbilicalProtocol umbilical,
+                      TaskReporter reporter,
                       OutputCommitter committer) throws IOException {
     int retries = MAX_RETRIES;
     while (true) {
@@ -675,7 +690,7 @@ abstract class Task implements Writable, Configurable {
           } catch(InterruptedException ie) {
             //ignore
           }
-          setProgressFlag();
+          reporter.setProgressFlag();
         }
         // task can Commit now  
         try {
@@ -710,22 +725,24 @@ abstract class Task implements Writable, Configurable {
     }
   }
 
-  protected void runCleanup(TaskUmbilicalProtocol umbilical) 
-  throws IOException {
+  protected void runCleanup(TaskUmbilicalProtocol umbilical,
+                            TaskReporter reporter
+                            ) throws IOException, InterruptedException {
     // set phase for this task
     setPhase(TaskStatus.Phase.CLEANUP);
     getProgress().setStatus("cleanup");
     // do the cleanup
     conf.getOutputCommitter().cleanupJob(jobContext);
-    done(umbilical);
+    done(umbilical, reporter);
   }
 
-  protected void runSetupJob(TaskUmbilicalProtocol umbilical) 
-  throws IOException {
+  protected void runSetupJob(TaskUmbilicalProtocol umbilical,
+                             TaskReporter reporter
+                             ) throws IOException, InterruptedException {
     // do the setup
     getProgress().setStatus("setup");
     conf.getOutputCommitter().setupJob(jobContext);
-    done(umbilical);
+    done(umbilical, reporter);
   }
   
   public void setConf(Configuration conf) {
diff --git src/mapred/org/apache/hadoop/mapred/TaskAttemptContext.java src/mapred/org/apache/hadoop/mapred/TaskAttemptContext.java
index 3fa57d8..ffa08c4 100644
--- src/mapred/org/apache/hadoop/mapred/TaskAttemptContext.java
+++ src/mapred/org/apache/hadoop/mapred/TaskAttemptContext.java
@@ -19,20 +19,18 @@ package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.util.Progressable;
 
-public class TaskAttemptContext extends JobContext {
+public class TaskAttemptContext 
+       extends org.apache.hadoop.mapreduce.TaskAttemptContext {
+  private Progressable progress;
 
-  private JobConf conf;
-  private TaskAttemptID taskid;
-  
   TaskAttemptContext(JobConf conf, TaskAttemptID taskid) {
     this(conf, taskid, Reporter.NULL);
   }
   
   TaskAttemptContext(JobConf conf, TaskAttemptID taskid,
                      Progressable progress) {
-    super(conf, progress);
-    this.conf = conf;
-    this.taskid = taskid;
+    super(conf, taskid);
+    this.progress = progress;
   }
   
   /**
@@ -41,16 +39,19 @@ public class TaskAttemptContext extends JobContext {
    * @return TaskAttemptID
    */
   public TaskAttemptID getTaskAttemptID() {
-    return taskid;
+    return (TaskAttemptID) getTaskAttemptId();
+  }
+  
+  public Progressable getProgressible() {
+    return progress;
   }
   
-  /**
-   * Get the job Configuration.
-   * 
-   * @return JobConf
-   */
   public JobConf getJobConf() {
-    return conf;
+    return (JobConf) getConfiguration();
   }
 
+  @Override
+  public void progress() {
+    progress.progress();
+  }
 }
diff --git src/mapred/org/apache/hadoop/mapred/TaskAttemptID.java src/mapred/org/apache/hadoop/mapred/TaskAttemptID.java
index 36124b5..a1c5f93 100644
--- src/mapred/org/apache/hadoop/mapred/TaskAttemptID.java
+++ src/mapred/org/apache/hadoop/mapred/TaskAttemptID.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapred;
 
 import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 
 /**
@@ -42,9 +41,8 @@ import java.io.IOException;
  * @see JobID
  * @see TaskID
  */
-public class TaskAttemptID extends ID {
-  private static final String ATTEMPT = "attempt";
-  private final TaskID taskId;
+@Deprecated
+public class TaskAttemptID extends org.apache.hadoop.mapreduce.TaskAttemptID {
   
   /**
    * Constructs a TaskAttemptID object from given {@link TaskID}.  
@@ -52,11 +50,7 @@ public class TaskAttemptID extends ID {
    * @param id the task attempt number
    */
   public TaskAttemptID(TaskID taskId, int id) {
-    super(id);
-    if(taskId == null) {
-      throw new IllegalArgumentException("taskId cannot be null");
-    }
-    this.taskId = taskId;
+    super(taskId, id);
   }
   
   /**
@@ -73,77 +67,31 @@ public class TaskAttemptID extends ID {
   }
   
   public TaskAttemptID() { 
-    taskId = new TaskID();
-  }
-  
-  /** Returns the {@link JobID} object that this task attempt belongs to */
-  public JobID getJobID() {
-    return taskId.getJobID();
-  }
-  
-  /** Returns the {@link TaskID} object that this task attempt belongs to */
-  public TaskID getTaskID() {
-    return taskId;
-  }
-  
-  /**Returns whether this TaskAttemptID is a map ID */
-  public boolean isMap() {
-    return taskId.isMap();
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (!super.equals(o))
-      return false;
-    if(o.getClass().equals(TaskAttemptID.class)) {
-      TaskAttemptID that = (TaskAttemptID)o;
-      return this.id==that.id
-             && this.taskId.equals(that.taskId);
-    }
-    else return false;
-  }
-  
-  /**Compare TaskIds by first tipIds, then by task numbers. */
-  @Override
-  public int compareTo(ID o) {
-    TaskAttemptID that = (TaskAttemptID)o;
-    int tipComp = this.taskId.compareTo(that.taskId);
-    if(tipComp == 0) {
-      return this.id - that.id;
-    }
-    else return tipComp;
-  }
-  @Override
-  public String toString() { 
-    return appendTo(new StringBuilder(ATTEMPT)).toString();
+    super(new TaskID(), 0);
   }
 
   /**
-   * Add the unique string to the StringBuilder
-   * @param builder the builder to append ot
-   * @return the builder that was passed in.
+   * Downgrade a new TaskAttemptID to an old one
+   * @param old the new id
+   * @return either old or a new TaskAttemptID constructed to match old
    */
-  protected StringBuilder appendTo(StringBuilder builder) {
-    return taskId.appendTo(builder).append(SEPARATOR).append(id);
-  }
-  
-  @Override
-  public int hashCode() {
-    return taskId.hashCode() * 5 + id;
+  public static 
+  TaskAttemptID downgrade(org.apache.hadoop.mapreduce.TaskAttemptID old) {
+    if (old instanceof TaskAttemptID) {
+      return (TaskAttemptID) old;
+    } else {
+      return new TaskAttemptID(TaskID.downgrade(old.getTaskID()), old.getId());
+    }
   }
-  
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    taskId.readFields(in);
+
+  public TaskID getTaskID() {
+    return (TaskID) super.getTaskID();
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    taskId.write(out);
+  public JobID getJobID() {
+    return (JobID) super.getJobID();
   }
-  
+
   @Deprecated
   public static TaskAttemptID read(DataInput in) throws IOException {
     TaskAttemptID taskId = new TaskAttemptID();
@@ -157,25 +105,8 @@ public class TaskAttemptID extends ID {
    */
   public static TaskAttemptID forName(String str
                                       ) throws IllegalArgumentException {
-    if(str == null)
-      return null;
-    try {
-      String[] parts = str.split(Character.toString(SEPARATOR));
-      if(parts.length == 6) {
-        if(parts[0].equals(ATTEMPT)) {
-          boolean isMap = false;
-          if(parts[3].equals("m")) isMap = true;
-          else if(parts[3].equals("r")) isMap = false;
-          else throw new Exception();
-          return new TaskAttemptID(parts[1], Integer.parseInt(parts[2]),
-              isMap, Integer.parseInt(parts[4]), Integer.parseInt(parts[5]));
-        }
-      }
-    } catch (Exception ex) {
-      //fall below
-    }
-    throw new IllegalArgumentException("TaskAttemptId string : " + str 
-        + " is not properly formed");
+    return (TaskAttemptID) 
+             org.apache.hadoop.mapreduce.TaskAttemptID.forName(str);
   }
   
   /** 
@@ -215,5 +146,4 @@ public class TaskAttemptID extends ID {
         .append(attemptId != null ? attemptId : "[0-9]*");
     return builder;
   }
-  
 }
diff --git src/mapred/org/apache/hadoop/mapred/TaskID.java src/mapred/org/apache/hadoop/mapred/TaskID.java
index 1f7707d..5f276a8 100644
--- src/mapred/org/apache/hadoop/mapred/TaskID.java
+++ src/mapred/org/apache/hadoop/mapred/TaskID.java
@@ -19,9 +19,7 @@
 package org.apache.hadoop.mapred;
 
 import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
-import java.text.NumberFormat;
 
 /**
  * TaskID represents the immutable and unique identifier for 
@@ -45,16 +43,8 @@ import java.text.NumberFormat;
  * @see JobID
  * @see TaskAttemptID
  */
-public class TaskID extends ID {
-  private static final String TASK = "task";
-  private static final NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setGroupingUsed(false);
-    idFormat.setMinimumIntegerDigits(6);
-  }
-  
-  private JobID jobId;
-  private boolean isMap;
+@Deprecated
+public class TaskID extends org.apache.hadoop.mapreduce.TaskID {
 
   /**
    * Constructs a TaskID object from given {@link JobID}.  
@@ -62,13 +52,8 @@ public class TaskID extends ID {
    * @param isMap whether the tip is a map 
    * @param id the tip number
    */
-  public TaskID(JobID jobId, boolean isMap, int id) {
-    super(id);
-    if(jobId == null) {
-      throw new IllegalArgumentException("jobId cannot be null");
-    }
-    this.jobId = jobId;
-    this.isMap = isMap;
+  public TaskID(org.apache.hadoop.mapreduce.JobID jobId, boolean isMap,int id) {
+    super(jobId, isMap, id);
   }
   
   /**
@@ -82,81 +67,22 @@ public class TaskID extends ID {
     this(new JobID(jtIdentifier, jobId), isMap, id);
   }
   
-  public TaskID() { 
-    jobId = new JobID();
+  public TaskID() {
+    super(new JobID(), false, 0);
   }
   
-  /** Returns the {@link JobID} object that this tip belongs to */
-  public JobID getJobID() {
-    return jobId;
-  }
-  
-  /**Returns whether this TaskID is a map ID */
-  public boolean isMap() {
-    return isMap;
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (!super.equals(o))
-      return false;
-
-    TaskID that = (TaskID)o;
-    return this.isMap == that.isMap && this.jobId.equals(that.jobId);
-  }
-
-  /**Compare TaskInProgressIds by first jobIds, then by tip numbers. Reduces are 
-   * defined as greater then maps.*/
-  @Override
-  public int compareTo(ID o) {
-    TaskID that = (TaskID)o;
-    int jobComp = this.jobId.compareTo(that.jobId);
-    if (jobComp == 0) {
-      if (this.isMap == that.isMap) {
-        return this.id - that.id;
-      } else {
-        return this.isMap ? -1 : 1;
-      }
-    } else {
-      return jobComp;
-    }
-  }
-  
-  @Override
-  public String toString() { 
-    return appendTo(new StringBuilder(TASK)).toString();
-  }
-
   /**
-   * Add the unique string to the given builder.
-   * @param builder the builder to append to
-   * @return the builder that was passed in
+   * Downgrade a new TaskID to an old one
+   * @param old a new or old TaskID
+   * @return either old or a new TaskID build to match old
    */
-  protected StringBuilder appendTo(StringBuilder builder) {
-    return jobId.appendTo(builder).
-                 append(SEPARATOR).
-                 append(isMap ? 'm' : 'r').
-                 append(SEPARATOR).
-                 append(idFormat.format(id));
-  }
-  
-  @Override
-  public int hashCode() {
-    return jobId.hashCode() * 524287 + id;
-  }
-  
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    jobId.readFields(in);
-    isMap = in.readBoolean();
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    jobId.write(out);
-    out.writeBoolean(isMap);
+  public static TaskID downgrade(org.apache.hadoop.mapreduce.TaskID old) {
+    if (old instanceof TaskID) {
+      return (TaskID) old;
+    } else {
+      return new TaskID(JobID.downgrade(old.getJobID()), old.isMap(), 
+                        old.getId());
+    }
   }
 
   @Deprecated
@@ -166,32 +92,10 @@ public class TaskID extends ID {
     return tipId;
   }
   
-  /** Construct a TaskID object from given string 
-   * @return constructed TaskID object or null if the given String is null
-   * @throws IllegalArgumentException if the given string is malformed
-   */
-  public static TaskID forName(String str) 
-    throws IllegalArgumentException {
-    if(str == null)
-      return null;
-    try {
-      String[] parts = str.split(Character.toString(SEPARATOR));
-      if(parts.length == 5) {
-        if(parts[0].equals(TASK)) {
-          boolean isMap = false;
-          if(parts[3].equals("m")) isMap = true;
-          else if(parts[3].equals("r")) isMap = false;
-          else throw new Exception();
-          return new TaskID(parts[1], Integer.parseInt(parts[2]),
-              isMap, Integer.parseInt(parts[4]));
-        }
-      }
-    }catch (Exception ex) {//fall below
-    }
-    throw new IllegalArgumentException("TaskId string : " + str 
-        + " is not properly formed");
+  public JobID getJobID() {
+    return (JobID) super.getJobID();
   }
-  
+
   /** 
    * Returns a regex pattern which matches task IDs. Arguments can 
    * be given null, in which case that part of the regex will be generic.  
@@ -226,5 +130,10 @@ public class TaskID extends ID {
       .append(taskId != null ? idFormat.format(taskId) : "[0-9]*");
     return builder;
   }
-  
+
+  public static TaskID forName(String str
+                               ) throws IllegalArgumentException {
+    return (TaskID) org.apache.hadoop.mapreduce.TaskID.forName(str);
+  }
+
 }
diff --git src/mapred/org/apache/hadoop/mapred/TaskReport.java src/mapred/org/apache/hadoop/mapred/TaskReport.java
index 319cffe..045f67f 100644
--- src/mapred/org/apache/hadoop/mapred/TaskReport.java
+++ src/mapred/org/apache/hadoop/mapred/TaskReport.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.io.WritableUtils;
 
 /** A report on the state of a task. */
 public class TaskReport implements Writable {
-  private final TaskID taskid;
+  private TaskID taskid;
   private float progress;
   private String state;
   private String[] diagnostics;
@@ -36,9 +36,7 @@ public class TaskReport implements Writable {
   private long finishTime; 
   private Counters counters;
 
-  public TaskReport() {
-    taskid = new TaskID();
-  }
+  public TaskReport() {}
 
   TaskReport(TaskID taskid, float progress, String state,
              String[] diagnostics, long startTime, long finishTime,
@@ -103,7 +101,7 @@ public class TaskReport implements Writable {
       return false;
     if(o.getClass().equals(TaskReport.class)) {
       TaskReport report = (TaskReport) o;
-      return counters.contentEquals(report.getCounters())
+      return counters.equals(report.getCounters())
              && Arrays.toString(this.diagnostics)
                       .equals(Arrays.toString(report.getDiagnostics()))
              && this.finishTime == report.getFinishTime()
@@ -135,11 +133,11 @@ public class TaskReport implements Writable {
   }
 
   public void readFields(DataInput in) throws IOException {
-    taskid.readFields(in);
-    progress = in.readFloat();
-    state = Text.readString(in);
-    startTime = in.readLong(); 
-    finishTime = in.readLong();
+    this.taskid = TaskID.read(in);
+    this.progress = in.readFloat();
+    this.state = Text.readString(in);
+    this.startTime = in.readLong(); 
+    this.finishTime = in.readLong();
     
     diagnostics = WritableUtils.readStringArray(in);
     counters = new Counters();
diff --git src/mapred/org/apache/hadoop/mapreduce/Counter.java src/mapred/org/apache/hadoop/mapreduce/Counter.java
index 656ce55..df14370 100644
--- src/mapred/org/apache/hadoop/mapreduce/Counter.java
+++ src/mapred/org/apache/hadoop/mapreduce/Counter.java
@@ -38,23 +38,32 @@ import org.apache.hadoop.io.WritableUtils;
  */
 public class Counter implements Writable {
 
+  private String name;
   private String displayName;
-  private long value;
+  private long value = 0;
     
-  Counter() { 
-    value = 0L;
+  protected Counter() { 
   }
 
-  Counter(String displayName, long value) {
+  protected Counter(String name, String displayName) {
+    this.name = name;
+    this.displayName = displayName;
+  }
+  
+  protected void setDisplayName(String displayName) {
     this.displayName = displayName;
-    this.value = value;
   }
     
   /**
    * Read the binary representation of the counter
    */
   public synchronized void readFields(DataInput in) throws IOException {
-    displayName = Text.readString(in);
+    name = Text.readString(in);
+    if (in.readBoolean()) {
+      displayName = Text.readString(in);
+    } else {
+      displayName = name;
+    }
     value = WritableUtils.readVLong(in);
   }
     
@@ -62,10 +71,19 @@ public class Counter implements Writable {
    * Write the binary representation of the counter
    */
   public synchronized void write(DataOutput out) throws IOException {
-    Text.writeString(out, displayName);
+    Text.writeString(out, name);
+    boolean distinctDisplayName = ! name.equals(displayName);
+    out.writeBoolean(distinctDisplayName);
+    if (distinctDisplayName) {
+      Text.writeString(out, displayName);
+    }
     WritableUtils.writeVLong(out, value);
   }
-    
+
+  public String getName() {
+    return name;
+  }
+
   /**
    * Get the name of the counter.
    * @return the user facing name of the counter
@@ -89,4 +107,13 @@ public class Counter implements Writable {
   public synchronized void increment(long incr) {
     value += incr;
   }
+  
+  public boolean equals(Object genericRight) {
+    if (genericRight instanceof Counter) {
+      Counter right = (Counter) genericRight;
+      return name.equals(right.name) && displayName.equals(right.displayName) &&
+             value == right.value;
+    }
+    return false;
+  }
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/CounterGroup.java src/mapred/org/apache/hadoop/mapreduce/CounterGroup.java
index 2b89a40..7c72a01 100644
--- src/mapred/org/apache/hadoop/mapreduce/CounterGroup.java
+++ src/mapred/org/apache/hadoop/mapreduce/CounterGroup.java
@@ -18,10 +18,162 @@
 
 package org.apache.hadoop.mapreduce;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.MissingResourceException;
+import java.util.ResourceBundle;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
 /**
  * A group of {@link Counter}s that logically belong together. Typically,
  * it is an {@link Enum} subclass and the counters are the values.
  */
-public abstract class CounterGroup implements Iterable<Counter> {
-  abstract public String getName();
+public class CounterGroup implements Writable, Iterable<Counter> {
+  private String name;
+  private String displayName;
+  private TreeMap<String, Counter> counters = new TreeMap<String, Counter>();
+  // Optional ResourceBundle for localization of group and counter names.
+  private ResourceBundle bundle = null;    
+  
+  /**
+   * Returns the specified resource bundle, or throws an exception.
+   * @throws MissingResourceException if the bundle isn't found
+   */
+  private static ResourceBundle getResourceBundle(String enumClassName) {
+    String bundleName = enumClassName.replace('$','_');
+    return ResourceBundle.getBundle(bundleName);
+  }
+
+  protected CounterGroup(String name) {
+    this.name = name;
+    try {
+      bundle = getResourceBundle(name);
+    }
+    catch (MissingResourceException neverMind) {
+    }
+    displayName = localize("CounterGroupName", name);
+  }
+  
+  protected CounterGroup(String name, String displayName) {
+    this.name = name;
+    this.displayName = displayName;
+  }
+ 
+  public String getName() {
+    return name;
+  }
+  
+  public String getDisplayName() {
+    return displayName;
+  }
+
+  public void addCounter(Counter counter) {
+    counters.put(counter.getName(), counter);
+  }
+
+  /**
+   * Internal to find a counter in a group.
+   * @param counterName the name of the counter
+   * @param displayName the display name of the counter
+   * @return the counter that was found or added
+   */
+  protected Counter findCounter(String counterName, String displayName) {
+    Counter result = counters.get(counterName);
+    if (result == null) {
+      result = new Counter(counterName, displayName);
+      counters.put(counterName, result);
+    }
+    return result;
+  }
+
+  public Counter findCounter(String counterName) {
+    Counter result = counters.get(counterName);
+    if (result == null) {
+      String displayName = localize(counterName, counterName);
+      result = new Counter(counterName, displayName);
+      counters.put(counterName, result);
+    }
+    return result;
+  }
+  
+  public Iterator<Counter> iterator() {
+    return counters.values().iterator();
+  }
+
+  public synchronized void write(DataOutput out) throws IOException {
+    Text.writeString(out, displayName);
+    WritableUtils.writeVInt(out, counters.size());
+    for(Counter counter: counters.values()) {
+      counter.write(out);
+    }
+  }
+  
+  public synchronized void readFields(DataInput in) throws IOException {
+    displayName = Text.readString(in);
+    counters.clear();
+    int size = WritableUtils.readVInt(in);
+    for(int i=0; i < size; i++) {
+      Counter counter = new Counter();
+      counter.readFields(in);
+      counters.put(counter.getName(), counter);
+    }
+  }
+
+  protected void setDisplayName(String dipName) {
+    displayName = dipName;
+  }
+  /**
+   * Looks up key in the ResourceBundle and returns the corresponding value.
+   * If the bundle or the key doesn't exist, returns the default value.
+   */
+  private String localize(String key, String defaultValue) {
+    String result = defaultValue;
+    if (bundle != null) {
+      try {
+        result = bundle.getString(key);
+      }
+      catch (MissingResourceException mre) {
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Returns the number of counters in this group.
+   */
+  public synchronized int size() {
+    return counters.size();
+  }
+
+  public boolean equals(Object genericRight) {
+    if (genericRight instanceof CounterGroup) {
+      Iterator<Counter> right = ((CounterGroup) genericRight).counters.
+                                       values().iterator();
+      Iterator<Counter> left = counters.values().iterator();
+      while (left.hasNext()) {
+        if (!right.hasNext() || !left.next().equals(right.next())) {
+          return false;
+        }
+      }
+      return !right.hasNext();
+    }
+    return false;
+  }
+
+  public int hashCode() {
+    return counters.hashCode();
+  }
+  
+  public void incrAllCounters(CounterGroup rightGroup) {
+    for(Counter right: rightGroup.counters.values()) {
+      Counter left = findCounter(right.getName(), right.getDisplayName());
+      left.increment(right.getValue());
+    }
+  }
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/Counters.java src/mapred/org/apache/hadoop/mapreduce/Counters.java
new file mode 100644
index 0000000..5c81f22
--- /dev/null
+++ src/mapred/org/apache/hadoop/mapreduce/Counters.java
@@ -0,0 +1,180 @@
+package org.apache.hadoop.mapreduce;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+public class Counters implements Writable,Iterable<CounterGroup> {
+  /**
+   * A cache from enum values to the associated counter. Dramatically speeds up
+   * typical usage.
+   */
+  private Map<Enum<?>, Counter> cache = new IdentityHashMap<Enum<?>, Counter>();
+
+  private TreeMap<String, CounterGroup> groups = 
+      new TreeMap<String, CounterGroup>();
+  
+  public Counters() {
+  }
+  
+  Counters(org.apache.hadoop.mapred.Counters counters) {
+    for(org.apache.hadoop.mapred.Counters.Group group: counters) {
+      String name = group.getName();
+      CounterGroup newGroup = new CounterGroup(name, group.getDisplayName());
+      groups.put(name, newGroup);
+      for(Counter counter: group) {
+        newGroup.addCounter(counter);
+      }
+    }
+  }
+
+  public Counter findCounter(String groupName, String counterName) {
+    CounterGroup grp = groups.get(groupName);
+    if (grp == null) {
+      grp = new CounterGroup(groupName);
+      groups.put(groupName, grp);
+    }
+    return grp.findCounter(counterName);
+  }
+
+  /**
+   * Find the counter for the given enum. The same enum will always return the
+   * same counter.
+   * @param key the counter key
+   * @return the matching counter object
+   */
+  public synchronized Counter findCounter(Enum<?> key) {
+    Counter counter = cache.get(key);
+    if (counter == null) {
+      counter = findCounter(key.getDeclaringClass().getName(), key.toString());
+      cache.put(key, counter);
+    }
+    return counter;    
+  }
+
+  /**
+   * Returns the names of all counter classes.
+   * @return Set of counter names.
+   */
+  public synchronized Collection<String> getGroupNames() {
+    return groups.keySet();
+  }
+
+  @Override
+  public Iterator<CounterGroup> iterator() {
+    return groups.values().iterator();
+  }
+
+  /**
+   * Returns the named counter group, or an empty group if there is none
+   * with the specified name.
+   */
+  public synchronized CounterGroup getGroup(String groupName) {
+    return groups.get(groupName);
+  }
+
+  /**
+   * Returns the total number of counters, by summing the number of counters
+   * in each group.
+   */
+  public synchronized  int countCounters() {
+    int result = 0;
+    for (CounterGroup group : this) {
+      result += group.size();
+    }
+    return result;
+  }
+
+  /**
+   * Write the set of groups.
+   * The external format is:
+   *     #groups (groupName group)*
+   *
+   * i.e. the number of groups followed by 0 or more groups, where each 
+   * group is of the form:
+   *
+   *     groupDisplayName #counters (false | true counter)*
+   *
+   * where each counter is of the form:
+   *
+   *     name (false | true displayName) value
+   */
+  @Override
+  public synchronized void write(DataOutput out) throws IOException {
+    out.writeInt(groups.size());
+    for (org.apache.hadoop.mapreduce.CounterGroup group: groups.values()) {
+      Text.writeString(out, group.getName());
+      group.write(out);
+    }
+  }
+  
+  /**
+   * Read a set of groups.
+   */
+  @Override
+  public synchronized void readFields(DataInput in) throws IOException {
+    int numClasses = in.readInt();
+    groups.clear();
+    while (numClasses-- > 0) {
+      String groupName = Text.readString(in);
+      CounterGroup group = new CounterGroup(groupName);
+      group.readFields(in);
+      groups.put(groupName, group);
+    }
+  }
+
+  /**
+   * Return textual representation of the counter values.
+   */
+  public synchronized String toString() {
+    StringBuilder sb = new StringBuilder("Counters: " + countCounters());
+    for (CounterGroup group: this) {
+      sb.append("\n\t" + group.getDisplayName());
+      for (Counter counter: group) {
+        sb.append("\n\t\t" + counter.getDisplayName() + "=" + 
+                  counter.getValue());
+      }
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Increments multiple counters by their amounts in another Counters 
+   * instance.
+   * @param other the other Counters instance
+   */
+  public synchronized void incrAllCounters(Counters other) {
+    for(Map.Entry<String, CounterGroup> rightEntry: other.groups.entrySet()) {
+      CounterGroup left = groups.get(rightEntry.getKey());
+      CounterGroup right = rightEntry.getValue();
+      if (left == null) {
+        left = new CounterGroup(right.getName(), right.getDisplayName());
+        groups.put(rightEntry.getKey(), left);
+      }
+      left.incrAllCounters(right);
+    }
+  }
+
+  public boolean equals(Object genericRight) {
+    if (genericRight instanceof Counters) {
+      Iterator<CounterGroup> right = ((Counters) genericRight).groups.
+                                       values().iterator();
+      Iterator<CounterGroup> left = groups.values().iterator();
+      while (left.hasNext()) {
+        if (!right.hasNext() || !left.next().equals(right.next())) {
+          return false;
+        }
+      }
+      return !right.hasNext();
+    }
+    return false;
+  }
+}
diff --git src/mapred/org/apache/hadoop/mapreduce/ID.java src/mapred/org/apache/hadoop/mapreduce/ID.java
index 11f7662..49ca80c 100644
--- src/mapred/org/apache/hadoop/mapreduce/ID.java
+++ src/mapred/org/apache/hadoop/mapreduce/ID.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.io.WritableComparable;
  * @see TaskAttemptID
  */
 public abstract class ID implements WritableComparable<ID> {
+  protected static final char SEPARATOR = '_';
   protected int id;
 
   /** constructs an ID object from the given int */
@@ -85,4 +86,5 @@ public abstract class ID implements WritableComparable<ID> {
   public void write(DataOutput out) throws IOException {
     out.writeInt(id);
   }
+  
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/Job.java src/mapred/org/apache/hadoop/mapreduce/Job.java
index 21d9f3b..a8f2a3c 100644
--- src/mapred/org/apache/hadoop/mapreduce/Job.java
+++ src/mapred/org/apache/hadoop/mapreduce/Job.java
@@ -21,10 +21,12 @@ package org.apache.hadoop.mapreduce;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.TaskCompletionEvent;
 
 /**
@@ -34,27 +36,40 @@ import org.apache.hadoop.mapred.TaskCompletionEvent;
  * IllegalStateException.
  */
 public class Job extends JobContext {  
-  
-  public Job() {
+  public static enum JobState {DEFINE, RUNNING};
+  private JobState state = JobState.DEFINE;
+  private JobClient jobTracker;
+  private RunningJob info;
+
+  public Job() throws IOException {
     this(new Configuration());
   }
 
-  public Job(Configuration conf) {
+  public Job(Configuration conf) throws IOException {
     super(conf, null);
+    jobTracker = new JobClient((JobConf) getConfiguration());
   }
 
-  public Job(Configuration conf, String jobName) {
+  public Job(Configuration conf, String jobName) throws IOException {
     this(conf);
     setJobName(jobName);
   }
 
+  private void ensureState(JobState state) throws IllegalStateException {
+    if (state != this.state) {
+      throw new IllegalStateException("Job in state "+ this.state + 
+                                      " instead of " + state);
+    }
+  }
+
   /**
    * Set the number of reduce tasks for the job.
    * @param tasks the number of reduce tasks
    * @throws IllegalStateException if the job is submitted
    */
   public void setNumReduceTasks(int tasks) throws IllegalStateException {
-    conf.setInt(NUM_REDUCES_ATTR, tasks);
+    ensureState(JobState.DEFINE);
+    conf.setNumReduceTasks(tasks);
   }
 
   /**
@@ -64,8 +79,8 @@ public class Job extends JobContext {
    * @throws IllegalStateException if the job is submitted
    */
   public void setWorkingDirectory(Path dir) throws IOException {
-    dir = dir.makeQualified(FileSystem.get(conf));
-    conf.set(WORKING_DIR_ATTR, dir.toString());
+    ensureState(JobState.DEFINE);
+    conf.setWorkingDirectory(dir);
   }
 
   /**
@@ -75,6 +90,7 @@ public class Job extends JobContext {
    */
   public void setInputFormatClass(Class<? extends InputFormat<?,?>> cls
                                   ) throws IllegalStateException {
+    ensureState(JobState.DEFINE);
     conf.setClass(INPUT_FORMAT_CLASS_ATTR, cls, InputFormat.class);
   }
 
@@ -85,6 +101,7 @@ public class Job extends JobContext {
    */
   public void setOutputFormatClass(Class<? extends OutputFormat<?,?>> cls
                                    ) throws IllegalStateException {
+    ensureState(JobState.DEFINE);
     conf.setClass(OUTPUT_FORMAT_CLASS_ATTR, cls, OutputFormat.class);
   }
 
@@ -95,16 +112,34 @@ public class Job extends JobContext {
    */
   public void setMapperClass(Class<? extends Mapper<?,?,?,?>> cls
                              ) throws IllegalStateException {
+    ensureState(JobState.DEFINE);
     conf.setClass(MAP_CLASS_ATTR, cls, Mapper.class);
   }
 
   /**
+   * Set the Jar by finding where a given class came from.
+   * @param cls the example class
+   */
+  public void setJarByClass(Class<?> cls) {
+    conf.setJarByClass(cls);
+  }
+  
+  /**
+   * Get the pathname of the job's jar.
+   * @return the pathname
+   */
+  public String getJar() {
+    return conf.getJar();
+  }
+
+  /**
    * Set the combiner class for the job.
    * @param cls the combiner to use
    * @throws IllegalStateException if the job is submitted
    */
   public void setCombinerClass(Class<? extends Reducer<?,?,?,?>> cls
                                ) throws IllegalStateException {
+    ensureState(JobState.DEFINE);
     conf.setClass(COMBINE_CLASS_ATTR, cls, Reducer.class);
   }
 
@@ -115,6 +150,7 @@ public class Job extends JobContext {
    */
   public void setReducerClass(Class<? extends Reducer<?,?,?,?>> cls
                               ) throws IllegalStateException {
+    ensureState(JobState.DEFINE);
     conf.setClass(REDUCE_CLASS_ATTR, cls, Reducer.class);
   }
 
@@ -125,6 +161,7 @@ public class Job extends JobContext {
    */
   public void setPartitionerClass(Class<? extends Partitioner<?,?>> cls
                                   ) throws IllegalStateException {
+    ensureState(JobState.DEFINE);
     conf.setClass(PARTITIONER_CLASS_ATTR, cls, Partitioner.class);
   }
 
@@ -138,7 +175,8 @@ public class Job extends JobContext {
    */
   public void setMapOutputKeyClass(Class<?> theClass
                                    ) throws IllegalStateException {
-    conf.setClass(MAP_OUTPUT_KEY_CLASS_ATTR, theClass, Object.class);
+    ensureState(JobState.DEFINE);
+    conf.setMapOutputKeyClass(theClass);
   }
 
   /**
@@ -151,7 +189,8 @@ public class Job extends JobContext {
    */
   public void setMapOutputValueClass(Class<?> theClass
                                      ) throws IllegalStateException {
-    conf.setClass(MAP_OUTPUT_VALUE_CLASS_ATTR, theClass, Object.class);
+    ensureState(JobState.DEFINE);
+    conf.setMapOutputValueClass(theClass);
   }
 
   /**
@@ -162,7 +201,8 @@ public class Job extends JobContext {
    */
   public void setOutputKeyClass(Class<?> theClass
                                 ) throws IllegalStateException {
-    conf.setClass(OUTPUT_KEY_CLASS_ATTR, theClass, Object.class);
+    ensureState(JobState.DEFINE);
+    conf.setOutputKeyClass(theClass);
   }
 
   /**
@@ -173,7 +213,8 @@ public class Job extends JobContext {
    */
   public void setOutputValueClass(Class<?> theClass
                                   ) throws IllegalStateException {
-    conf.setClass(OUTPUT_VALUE_CLASS_ATTR, theClass, Object.class);
+    ensureState(JobState.DEFINE);
+    conf.setOutputValueClass(theClass);
   }
 
   /**
@@ -184,19 +225,22 @@ public class Job extends JobContext {
    */
   public void setSortComparatorClass(Class<? extends RawComparator<?>> cls
                                      ) throws IllegalStateException {
-    conf.setClass(SORT_COMPARATOR_ATTR, cls, RawComparator.class);
+    ensureState(JobState.DEFINE);
+    conf.setOutputKeyComparatorClass(cls);
   }
 
   /**
    * Define the comparator that controls which keys are grouped together
    * for a single call to 
-   * {@link Reducer#reduce(Object, Iterable, org.apache.hadoop.mapreduce.Reducer.Context)}
+   * {@link Reducer#reduce(Object, Iterable, 
+   *                       org.apache.hadoop.mapreduce.Reducer.Context)}
    * @param cls the raw comparator to use
    * @throws IllegalStateException if the job is submitted
    */
   public void setGroupingComparatorClass(Class<? extends RawComparator<?>> cls
                                          ) throws IllegalStateException {
-    conf.setClass(GROUPING_COMPARATOR_ATTR, cls, RawComparator.class);
+    ensureState(JobState.DEFINE);
+    conf.setOutputValueGroupingComparator(cls);
   }
 
   /**
@@ -206,7 +250,8 @@ public class Job extends JobContext {
    * @throws IllegalStateException if the job is submitted
    */
   public void setJobName(String name) throws IllegalStateException {
-    conf.set(JOB_NAME_ATTR, name);
+    ensureState(JobState.DEFINE);
+    conf.setJobName(name);
   }
 
   /**
@@ -215,8 +260,8 @@ public class Job extends JobContext {
    * @return the URL where some job progress information will be displayed.
    */
   public String getTrackingURL() {
-    // TODO
-    return null;
+    ensureState(JobState.RUNNING);
+    return info.getTrackingURL();
   }
 
   /**
@@ -227,8 +272,8 @@ public class Job extends JobContext {
    * @throws IOException
    */
   public float mapProgress() throws IOException {
-    // TODO
-    return 0.0f;
+    ensureState(JobState.RUNNING);
+    return info.mapProgress();
   }
 
   /**
@@ -239,8 +284,8 @@ public class Job extends JobContext {
    * @throws IOException
    */
   public float reduceProgress() throws IOException {
-    // TODO
-    return 0.0f;
+    ensureState(JobState.RUNNING);
+    return info.reduceProgress();
   }
 
   /**
@@ -251,8 +296,8 @@ public class Job extends JobContext {
    * @throws IOException
    */
   public boolean isComplete() throws IOException {
-    // TODO
-    return false;
+    ensureState(JobState.RUNNING);
+    return info.isComplete();
   }
 
   /**
@@ -262,8 +307,8 @@ public class Job extends JobContext {
    * @throws IOException
    */
   public boolean isSuccessful() throws IOException {
-    // TODO
-    return false;
+    ensureState(JobState.RUNNING);
+    return info.isSuccessful();
   }
 
   /**
@@ -273,7 +318,8 @@ public class Job extends JobContext {
    * @throws IOException
    */
   public void killJob() throws IOException {
-    // TODO
+    ensureState(JobState.RUNNING);
+    info.killJob();
   }
     
   /**
@@ -285,8 +331,8 @@ public class Job extends JobContext {
    */
   public TaskCompletionEvent[] getTaskCompletionEvents(int startFrom
                                                        ) throws IOException {
-    // TODO
-    return null;
+    ensureState(JobState.RUNNING);
+    return info.getTaskCompletionEvents(startFrom);
   }
   
   /**
@@ -296,7 +342,9 @@ public class Job extends JobContext {
    * @throws IOException
    */
   public void killTask(TaskAttemptID taskId) throws IOException {
-    // TODO
+    ensureState(JobState.RUNNING);
+    info.killTask(org.apache.hadoop.mapred.TaskAttemptID.downgrade(taskId), 
+                  false);
   }
 
   /**
@@ -306,7 +354,9 @@ public class Job extends JobContext {
    * @throws IOException
    */
   public void failTask(TaskAttemptID taskId) throws IOException {
-    // TODO
+    ensureState(JobState.RUNNING);
+    info.killTask(org.apache.hadoop.mapred.TaskAttemptID.downgrade(taskId), 
+                  true);
   }
 
   /**
@@ -316,17 +366,77 @@ public class Job extends JobContext {
    * @throws IOException
    */
   public Iterable<CounterGroup> getCounters() throws IOException {
-    // TODO
-    return null;
+    ensureState(JobState.RUNNING);
+    return new Counters(info.getCounters());
+  }
+
+  private void ensureNotSet(String attr, String msg) throws IOException {
+    if (conf.get(attr) != null) {
+      throw new IOException(attr + " is incompatible with " + msg + " mode.");
+    }    
+  }
+
+  /**
+   * Default to the new APIs unless they are explicitly set or the old mapper or
+   * reduce attributes are used.
+   * @throws IOException if the configuration is inconsistant
+   */
+  private void setUseNewAPI() throws IOException {
+    int numReduces = conf.getNumReduceTasks();
+    String oldMapperClass = "mapred.mapper.class";
+    String oldReduceClass = "mapred.reducer.class";
+    String oldCombineClass = "mapred.combiner.class";
+    conf.setBooleanIfUnset("mapred.mapper.new-api",
+                           conf.get(oldMapperClass) == null);
+    if (conf.getUseNewMapper()) {
+      String mode = "new map API";
+      ensureNotSet("mapred.input.format.class", mode);
+      ensureNotSet(oldMapperClass, mode);
+      if (numReduces != 0) {
+        ensureNotSet(oldCombineClass, mode);
+        ensureNotSet("mapred.partitioner.class", mode);
+       } else {
+        ensureNotSet("mapred.output.format.class", mode);
+      }      
+    } else {
+      String mode = "map compatability";
+      ensureNotSet(JobContext.INPUT_FORMAT_CLASS_ATTR, mode);
+      ensureNotSet(JobContext.MAP_CLASS_ATTR, mode);
+      if (numReduces != 0) {
+        ensureNotSet(JobContext.COMBINE_CLASS_ATTR, mode);
+        ensureNotSet(JobContext.PARTITIONER_CLASS_ATTR, mode);
+       } else {
+        ensureNotSet(JobContext.OUTPUT_FORMAT_CLASS_ATTR, mode);
+      }
+    }
+    if (numReduces != 0) {
+      conf.setBooleanIfUnset("mapred.reducer.new-api",
+                             conf.get(oldReduceClass) == null);
+      if (conf.getUseNewReducer()) {
+        String mode = "new reduce API";
+        ensureNotSet("mapred.output.format.class", mode);
+        ensureNotSet(oldReduceClass, mode);   
+        ensureNotSet(oldCombineClass, mode);
+      } else {
+        String mode = "reduce compatability";
+        ensureNotSet(JobContext.OUTPUT_FORMAT_CLASS_ATTR, mode);
+        ensureNotSet(JobContext.REDUCE_CLASS_ATTR, mode);   
+        ensureNotSet(JobContext.COMBINE_CLASS_ATTR, mode);        
+      }
+    }   
   }
 
   /**
    * Submit the job to the cluster and return immediately.
    * @throws IOException
    */
-  public void submit() throws IOException {
-    // TODO
-  }
+  public void submit() throws IOException, InterruptedException, 
+                              ClassNotFoundException {
+    ensureState(JobState.DEFINE);
+    setUseNewAPI();
+    info = jobTracker.submitJobInternal(conf);
+    state = JobState.RUNNING;
+   }
   
   /**
    * Submit the job to the cluster and wait for it to finish.
@@ -334,8 +444,12 @@ public class Job extends JobContext {
    * @throws IOException thrown if the communication with the 
    *         <code>JobTracker</code> is lost
    */
-  public boolean waitForCompletion() throws IOException {
-    // TODO
-    return false;
+  public boolean waitForCompletion() throws IOException, InterruptedException,
+                                            ClassNotFoundException {
+    if (state == JobState.DEFINE) {
+      submit();
+    }
+    info.waitForCompletion();
+    return isSuccessful();
   }
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/JobContext.java src/mapred/org/apache/hadoop/mapreduce/JobContext.java
index 70de7b5..0800c05 100644
--- src/mapred/org/apache/hadoop/mapreduce/JobContext.java
+++ src/mapred/org/apache/hadoop/mapreduce/JobContext.java
@@ -21,15 +21,12 @@ package org.apache.hadoop.mapreduce;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.mapreduce.lib.partition.HashPartitioner;
 
 /**
  * A read-only view of the job that is provided to the tasks while they
@@ -38,35 +35,21 @@ import org.apache.hadoop.util.ReflectionUtils;
 public class JobContext {
   // Put all of the attribute names in here so that Job and JobContext are
   // consistent.
-  protected static final String INPUT_FORMAT_CLASS_ATTR = "mapreduce.map.class";
+  protected static final String INPUT_FORMAT_CLASS_ATTR = 
+    "mapreduce.inputformat.class";
   protected static final String MAP_CLASS_ATTR = "mapreduce.map.class";
   protected static final String COMBINE_CLASS_ATTR = "mapreduce.combine.class";
   protected static final String REDUCE_CLASS_ATTR = "mapreduce.reduce.class";
   protected static final String OUTPUT_FORMAT_CLASS_ATTR = 
     "mapreduce.outputformat.class";
-  protected static final String OUTPUT_KEY_CLASS_ATTR = 
-    "mapreduce.out.key.class";
-  protected static final String OUTPUT_VALUE_CLASS_ATTR = 
-    "mapreduce.out.value.class";
-  protected static final String MAP_OUTPUT_KEY_CLASS_ATTR = 
-    "mapreduce.map.out.key.class";
-  protected static final String MAP_OUTPUT_VALUE_CLASS_ATTR = 
-    "mapreduce.map.out.value.class";
-  protected static final String NUM_REDUCES_ATTR = "mapreduce.reduce.tasks";
-  protected static final String WORKING_DIR_ATTR = "mapreduce.work.dir";
-  protected static final String JOB_NAME_ATTR = "mapreduce.job.name";
-  protected static final String SORT_COMPARATOR_ATTR = 
-    "mapreduce.sort.comparator";
-  protected static final String GROUPING_COMPARATOR_ATTR = 
-    "mapreduce.grouping.comparator";
   protected static final String PARTITIONER_CLASS_ATTR = 
     "mapreduce.partitioner.class";
 
-  protected final Configuration conf;
+  protected final org.apache.hadoop.mapred.JobConf conf;
   private final JobID jobId;
   
   public JobContext(Configuration conf, JobID jobId) {
-    this.conf = conf;
+    this.conf = new org.apache.hadoop.mapred.JobConf(conf);
     this.jobId = jobId;
   }
 
@@ -92,7 +75,7 @@ public class JobContext {
    * @return the number of reduce tasks for this job.
    */
   public int getNumReduceTasks() {
-    return conf.getInt(NUM_REDUCES_ATTR, 1);
+    return conf.getNumReduceTasks();
   }
   
   /**
@@ -101,14 +84,7 @@ public class JobContext {
    * @return the directory name.
    */
   public Path getWorkingDirectory() throws IOException {
-    String name = conf.get(WORKING_DIR_ATTR);
-    if (name != null) {
-      return new Path(name);
-    } else {
-      Path dir = FileSystem.get(conf).getWorkingDirectory();
-      conf.set(WORKING_DIR_ATTR, dir.toString());
-      return dir;
-    }
+    return conf.getWorkingDirectory();
   }
 
   /**
@@ -116,8 +92,7 @@ public class JobContext {
    * @return the key class for the job output data.
    */
   public Class<?> getOutputKeyClass() {
-    return conf.getClass(OUTPUT_KEY_CLASS_ATTR,
-                         LongWritable.class, Object.class);
+    return conf.getOutputKeyClass();
   }
   
   /**
@@ -125,7 +100,7 @@ public class JobContext {
    * @return the value class for job outputs.
    */
   public Class<?> getOutputValueClass() {
-    return conf.getClass(OUTPUT_VALUE_CLASS_ATTR, Text.class, Object.class);
+    return conf.getOutputValueClass();
   }
 
   /**
@@ -135,12 +110,7 @@ public class JobContext {
    * @return the map output key class.
    */
   public Class<?> getMapOutputKeyClass() {
-    Class<?> retv = conf.getClass(MAP_OUTPUT_KEY_CLASS_ATTR, null, 
-                                  Object.class);
-    if (retv == null) {
-      retv = getOutputKeyClass();
-    }
-    return retv;
+    return conf.getMapOutputKeyClass();
   }
 
   /**
@@ -151,12 +121,7 @@ public class JobContext {
    * @return the map output value class.
    */
   public Class<?> getMapOutputValueClass() {
-    Class<?> retv = conf.getClass(MAP_OUTPUT_VALUE_CLASS_ATTR, null,
-        Object.class);
-    if (retv == null) {
-      retv = getOutputValueClass();
-    }
-    return retv;
+    return conf.getMapOutputValueClass();
   }
 
   /**
@@ -166,7 +131,7 @@ public class JobContext {
    * @return the job's name, defaulting to "".
    */
   public String getJobName() {
-    return conf.get(JOB_NAME_ATTR, "");
+    return conf.getJobName();
   }
 
   /**
@@ -178,7 +143,7 @@ public class JobContext {
   public Class<? extends InputFormat<?,?>> getInputFormatClass() 
      throws ClassNotFoundException {
     return (Class<? extends InputFormat<?,?>>) 
-      conf.getClass(INPUT_FORMAT_CLASS_ATTR, InputFormat.class);
+      conf.getClass(INPUT_FORMAT_CLASS_ATTR, TextInputFormat.class);
   }
 
   /**
@@ -202,7 +167,7 @@ public class JobContext {
   public Class<? extends Reducer<?,?,?,?>> getCombinerClass() 
      throws ClassNotFoundException {
     return (Class<? extends Reducer<?,?,?,?>>) 
-      conf.getClass(COMBINE_CLASS_ATTR, Reducer.class);
+      conf.getClass(COMBINE_CLASS_ATTR, null);
   }
 
   /**
@@ -226,7 +191,7 @@ public class JobContext {
   public Class<? extends OutputFormat<?,?>> getOutputFormatClass() 
      throws ClassNotFoundException {
     return (Class<? extends OutputFormat<?,?>>) 
-      conf.getClass(OUTPUT_FORMAT_CLASS_ATTR, OutputFormat.class);
+      conf.getClass(OUTPUT_FORMAT_CLASS_ATTR, TextOutputFormat.class);
   }
 
   /**
@@ -238,7 +203,7 @@ public class JobContext {
   public Class<? extends Partitioner<?,?>> getPartitionerClass() 
      throws ClassNotFoundException {
     return (Class<? extends Partitioner<?,?>>) 
-      conf.getClass(PARTITIONER_CLASS_ATTR, Partitioner.class);
+      conf.getClass(PARTITIONER_CLASS_ATTR, HashPartitioner.class);
   }
 
   /**
@@ -246,14 +211,16 @@ public class JobContext {
    * 
    * @return the {@link RawComparator} comparator used to compare keys.
    */
-  @SuppressWarnings("unchecked")
   public RawComparator<?> getSortComparator() {
-    Class<?> theClass = conf.getClass(SORT_COMPARATOR_ATTR, null,
-                                   RawComparator.class);
-    if (theClass != null)
-      return (RawComparator<?>) ReflectionUtils.newInstance(theClass, conf);
-    return WritableComparator.get(
-        (Class<? extends WritableComparable>)getMapOutputKeyClass());
+    return conf.getOutputKeyComparator();
+  }
+
+  /**
+   * Get the pathname of the job's jar.
+   * @return the pathname
+   */
+  public String getJar() {
+    return conf.getJar();
   }
 
   /** 
@@ -264,12 +231,6 @@ public class JobContext {
    * @see Job#setGroupingComparatorClass(Class) for details.  
    */
   public RawComparator<?> getGroupingComparator() {
-    Class<?> theClass = conf.getClass(GROUPING_COMPARATOR_ATTR, null,
-                                   RawComparator.class);
-    if (theClass == null) {
-      return getSortComparator();
-    }
-    return (RawComparator<?>) ReflectionUtils.newInstance(theClass, conf);
+    return conf.getOutputValueGroupingComparator();
   }
-
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/JobID.java src/mapred/org/apache/hadoop/mapreduce/JobID.java
index c133722..8d09588 100644
--- src/mapred/org/apache/hadoop/mapreduce/JobID.java
+++ src/mapred/org/apache/hadoop/mapreduce/JobID.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.text.NumberFormat;
 
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobTracker;
 
 /**
  * JobID represents the immutable and unique identifier for 
@@ -45,12 +44,12 @@ import org.apache.hadoop.mapred.JobTracker;
  * @see JobTracker#getNewJobId()
  * @see JobTracker#getStartTime()
  */
-public class JobID extends ID {
-  private static final String JOB = "job";
-  private String jtIdentifier;
-  private static char UNDERSCORE = '_';
+public class JobID extends org.apache.hadoop.mapred.ID 
+                   implements Comparable<ID> {
+  protected static final String JOB = "job";
+  private final Text jtIdentifier;
   
-  private static NumberFormat idFormat = NumberFormat.getInstance();
+  protected static NumberFormat idFormat = NumberFormat.getInstance();
   static {
     idFormat.setGroupingUsed(false);
     idFormat.setMinimumIntegerDigits(4);
@@ -63,13 +62,15 @@ public class JobID extends ID {
    */
   public JobID(String jtIdentifier, int id) {
     super(id);
-    this.jtIdentifier = jtIdentifier;
+    this.jtIdentifier = new Text(jtIdentifier);
   }
   
-  private JobID() { }
+  public JobID() { 
+    jtIdentifier = new Text();
+  }
   
   public String getJtIdentifier() {
-    return jtIdentifier;
+    return jtIdentifier.toString();
   }
   
   @Override
@@ -92,42 +93,40 @@ public class JobID extends ID {
     else return jtComp;
   }
   
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    return builder.append(JOB).append(UNDERSCORE)
-      .append(toStringWOPrefix()).toString();
-  }
-  
-  /** Returns the string representation w/o prefix */
-  StringBuilder toStringWOPrefix() {
-    StringBuilder builder = new StringBuilder();
-    builder.append(jtIdentifier).append(UNDERSCORE)
-    .append(idFormat.format(id)).toString();
+  /**
+   * Add the stuff after the "job" prefix to the given builder. This is useful,
+   * because the sub-ids use this substring at the start of their string.
+   * @param builder the builder to append to
+   * @return the builder that was passed in
+   */
+  public StringBuilder appendTo(StringBuilder builder) {
+    builder.append(SEPARATOR);
+    builder.append(jtIdentifier);
+    builder.append(SEPARATOR);
+    builder.append(idFormat.format(id));
     return builder;
   }
-  
+
   @Override
   public int hashCode() {
-    return toStringWOPrefix().toString().hashCode();
+    return jtIdentifier.hashCode() + id;
   }
-  
+
+  @Override
+  public String toString() {
+    return appendTo(new StringBuilder(JOB)).toString();
+  }
+
   @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
-    this.jtIdentifier = Text.readString(in);
+    this.jtIdentifier.readFields(in);
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
-    Text.writeString(out, jtIdentifier);
-  }
-  
-  public static JobID read(DataInput in) throws IOException {
-    JobID jobId = new JobID();
-    jobId.readFields(in);
-    return jobId;
+    jtIdentifier.write(out);
   }
   
   /** Construct a JobId object from given string 
@@ -141,7 +140,8 @@ public class JobID extends ID {
       String[] parts = str.split("_");
       if(parts.length == 3) {
         if(parts[0].equals(JOB)) {
-          return new JobID(parts[1], Integer.parseInt(parts[2]));
+          return new org.apache.hadoop.mapred.JobID(parts[1], 
+                                                    Integer.parseInt(parts[2]));
         }
       }
     }catch (Exception ex) {//fall below
@@ -150,32 +150,4 @@ public class JobID extends ID {
         + " is not properly formed");
   }
   
-  /** 
-   * Returns a regex pattern which matches task IDs. Arguments can 
-   * be given null, in which case that part of the regex will be generic.  
-   * For example to obtain a regex matching <i>any job</i> 
-   * run on the jobtracker started at <i>200707121733</i>, we would use :
-   * <pre> 
-   * JobID.getTaskIDsPattern("200707121733", null);
-   * </pre>
-   * which will return :
-   * <pre> "job_200707121733_[0-9]*" </pre> 
-   * @param jtIdentifier jobTracker identifier, or null
-   * @param jobId job number, or null
-   * @return a regex pattern matching JobIDs
-   */
-  public static String getJobIDsPattern(String jtIdentifier, Integer jobId) {
-    StringBuilder builder = new StringBuilder(JOB).append(UNDERSCORE);
-    builder.append(getJobIDsPatternWOPrefix(jtIdentifier, jobId));
-    return builder.toString();
-  }
-  
-  static StringBuilder getJobIDsPatternWOPrefix(String jtIdentifier
-      , Integer jobId) {
-    StringBuilder builder = new StringBuilder()
-      .append(jtIdentifier != null ? jtIdentifier : "[^_]*").append(UNDERSCORE)
-      .append(jobId != null ? idFormat.format(jobId) : "[0-9]*");
-    return builder;
-  }
-  
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/MapContext.java src/mapred/org/apache/hadoop/mapreduce/MapContext.java
index 6a2f6eb..3f87356 100644
--- src/mapred/org/apache/hadoop/mapreduce/MapContext.java
+++ src/mapred/org/apache/hadoop/mapreduce/MapContext.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -27,17 +29,42 @@ import org.apache.hadoop.conf.Configuration;
  * @param <KEYOUT> the key output type from the Mapper
  * @param <VALUEOUT> the value output type from the Mapper
  */
-public abstract class MapContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
+public class MapContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
   extends TaskInputOutputContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
+  private RecordReader<KEYIN,VALUEIN> reader;
+  private InputSplit split;
 
-  public MapContext(Configuration conf, TaskAttemptID taskid) {
-    super(conf, taskid);
+  public MapContext(Configuration conf, TaskAttemptID taskid,
+                    RecordReader<KEYIN,VALUEIN> reader,
+                    RecordWriter<KEYOUT,VALUEOUT> writer,
+                    StatusReporter reporter,
+                    InputSplit split) {
+    super(conf, taskid, writer, reporter);
+    this.reader = reader;
+    this.split = split;
   }
 
   /**
    * Get the input split for this map.
    */
-  public abstract InputSplit getInputSplit();
+  public InputSplit getInputSplit() {
+    return split;
+  }
+
+  @Override
+  public KEYIN getCurrentKey() throws IOException, InterruptedException {
+    return reader.getCurrentKey();
+  }
+
+  @Override
+  public VALUEIN getCurrentValue() throws IOException, InterruptedException {
+    return reader.getCurrentValue();
+  }
+
+  @Override
+  public boolean nextKeyValue() throws IOException, InterruptedException {
+    return reader.nextKeyValue();
+  }
 
 }
      
\ No newline at end of file
diff --git src/mapred/org/apache/hadoop/mapreduce/Mapper.java src/mapred/org/apache/hadoop/mapreduce/Mapper.java
index b7fc05a..0e58368 100644
--- src/mapred/org/apache/hadoop/mapreduce/Mapper.java
+++ src/mapred/org/apache/hadoop/mapreduce/Mapper.java
@@ -94,10 +94,14 @@ import org.apache.hadoop.io.compress.CompressionCodec;
  */
 public class Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
 
-  public abstract class Context 
+  public class Context 
     extends MapContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
-    public Context(Configuration conf, TaskAttemptID taskid) {
-      super(conf, taskid);
+    public Context(Configuration conf, TaskAttemptID taskid,
+                   RecordReader<KEYIN,VALUEIN> reader,
+                   RecordWriter<KEYOUT,VALUEOUT> writer,
+                   StatusReporter reporter,
+                   InputSplit split) throws IOException, InterruptedException {
+      super(conf, taskid, reader, writer, reporter, split);
     }
   }
   
@@ -116,7 +120,7 @@ public class Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
   @SuppressWarnings("unchecked")
   protected void map(KEYIN key, VALUEIN value, 
                      Context context) throws IOException, InterruptedException {
-    context.collect((KEYOUT) key, (VALUEOUT) value);
+    context.write((KEYOUT) key, (VALUEOUT) value);
   }
 
   /**
@@ -135,12 +139,8 @@ public class Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
    */
   public void run(Context context) throws IOException, InterruptedException {
     setup(context);
-    KEYIN key = context.nextKey(null);
-    VALUEIN value = null;
-    while (key != null) {
-      value = context.nextValue(value);
-      map(key, value, context);
-      key = context.nextKey(key);
+    while (context.nextKeyValue()) {
+      map(context.getCurrentKey(), context.getCurrentValue(), context);
     }
     cleanup(context);
   }
diff --git src/mapred/org/apache/hadoop/mapreduce/Partitioner.java src/mapred/org/apache/hadoop/mapreduce/Partitioner.java
index 2fec5dd..a8ded0f 100644
--- src/mapred/org/apache/hadoop/mapreduce/Partitioner.java
+++ src/mapred/org/apache/hadoop/mapreduce/Partitioner.java
@@ -44,4 +44,5 @@ public abstract class Partitioner<KEY, VALUE> {
    * @return the partition number for the <code>key</code>.
    */
   public abstract int getPartition(KEY key, VALUE value, int numPartitions);
+  
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/RecordReader.java src/mapred/org/apache/hadoop/mapreduce/RecordReader.java
index 572db9e..079d58b 100644
--- src/mapred/org/apache/hadoop/mapreduce/RecordReader.java
+++ src/mapred/org/apache/hadoop/mapreduce/RecordReader.java
@@ -41,24 +41,31 @@ public abstract class RecordReader<KEYIN, VALUEIN> implements Closeable {
                                   ) throws IOException, InterruptedException;
 
   /**
-   * Read the next key.
-   * @param key the object to be read into, which may be null
-   * @return the object that was read
+   * Read the next key, value pair.
+   * @return true if a key/value pair was read
    * @throws IOException
    * @throws InterruptedException
    */
-  public abstract KEYIN nextKey(KEYIN key
-                                ) throws IOException, InterruptedException;
+  public abstract 
+  boolean nextKeyValue() throws IOException, InterruptedException;
 
   /**
-   * Read the next value. It must be called after {@link #nextKey(Object)}.
-   * @param value the object to read into, which may be null
+   * Get the current key
+   * @return the current key or null if there is no current key
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract
+  KEYIN getCurrentKey() throws IOException, InterruptedException;
+  
+  /**
+   * Get the current value.
    * @return the object that was read
    * @throws IOException
    * @throws InterruptedException
    */
-  public abstract VALUEIN nextValue(VALUEIN value
-                                    ) throws IOException, InterruptedException;
+  public abstract 
+  VALUEIN getCurrentValue() throws IOException, InterruptedException;
   
   /**
    * The current progress of the record reader through its data.
diff --git src/mapred/org/apache/hadoop/mapreduce/RecordWriter.java src/mapred/org/apache/hadoop/mapreduce/RecordWriter.java
index d2ff36f..971c962 100644
--- src/mapred/org/apache/hadoop/mapreduce/RecordWriter.java
+++ src/mapred/org/apache/hadoop/mapreduce/RecordWriter.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
  * 
  * @see OutputFormat
  */
-public interface RecordWriter<K, V> {
+public abstract class RecordWriter<K, V> {
   /** 
    * Writes a key/value pair.
    *
@@ -39,7 +39,8 @@ public interface RecordWriter<K, V> {
    * @param value the value to write.
    * @throws IOException
    */      
-  void write(K key, V value) throws IOException, InterruptedException;
+  public abstract void write(K key, V value
+                             ) throws IOException, InterruptedException;
 
   /** 
    * Close this <code>RecordWriter</code> to future operations.
@@ -47,6 +48,6 @@ public interface RecordWriter<K, V> {
    * @param context the context of the task
    * @throws IOException
    */ 
-  void close(TaskAttemptContext context
-             ) throws IOException, InterruptedException;
+  public abstract void close(TaskAttemptContext context
+                             ) throws IOException, InterruptedException;
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/ReduceContext.java src/mapred/org/apache/hadoop/mapreduce/ReduceContext.java
index 7d796ab..1f4c289 100644
--- src/mapred/org/apache/hadoop/mapreduce/ReduceContext.java
+++ src/mapred/org/apache/hadoop/mapreduce/ReduceContext.java
@@ -19,8 +19,17 @@
 package org.apache.hadoop.mapreduce;
 
 import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.mapred.RawKeyValueIterator;
+import org.apache.hadoop.util.Progressable;
 
 /**
  * The context passed to the {@link Reducer}.
@@ -29,19 +38,143 @@ import org.apache.hadoop.conf.Configuration;
  * @param <KEYOUT> the class of the output keys
  * @param <VALUEOUT> the class of the output values
  */
-public abstract class ReduceContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
+public class ReduceContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
     extends TaskInputOutputContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
+  private RawKeyValueIterator input;
+  private RawComparator<KEYIN> comparator;
+  private KEYIN key;                                  // current key
+  private VALUEIN value;                              // current value
+  private boolean firstValue = false;                 // first value in key
+  private boolean nextKeyIsSame = false;              // more w/ this key
+  private boolean hasMore;                            // more in file
+  protected Progressable reporter;
+  private Deserializer<KEYIN> keyDeserializer;
+  private Deserializer<VALUEIN> valueDeserializer;
+  private DataInputBuffer buffer = new DataInputBuffer();
+  private BytesWritable currentRawKey = new BytesWritable();
+  private ValueIterable iterable = new ValueIterable();
 
-  public ReduceContext(Configuration conf, TaskAttemptID taskid) {
-    super(conf, taskid);
+  public ReduceContext(Configuration conf, TaskAttemptID taskid,
+                       RawKeyValueIterator input, 
+                       RecordWriter<KEYOUT,VALUEOUT> output,
+                       StatusReporter reporter,
+                       RawComparator<KEYIN> comparator,
+                       Class<KEYIN> keyClass,
+                       Class<VALUEIN> valueClass
+                       ) throws InterruptedException, IOException{
+    super(conf, taskid, output, reporter);
+    this.input = input;
+    this.comparator = comparator;
+    SerializationFactory serializationFactory = new SerializationFactory(conf);
+    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
+    this.keyDeserializer.open(buffer);
+    this.valueDeserializer = serializationFactory.getDeserializer(valueClass);
+    this.valueDeserializer.open(buffer);
+    hasMore = input.next();
   }
 
+  /** Start processing next unique key. */
+  public boolean nextKey() throws IOException,InterruptedException {
+    while (hasMore && !nextKeyIsSame) {
+      nextKeyValue();
+    }
+    return hasMore && nextKeyValue();
+  }
+
+  /**
+   * Advance to the next key/value pair.
+   */
+  @Override
+  public boolean nextKeyValue() throws IOException, InterruptedException {
+    if (!hasMore) {
+      key = null;
+      value = null;
+      return false;
+    }
+    firstValue = !nextKeyIsSame;
+    DataInputBuffer next = input.getKey();
+    buffer.reset(next.getData(), next.getPosition(), next.getLength());
+    currentRawKey.set(next.getData(), next.getPosition(), next.getLength());
+    key = keyDeserializer.deserialize(key);
+    next = input.getValue();
+    buffer.reset(next.getData(), next.getPosition(), next.getLength());
+    value = valueDeserializer.deserialize(value);
+    hasMore = input.next();
+    if (hasMore) {
+      next = input.getKey();
+      nextKeyIsSame = comparator.compare(currentRawKey.getBytes(), 0, 
+                                         currentRawKey.getLength(),
+                                         next.getData(),
+                                         next.getPosition(),
+                                         next.getLength()) == 0;
+    } else {
+      nextKeyIsSame = false;
+    }
+    return true;
+  }
+
+  public KEYIN getCurrentKey() {
+    return key;
+  }
+
+  @Override
+  public VALUEIN getCurrentValue() {
+    return value;
+  }
+
+  protected class ValueIterator implements Iterator<VALUEIN> {
+
+    @Override
+    public boolean hasNext() {
+      return nextKeyIsSame;
+    }
+
+    @Override
+    public VALUEIN next() {
+      // if this is the first record, we don't need to advance
+      if (firstValue) {
+        return value;
+      }
+      // if this isn't the first record and the next key is different, they
+      // can't advance it here.
+      if (!nextKeyIsSame) {
+        throw new NoSuchElementException("iterate past last value");
+      }
+      // otherwise, go to the next key/value pair
+      try {
+        nextKeyValue();
+        return value;
+      } catch (IOException ie) {
+        throw new RuntimeException("next value iterator failed", ie);
+      } catch (InterruptedException ie) {
+        // this is bad, but we can't modify the exception list of java.util
+        throw new RuntimeException("next value iterator interrupted", ie);        
+      }
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException("remove not implemented");
+    }
+    
+  }
+
+  protected class ValueIterable implements Iterable<VALUEIN> {
+    private ValueIterator iterator = new ValueIterator();
+    @Override
+    public Iterator<VALUEIN> iterator() {
+      return iterator;
+    } 
+  }
+  
   /**
    * Iterate through the values for the current key, reusing the same value 
    * object, which is stored in the context.
-   * @return the series of values associated with the current key
+   * @return the series of values associated with the current key. All of the 
+   * objects returned directly and indirectly from this method are reused.
    */
-  public abstract 
-  Iterable<VALUEIN> getValues() throws IOException, InterruptedException;
-
+  public 
+  Iterable<VALUEIN> getValues() throws IOException, InterruptedException {
+    return iterable;
+  }
 }
\ No newline at end of file
diff --git src/mapred/org/apache/hadoop/mapreduce/Reducer.java src/mapred/org/apache/hadoop/mapreduce/Reducer.java
index b3ee0f5..564bdbf 100644
--- src/mapred/org/apache/hadoop/mapreduce/Reducer.java
+++ src/mapred/org/apache/hadoop/mapreduce/Reducer.java
@@ -21,8 +21,8 @@ package org.apache.hadoop.mapreduce;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapred.RawKeyValueIterator;
 
 /** 
  * Reduces a set of intermediate values which share a key to a smaller set of
@@ -117,10 +117,18 @@ import org.apache.hadoop.mapreduce.Partitioner;
  */
 public abstract class Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
 
-  protected abstract class Context 
+  public class Context 
     extends ReduceContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
-    public Context(Configuration conf, TaskAttemptID taskid) {
-      super(conf, taskid);
+    public Context(Configuration conf, TaskAttemptID taskid,
+                   RawKeyValueIterator input, 
+                   RecordWriter<KEYOUT,VALUEOUT> output,
+                   StatusReporter reporter,
+                   RawComparator<KEYIN> comparator,
+                   Class<KEYIN> keyClass,
+                   Class<VALUEIN> valueClass
+                   ) throws IOException, InterruptedException {
+      super(conf, taskid, input, output, reporter, comparator, keyClass, 
+            valueClass);
     }
   }
 
@@ -141,7 +149,7 @@ public abstract class Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
   protected void reduce(KEYIN key, Iterable<VALUEIN> values, Context context
                         ) throws IOException, InterruptedException {
     for(VALUEIN value: values) {
-      context.collect((KEYOUT) key, (VALUEOUT) value);
+      context.write((KEYOUT) key, (VALUEOUT) value);
     }
   }
 
@@ -160,10 +168,8 @@ public abstract class Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
    */
   public void run(Context context) throws IOException, InterruptedException {
     setup(context);
-    KEYIN key = context.nextKey(null);
-    while(key != null) {
-      reduce(key, context.getValues(), context);
-      key = context.nextKey(key);
+    while (context.nextKey()) {
+      reduce(context.getCurrentKey(), context.getValues(), context);
     }
     cleanup(context);
   }
diff --git src/mapred/org/apache/hadoop/mapreduce/StatusReporter.java src/mapred/org/apache/hadoop/mapreduce/StatusReporter.java
new file mode 100644
index 0000000..0bc4525
--- /dev/null
+++ src/mapred/org/apache/hadoop/mapreduce/StatusReporter.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapreduce;
+
+public abstract class StatusReporter {
+  public abstract Counter getCounter(Enum<?> name);
+  public abstract Counter getCounter(String group, String name);
+  public abstract void progress();
+  public abstract void setStatus(String status);
+}
diff --git src/mapred/org/apache/hadoop/mapreduce/TaskAttemptContext.java src/mapred/org/apache/hadoop/mapreduce/TaskAttemptContext.java
index 08d88b2..918b53e 100644
--- src/mapred/org/apache/hadoop/mapreduce/TaskAttemptContext.java
+++ src/mapred/org/apache/hadoop/mapreduce/TaskAttemptContext.java
@@ -26,8 +26,7 @@ import org.apache.hadoop.util.Progressable;
 /**
  * The context for task attempts.
  */
-public abstract class TaskAttemptContext extends JobContext 
-    implements Progressable {
+public class TaskAttemptContext extends JobContext implements Progressable {
   private final TaskAttemptID taskId;
   private String status = "";
   
@@ -59,13 +58,8 @@ public abstract class TaskAttemptContext extends JobContext
   }
 
   /**
-   * Lookup a counter by an enum.
+   * Report progress. The subtypes actually do work in this method.
    */
-  public abstract Counter getCounter(Enum<?> counterName);
-
-  /**
-   * Lookup a counter by group and counter name. The enum-based interface is
-   * preferred.
-   */
-  public abstract Counter getCounter(String groupName, String counterName);
+  public void progress() { 
+  }
 }
\ No newline at end of file
diff --git src/mapred/org/apache/hadoop/mapreduce/TaskAttemptID.java src/mapred/org/apache/hadoop/mapreduce/TaskAttemptID.java
index 68982a9..6d87289 100644
--- src/mapred/org/apache/hadoop/mapreduce/TaskAttemptID.java
+++ src/mapred/org/apache/hadoop/mapreduce/TaskAttemptID.java
@@ -33,7 +33,7 @@ import java.io.IOException;
  * An example TaskAttemptID is : 
  * <code>attempt_200707121733_0003_m_000005_0</code> , which represents the
  * zeroth task attempt for the fifth map task in the third job 
- * running at the jobtracker started at <code>200707121733</code>. 
+ * running at the jobtracker started at <code>200707121733</code>.
  * <p>
  * Applications should never construct or parse TaskAttemptID strings
  * , but rather use appropriate constructors or {@link #forName(String)} 
@@ -42,10 +42,9 @@ import java.io.IOException;
  * @see JobID
  * @see TaskID
  */
-public class TaskAttemptID extends ID {
-  private static final String ATTEMPT = "attempt";
+public class TaskAttemptID extends org.apache.hadoop.mapred.ID {
+  protected static final String ATTEMPT = "attempt";
   private TaskID taskId;
-  private static final char UNDERSCORE = '_';
   
   /**
    * Constructs a TaskAttemptID object from given {@link TaskID}.  
@@ -68,12 +67,14 @@ public class TaskAttemptID extends ID {
    * @param taskId taskId number
    * @param id the task attempt number
    */
-  public TaskAttemptID(String jtIdentifier, int jobId, boolean isMap
-      , int taskId, int id) {
+  public TaskAttemptID(String jtIdentifier, int jobId, boolean isMap, 
+                       int taskId, int id) {
     this(new TaskID(jtIdentifier, jobId, isMap, taskId), id);
   }
   
-  private TaskAttemptID() { }
+  public TaskAttemptID() { 
+    taskId = new TaskID();
+  }
   
   /** Returns the {@link JobID} object that this task attempt belongs to */
   public JobID getJobID() {
@@ -99,38 +100,19 @@ public class TaskAttemptID extends ID {
     return this.taskId.equals(that.taskId);
   }
   
-  /**Compare TaskIds by first tipIds, then by task numbers. */
-  @Override
-  public int compareTo(ID o) {
-    TaskAttemptID that = (TaskAttemptID)o;
-    int tipComp = this.taskId.compareTo(that.taskId);
-    if(tipComp == 0) {
-      return this.id - that.id;
-    }
-    else return tipComp;
-  }
-  @Override
-  public String toString() { 
-    StringBuilder builder = new StringBuilder();
-    return builder.append(ATTEMPT).append(UNDERSCORE)
-      .append(toStringWOPrefix()).toString();
-  }
-
-  StringBuilder toStringWOPrefix() {
-    StringBuilder builder = new StringBuilder();
-    return builder.append(taskId.toStringWOPrefix())
-      .append(UNDERSCORE).append(id);
-  }
-  
-  @Override
-  public int hashCode() {
-    return toStringWOPrefix().toString().hashCode();
+  /**
+   * Add the unique string to the StringBuilder
+   * @param builder the builder to append ot
+   * @return the builder that was passed in.
+   */
+  protected StringBuilder appendTo(StringBuilder builder) {
+    return taskId.appendTo(builder).append(SEPARATOR).append(id);
   }
   
   @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
-    this.taskId = TaskID.read(in);
+    taskId.readFields(in);
   }
 
   @Override
@@ -138,72 +120,55 @@ public class TaskAttemptID extends ID {
     super.write(out);
     taskId.write(out);
   }
-  
-  public static TaskAttemptID read(DataInput in) throws IOException {
-    TaskAttemptID taskId = new TaskAttemptID();
-    taskId.readFields(in);
-    return taskId;
+
+  @Override
+  public int hashCode() {
+    return taskId.hashCode() * 5 + id;
   }
   
+  /**Compare TaskIds by first tipIds, then by task numbers. */
+  @Override
+  public int compareTo(ID o) {
+    TaskAttemptID that = (TaskAttemptID)o;
+    int tipComp = this.taskId.compareTo(that.taskId);
+    if(tipComp == 0) {
+      return this.id - that.id;
+    }
+    else return tipComp;
+  }
+  @Override
+  public String toString() { 
+    return appendTo(new StringBuilder(ATTEMPT)).toString();
+  }
+
   /** Construct a TaskAttemptID object from given string 
    * @return constructed TaskAttemptID object or null if the given String is null
    * @throws IllegalArgumentException if the given string is malformed
    */
-  public static TaskAttemptID forName(String str) throws IllegalArgumentException {
+  public static TaskAttemptID forName(String str
+                                      ) throws IllegalArgumentException {
     if(str == null)
       return null;
     try {
-      String[] parts = str.split("_");
+      String[] parts = str.split(Character.toString(SEPARATOR));
       if(parts.length == 6) {
         if(parts[0].equals(ATTEMPT)) {
           boolean isMap = false;
           if(parts[3].equals("m")) isMap = true;
           else if(parts[3].equals("r")) isMap = false;
           else throw new Exception();
-          return new TaskAttemptID(parts[1], Integer.parseInt(parts[2]),
-              isMap, Integer.parseInt(parts[4]), Integer.parseInt(parts[5]));
+          return new org.apache.hadoop.mapred.TaskAttemptID
+                       (parts[1],
+                        Integer.parseInt(parts[2]),
+                        isMap, Integer.parseInt(parts[4]), 
+                        Integer.parseInt(parts[5]));
         }
       }
-    }catch (Exception ex) {//fall below
+    } catch (Exception ex) {
+      //fall below
     }
     throw new IllegalArgumentException("TaskAttemptId string : " + str 
         + " is not properly formed");
   }
-  
-  /** 
-   * Returns a regex pattern which matches task attempt IDs. Arguments can 
-   * be given null, in which case that part of the regex will be generic.  
-   * For example to obtain a regex matching <i>all task attempt IDs</i> 
-   * of <i>any jobtracker</i>, in <i>any job</i>, of the <i>first 
-   * map task</i>, we would use :
-   * <pre> 
-   * TaskAttemptID.getTaskAttemptIDsPattern(null, null, true, 1, null);
-   * </pre>
-   * which will return :
-   * <pre> "attempt_[^_]*_[0-9]*_m_000001_[0-9]*" </pre> 
-   * @param jtIdentifier jobTracker identifier, or null
-   * @param jobId job number, or null
-   * @param isMap whether the tip is a map, or null 
-   * @param taskId taskId number, or null
-   * @param attemptId the task attempt number, or null
-   * @return a regex pattern matching TaskAttemptIDs
-   */
-  public static String getTaskAttemptIDsPattern(String jtIdentifier,
-      Integer jobId, Boolean isMap, Integer taskId, Integer attemptId) {
-    StringBuilder builder = new StringBuilder(ATTEMPT).append(UNDERSCORE);
-    builder.append(getTaskAttemptIDsPatternWOPrefix(jtIdentifier, jobId,
-        isMap, taskId, attemptId));
-    return builder.toString();
-  }
-  
-  static StringBuilder getTaskAttemptIDsPatternWOPrefix(String jtIdentifier
-      , Integer jobId, Boolean isMap, Integer taskId, Integer attemptId) {
-    StringBuilder builder = new StringBuilder();
-    builder.append(TaskID.getTaskIDsPatternWOPrefix(jtIdentifier
-        , jobId, isMap, taskId))
-        .append(UNDERSCORE)
-        .append(attemptId != null ? attemptId : "[0-9]*");
-    return builder;
-  }
-  
+
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/TaskID.java src/mapred/org/apache/hadoop/mapreduce/TaskID.java
index ac02134..60377ff 100644
--- src/mapred/org/apache/hadoop/mapreduce/TaskID.java
+++ src/mapred/org/apache/hadoop/mapreduce/TaskID.java
@@ -45,10 +45,9 @@ import java.text.NumberFormat;
  * @see JobID
  * @see TaskAttemptID
  */
-public class TaskID extends ID {
-  private static final String TASK = "task";
-  private static char UNDERSCORE = '_';  
-  private static NumberFormat idFormat = NumberFormat.getInstance();
+public class TaskID extends org.apache.hadoop.mapred.ID {
+  protected static final String TASK = "task";
+  protected static final NumberFormat idFormat = NumberFormat.getInstance();
   static {
     idFormat.setGroupingUsed(false);
     idFormat.setMinimumIntegerDigits(6);
@@ -83,7 +82,9 @@ public class TaskID extends ID {
     this(new JobID(jtIdentifier, jobId), isMap, id);
   }
   
-  private TaskID() { }
+  public TaskID() { 
+    jobId = new JobID();
+  }
   
   /** Returns the {@link JobID} object that this tip belongs to */
   public JobID getJobID() {
@@ -118,31 +119,34 @@ public class TaskID extends ID {
     }
     else return jobComp;
   }
-  
   @Override
   public String toString() { 
-    StringBuilder builder = new StringBuilder();
-    return builder.append(TASK).append(UNDERSCORE)
-      .append(toStringWOPrefix()).toString();
+    return appendTo(new StringBuilder(TASK)).toString();
   }
 
-  StringBuilder toStringWOPrefix() {
-    StringBuilder builder = new StringBuilder();
-    builder.append(jobId.toStringWOPrefix())
-      .append(isMap ? "_m_" : "_r_");
-    return builder.append(idFormat.format(id));
+  /**
+   * Add the unique string to the given builder.
+   * @param builder the builder to append to
+   * @return the builder that was passed in
+   */
+  protected StringBuilder appendTo(StringBuilder builder) {
+    return jobId.appendTo(builder).
+                 append(SEPARATOR).
+                 append(isMap ? 'm' : 'r').
+                 append(SEPARATOR).
+                 append(idFormat.format(id));
   }
   
   @Override
   public int hashCode() {
-    return toStringWOPrefix().toString().hashCode();
+    return jobId.hashCode() * 524287 + id;
   }
   
   @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
-    this.jobId = JobID.read(in);
-    this.isMap = in.readBoolean();
+    jobId.readFields(in);
+    isMap = in.readBoolean();
   }
 
   @Override
@@ -152,12 +156,6 @@ public class TaskID extends ID {
     out.writeBoolean(isMap);
   }
   
-  public static TaskID read(DataInput in) throws IOException {
-    TaskID tipId = new TaskID();
-    tipId.readFields(in);
-    return tipId;
-  }
-  
   /** Construct a TaskID object from given string 
    * @return constructed TaskID object or null if the given String is null
    * @throws IllegalArgumentException if the given string is malformed
@@ -174,8 +172,10 @@ public class TaskID extends ID {
           if(parts[3].equals("m")) isMap = true;
           else if(parts[3].equals("r")) isMap = false;
           else throw new Exception();
-          return new TaskID(parts[1], Integer.parseInt(parts[2]),
-              isMap, Integer.parseInt(parts[4]));
+          return new org.apache.hadoop.mapred.TaskID(parts[1], 
+                                                     Integer.parseInt(parts[2]),
+                                                     isMap, 
+                                                     Integer.parseInt(parts[4]));
         }
       }
     }catch (Exception ex) {//fall below
@@ -184,37 +184,4 @@ public class TaskID extends ID {
         + " is not properly formed");
   }
   
-  /** 
-   * Returns a regex pattern which matches task IDs. Arguments can 
-   * be given null, in which case that part of the regex will be generic.  
-   * For example to obtain a regex matching <i>the first map task</i> 
-   * of <i>any jobtracker</i>, of <i>any job</i>, we would use :
-   * <pre> 
-   * TaskID.getTaskIDsPattern(null, null, true, 1);
-   * </pre>
-   * which will return :
-   * <pre> "task_[^_]*_[0-9]*_m_000001*" </pre> 
-   * @param jtIdentifier jobTracker identifier, or null
-   * @param jobId job number, or null
-   * @param isMap whether the tip is a map, or null 
-   * @param taskId taskId number, or null
-   * @return a regex pattern matching TaskIDs
-   */
-  public static String getTaskIDsPattern(String jtIdentifier, Integer jobId
-      , Boolean isMap, Integer taskId) {
-    StringBuilder builder = new StringBuilder(TASK).append(UNDERSCORE)
-      .append(getTaskIDsPatternWOPrefix(jtIdentifier, jobId, isMap, taskId));
-    return builder.toString();
-  }
-  
-  static StringBuilder getTaskIDsPatternWOPrefix(String jtIdentifier
-      , Integer jobId, Boolean isMap, Integer taskId) {
-    StringBuilder builder = new StringBuilder();
-    builder.append(JobID.getJobIDsPatternWOPrefix(jtIdentifier, jobId))
-      .append(UNDERSCORE)
-      .append(isMap != null ? (isMap ? "m" : "r") : "(m|r)").append(UNDERSCORE)
-      .append(taskId != null ? idFormat.format(taskId) : "[0-9]*");
-    return builder;
-  }
-  
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/TaskInputOutputContext.java src/mapred/org/apache/hadoop/mapreduce/TaskInputOutputContext.java
index 18c7196..a618935 100644
--- src/mapred/org/apache/hadoop/mapreduce/TaskInputOutputContext.java
+++ src/mapred/org/apache/hadoop/mapreduce/TaskInputOutputContext.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Progressable;
 
 /**
  * A context object that allows input and output from the task. It is only
@@ -31,34 +32,67 @@ import org.apache.hadoop.conf.Configuration;
  * @param <VALUEOUT> the output value type for the task
  */
 public abstract class TaskInputOutputContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
-    extends TaskAttemptContext {
+    extends TaskAttemptContext implements Progressable {
+  private RecordWriter<KEYOUT,VALUEOUT> output;
+  private StatusReporter reporter;
 
-  public TaskInputOutputContext(Configuration conf, TaskAttemptID taskid) {
+  public TaskInputOutputContext(Configuration conf, TaskAttemptID taskid,
+                                RecordWriter<KEYOUT,VALUEOUT> output,
+                                StatusReporter reporter) {
     super(conf, taskid);
+    this.output = output;
+    this.reporter = reporter;
   }
 
   /**
-   * Advance to the next key, returning null if at end.
-   * @param key the key object to read in to, which may be null
-   * @return the key object that was read into
+   * Advance to the next key, value pair, returning null if at end.
+   * @return the key object that was read into, or null if no more
    */
-  public abstract KEYIN nextKey(KEYIN key
-                                ) throws IOException, InterruptedException;
-  
+  public abstract 
+  boolean nextKeyValue() throws IOException, InterruptedException;
+ 
   /**
-   * Read the next value. Must be called after nextKey.
+   * Get the current key.
+   * @return the current key object or null if there isn't one
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract 
+  KEYIN getCurrentKey() throws IOException, InterruptedException;
+
+  /**
+   * Get the current value.
    * @param value the value object to read in to, which may be null
    * @return the value object that was read into
    * @throws IOException
    * @throws InterruptedException
    */
-  public abstract VALUEIN nextValue(VALUEIN value
-                                    ) throws IOException, InterruptedException;
+  public abstract VALUEIN getCurrentValue() throws IOException, 
+                                                   InterruptedException;
 
   /**
    * Generate an output key/value pair.
    */
-  public abstract void collect(KEYOUT key, VALUEOUT value
-                               ) throws IOException, InterruptedException;
+  public void write(KEYOUT key, VALUEOUT value
+                    ) throws IOException, InterruptedException {
+    output.write(key, value);
+  }
 
+  public Counter getCounter(Enum<?> counterName) {
+    return reporter.getCounter(counterName);
+  }
+
+  public Counter getCounter(String groupName, String counterName) {
+    return reporter.getCounter(groupName, counterName);
+  }
+
+  @Override
+  public void progress() {
+    reporter.progress();
+  }
+
+  @Override
+  public void setStatus(String status) {
+    reporter.setStatus(status);
+  }
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/example/WordCount.java src/mapred/org/apache/hadoop/mapreduce/example/WordCount.java
deleted file mode 100644
index a79123c..0000000
--- src/mapred/org/apache/hadoop/mapreduce/example/WordCount.java
+++ /dev/null
@@ -1,69 +0,0 @@
-package org.apache.hadoop.mapreduce.example;
-
-import java.io.IOException;
-import java.util.StringTokenizer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.GenericOptionsParser;
-
-public class WordCount {
-
-  public static class TokenizerMapper 
-       extends Mapper<Object, Text, Text, IntWritable>{
-    
-    private final static IntWritable one = new IntWritable(1);
-    private Text word = new Text();
-      
-    public void map(Object key, Text value, Context context
-                    ) throws IOException, InterruptedException {
-      StringTokenizer itr = new StringTokenizer(value.toString());
-      while (itr.hasMoreTokens()) {
-        word.set(itr.nextToken());
-        context.collect(word, one);
-      }
-    }
-  }
-  
-  public static class IntSumReducer 
-       extends Reducer<Text,IntWritable,Text,IntWritable> {
-    private IntWritable result = new IntWritable();
-
-    public void reduce(Text key, Iterable<IntWritable> values, 
-                       Context context
-                       ) throws IOException, InterruptedException {
-      int sum = 0;
-      for (IntWritable val : values) {
-        sum += val.get();
-      }
-      result.set(sum);
-      context.collect(key, result);
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
-    Configuration conf = new Configuration();
-    GenericOptionsParser parser = new GenericOptionsParser(conf, args);
-    String[] otherArgs = parser.getRemainingArgs();
-    if (otherArgs.length != 2) {
-      System.err.println("Usage: wordcount <in> <out>");
-      System.exit(2);
-    }
-    Job job = new Job(conf, "word count");
-    job.setMapperClass(TokenizerMapper.class);
-    job.setCombinerClass(IntSumReducer.class);
-    job.setReducerClass(IntSumReducer.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(IntWritable.class);
-    FileInputFormat.addInputPath(conf, new Path(otherArgs[0]));
-    FileOutputFormat.setOutputPath(conf, new Path(otherArgs[1]));
-    System.exit(job.waitForCompletion() ? 0 : 1);
-  }
-}
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java src/mapred/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
index 6db56ff..a7417b4 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -108,26 +109,37 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
 
   /**
    * Set a PathFilter to be applied to the input paths for the map-reduce job.
-   *
+   * @param job the job to modify
    * @param filter the PathFilter class use for filtering the input paths.
    */
-  public static void setInputPathFilter(Configuration conf,
+  public static void setInputPathFilter(Job job,
                                         Class<? extends PathFilter> filter) {
-    conf.setClass("mapred.input.pathFilter.class", filter, PathFilter.class);
+    job.getConfiguration().setClass("mapred.input.pathFilter.class", filter, 
+                                    PathFilter.class);
   }
 
-  public static void setMinInputSplitSize(Configuration conf,
+  /**
+   * Set the minimum input split size
+   * @param job the job to modify
+   * @param size the minimum size
+   */
+  public static void setMinInputSplitSize(Job job,
                                           long size) {
-    conf.setLong("mapred.min.split.size", size);
+    job.getConfiguration().setLong("mapred.min.split.size", size);
   }
 
   public static long getMinSplitSize(Configuration conf) {
     return conf.getLong("mapred.min.split.size", 1L);
   }
 
-  public static void setMaxInputSplitSize(Configuration conf,
+  /**
+   * Set the maximum split size
+   * @param job the job to modify
+   * @param size the maximum split size
+   */
+  public static void setMaxInputSplitSize(Job job,
                                           long size) {
-    conf.setLong("mapred.max.split.size", size);
+    job.getConfiguration().setLong("mapred.max.split.size", size);
   }
 
   public static long getMaxSplitSize(Configuration conf) {
@@ -271,14 +283,14 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
    * Sets the given comma separated paths as the list of inputs 
    * for the map-reduce job.
    * 
-   * @param conf Configuration of the job
+   * @param job the job
    * @param commaSeparatedPaths Comma separated paths to be set as 
    *        the list of inputs for the map-reduce job.
    */
-  public static void setInputPaths(Configuration conf, 
+  public static void setInputPaths(Job job, 
                                    String commaSeparatedPaths
                                    ) throws IOException {
-    setInputPaths(conf, StringUtils.stringToPath(
+    setInputPaths(job, StringUtils.stringToPath(
                         getPathStrings(commaSeparatedPaths)));
   }
 
@@ -286,15 +298,15 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
    * Add the given comma separated paths to the list of inputs for
    *  the map-reduce job.
    * 
-   * @param conf The configuration of the job 
+   * @param job The job to modify
    * @param commaSeparatedPaths Comma separated paths to be added to
    *        the list of inputs for the map-reduce job.
    */
-  public static void addInputPaths(Configuration conf, 
+  public static void addInputPaths(Job job, 
                                    String commaSeparatedPaths
                                    ) throws IOException {
     for (String str : getPathStrings(commaSeparatedPaths)) {
-      addInputPath(conf, new Path(str));
+      addInputPath(job, new Path(str));
     }
   }
 
@@ -306,8 +318,9 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
    * @param inputPaths the {@link Path}s of the input directories/files 
    * for the map-reduce job.
    */ 
-  public static void setInputPaths(Configuration conf, 
+  public static void setInputPaths(Job job, 
                                    Path... inputPaths) throws IOException {
+    Configuration conf = job.getConfiguration();
     FileSystem fs = FileSystem.get(conf);
     Path path = inputPaths[0].makeQualified(fs);
     StringBuffer str = new StringBuffer(StringUtils.escapeString(path.toString()));
@@ -326,8 +339,9 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
    * @param path {@link Path} to be added to the list of inputs for 
    *            the map-reduce job.
    */
-  public static void addInputPath(Configuration conf, 
+  public static void addInputPath(Job job, 
                                   Path path) throws IOException {
+    Configuration conf = job.getConfiguration();
     FileSystem fs = FileSystem.get(conf);
     path = path.makeQualified(fs);
     String dirStr = StringUtils.escapeString(path.toString());
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/input/InvalidInputException.java src/mapred/org/apache/hadoop/mapreduce/lib/input/InvalidInputException.java
index 8bd71f3..ffdf545 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/input/InvalidInputException.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/input/InvalidInputException.java
@@ -27,6 +27,7 @@ import java.util.Iterator;
  * by one.
  */
 public class InvalidInputException extends IOException {
+  private static final long serialVersionUID = -380668190578456802L;
   private List<IOException> problems;
   
   /**
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java src/mapred/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
index 0a3e947..a54b0d7 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
@@ -46,7 +46,9 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
   private long pos;
   private long end;
   private LineReader in;
-  int maxLineLength;
+  private int maxLineLength;
+  private LongWritable key = null;
+  private Text value = null;
 
   public void initialize(InputSplit genericSplit,
                          TaskAttemptContext context) throws IOException {
@@ -82,25 +84,21 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
     this.pos = start;
   }
   
-  public LongWritable nextKey(LongWritable key) throws IOException {
+  public boolean nextKeyValue() throws IOException {
     if (key == null) {
       key = new LongWritable();
     }
     key.set(pos);
-    return key;
-  }
-
-  public Text nextValue(Text value) throws IOException {
     if (value == null) {
       value = new Text();
     }
+    int newSize = 0;
     while (pos < end) {
-      int newSize = in.readLine(value, maxLineLength,
-                                Math.max((int)Math.min(Integer.MAX_VALUE, 
-                                                       end-pos),
-                                         maxLineLength));
+      newSize = in.readLine(value, maxLineLength,
+                            Math.max((int)Math.min(Integer.MAX_VALUE, end-pos),
+                                     maxLineLength));
       if (newSize == 0) {
-        return null;
+        break;
       }
       pos += newSize;
       if (newSize < maxLineLength) {
@@ -111,6 +109,22 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
       LOG.info("Skipped line of size " + newSize + " at pos " + 
                (pos - newSize));
     }
+    if (newSize == 0) {
+      key = null;
+      value = null;
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  @Override
+  public LongWritable getCurrentKey() {
+    return key;
+  }
+
+  @Override
+  public Text getCurrentValue() {
     return value;
   }
 
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileRecordReader.java src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileRecordReader.java
index 4e20e8d..f15657e 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileRecordReader.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileRecordReader.java
@@ -35,6 +35,8 @@ public class SequenceFileRecordReader<K, V> extends RecordReader<K, V> {
   private long start;
   private long end;
   private boolean more = true;
+  private K key = null;
+  private V value = null;
   protected Configuration conf;
   
   @Override
@@ -58,23 +60,30 @@ public class SequenceFileRecordReader<K, V> extends RecordReader<K, V> {
 
   @Override
   @SuppressWarnings("unchecked")
-  public K nextKey(K key) throws IOException, InterruptedException {
+  public boolean nextKeyValue() throws IOException, InterruptedException {
     if (!more) {
-      return null;
+      return false;
     }
     long pos = in.getPosition();
-    K result = (K) in.next(key);
-    if (result == null || (pos >= end && in.syncSeen())) {
+    key = (K) in.next(key);
+    if (key == null || (pos >= end && in.syncSeen())) {
       more = false;
-      result = null;
+      key = null;
+      value = null;
+    } else {
+      value = (V) in.getCurrentValue(value);
     }
-    return result;
+    return more;
   }
 
   @Override
-  @SuppressWarnings("unchecked")
-  public V nextValue(V value) throws IOException, InterruptedException {
-    return (V) in.getCurrentValue(value);
+  public K getCurrentKey() {
+    return key;
+  }
+  
+  @Override
+  public V getCurrentValue() {
+    return value;
   }
   
   /**
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/map/InverseMapper.java src/mapred/org/apache/hadoop/mapreduce/lib/map/InverseMapper.java
index cf3fc04..7b93f00 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/map/InverseMapper.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/map/InverseMapper.java
@@ -28,7 +28,7 @@ public class InverseMapper<K, V> extends Mapper<K,V,V,K> {
   /** The inverse function.  Input keys and values are swapped.*/
   public void map(K key, V value, Context context
                   ) throws IOException, InterruptedException {
-    context.collect(value, key);
+    context.write(value, key);
   }
   
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java src/mapred/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java
index 71c77f3..2c483db 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java
@@ -23,10 +23,16 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.StatusReporter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * Multithreaded implementation for @link org.apache.hadoop.mapreduce.Mapper.
@@ -50,7 +56,7 @@ public class MultithreadedMapper<K1, V1, K2, V2>
   private static final Log LOG = LogFactory.getLog(MultithreadedMapper.class);
   private Class<Mapper<K1,V1,K2,V2>> mapClass;
   private Context outer;
-  private MapRunner[] runners;
+  private List<MapRunner> runners;
 
   public static int getNumberOfThreads(Configuration conf) {
     return conf.getInt("mapred.map.multithreadedrunner.threads", 10);
@@ -88,14 +94,16 @@ public class MultithreadedMapper<K1, V1, K2, V2>
                 " threads");
     }
     
-    runners = (MapRunner[]) new Object[numberOfThreads];
+    runners =  new ArrayList<MapRunner>(numberOfThreads);
     for(int i=0; i < numberOfThreads; ++i) {
-      runners[i] = new MapRunner();
-      runners[i].start();
+      MapRunner thread = new MapRunner();
+      thread.start();
+      runners.set(i, thread);
     }
     for(int i=0; i < numberOfThreads; ++i) {
-      runners[i].join();
-      Throwable th = runners[i].throwable;
+      MapRunner thread = runners.get(i);
+      thread.join();
+      Throwable th = thread.throwable;
       if (th != null) {
         if (th instanceof IOException) {
           throw (IOException) th;
@@ -108,64 +116,89 @@ public class MultithreadedMapper<K1, V1, K2, V2>
     }
   }
 
-  private class SubMapContext extends Context {
+  private class SubMapRecordReader extends RecordReader<K1,V1> {
     private K1 key;
     private V1 value;
-    
-    SubMapContext() {
-      super(outer.getConfiguration(), outer.getTaskAttemptId());
+    private Configuration conf;
+
+    @Override
+    public void close() throws IOException {
     }
 
     @Override
-    public InputSplit getInputSplit() {
-      synchronized (outer) {
-        return outer.getInputSplit();
-      }
+    public float getProgress() throws IOException, InterruptedException {
+      return 0;
     }
 
     @Override
-    public Counter getCounter(Enum<?> counterName) {
-      synchronized (outer) {
-        return outer.getCounter(counterName);
-      }
+    public void initialize(InputSplit split, 
+                           TaskAttemptContext context
+                           ) throws IOException, InterruptedException {
+      conf = context.getConfiguration();
     }
 
+
     @Override
-    public Counter getCounter(String groupName, String counterName) {
+    public boolean nextKeyValue() throws IOException, InterruptedException {
       synchronized (outer) {
-        return outer.getCounter(groupName, counterName);
+        if (!outer.nextKeyValue()) {
+          return false;
+        }
+        ReflectionUtils.cloneInto(outer.getConfiguration(),
+                                  outer.getCurrentKey(), key);
+        ReflectionUtils.cloneInto(conf, outer.getCurrentValue(), value);
+        return true;
       }
     }
 
+    public K1 getCurrentKey() {
+      return key;
+    }
+
     @Override
-    public void progress() {
-      synchronized (outer) {
-        outer.progress();
-      }
+    public V1 getCurrentValue() {
+      return value;
     }
+  }
+  
+  private class SubMapRecordWriter extends RecordWriter<K2,V2> {
 
     @Override
-    public void collect(K2 key, V2 value) throws IOException,
-                                         InterruptedException {
-      synchronized (outer) {
-        outer.collect(key, value);
-      }
+    public void close(TaskAttemptContext context) throws IOException,
+                                                 InterruptedException {
     }
 
     @Override
-    public K1 nextKey(K1 k) throws IOException, InterruptedException {
+    public void write(K2 key, V2 value) throws IOException,
+                                               InterruptedException {
       synchronized (outer) {
-        key = outer.nextKey(key);
-        if (key != null) {
-          value = outer.nextValue(value);
-        }
-        return key;
+        outer.write(key, value);
       }
+    }  
+  }
+
+  private class SubMapStatusReporter extends StatusReporter {
+
+    @Override
+    public Counter getCounter(Enum<?> name) {
+      return outer.getCounter(name);
     }
-    
-    public V1 nextValue(V1 v) throws IOException, InterruptedException {
-      return value;
+
+    @Override
+    public Counter getCounter(String group, String name) {
+      return outer.getCounter(group, name);
+    }
+
+    @Override
+    public void progress() {
+      outer.progress();
     }
+
+    @Override
+    public void setStatus(String status) {
+      outer.setStatus(status);
+    }
+    
   }
 
   private class MapRunner extends Thread {
@@ -173,11 +206,15 @@ public class MultithreadedMapper<K1, V1, K2, V2>
     private Context context;
     private Throwable throwable;
 
-    @SuppressWarnings("unchecked")
-    MapRunner() {
-      mapper = (Mapper<K1,V1,K2,V2>) 
-        ReflectionUtils.newInstance(mapClass, context.getConfiguration());
-      context = new SubMapContext();
+    MapRunner() throws IOException, InterruptedException {
+      mapper = ReflectionUtils.newInstance(mapClass, 
+                                           context.getConfiguration());
+      context = new Context(outer.getConfiguration(), 
+                            outer.getTaskAttemptId(),
+                            new SubMapRecordReader(),
+                            new SubMapRecordWriter(), 
+                            new SubMapStatusReporter(),
+                            outer.getInputSplit());
     }
 
     public Throwable getThrowable() {
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/map/TokenCounterMapper.java src/mapred/org/apache/hadoop/mapreduce/lib/map/TokenCounterMapper.java
index 5810b8e..258fee6 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/map/TokenCounterMapper.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/map/TokenCounterMapper.java
@@ -35,7 +35,7 @@ public class TokenCounterMapper extends Mapper<Object, Text, Text, IntWritable>{
     StringTokenizer itr = new StringTokenizer(value.toString());
     while (itr.hasMoreTokens()) {
       word.set(itr.nextToken());
-      context.collect(word, one);
+      context.write(word, one);
     }
   }
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java src/mapred/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
index 02831f5..d44af41 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.InvalidJobConfException;
+import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.RecordWriter;
@@ -35,14 +36,14 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 /** A base class for {@link OutputFormat}s that read from {@link FileSystem}s.*/
 public abstract class FileOutputFormat<K, V> extends OutputFormat<K, V> {
 
-  private static final String TEMP_DIR_NAME = "_temp";
+  private static final String TEMP_DIR_NAME = "_temporary";
   /**
    * Set whether the output of the job is compressed.
-   * @param conf the {@link Configuration} to modify
+   * @param job the job to modify
    * @param compress should the output of the job be compressed?
    */
-  public static void setCompressOutput(Configuration conf, boolean compress) {
-    conf.setBoolean("mapred.output.compress", compress);
+  public static void setCompressOutput(Job job, boolean compress) {
+    job.getConfiguration().setBoolean("mapred.output.compress", compress);
   }
   
   /**
@@ -57,16 +58,17 @@ public abstract class FileOutputFormat<K, V> extends OutputFormat<K, V> {
   
   /**
    * Set the {@link CompressionCodec} to be used to compress job outputs.
-   * @param conf the {@link Configuration} to modify
+   * @param job the job to modify
    * @param codecClass the {@link CompressionCodec} to be used to
    *                   compress the job outputs
    */
   public static void 
-  setOutputCompressorClass(Configuration conf, 
+  setOutputCompressorClass(Job job, 
                            Class<? extends CompressionCodec> codecClass) {
-    setCompressOutput(conf, true);
-    conf.setClass("mapred.output.compression.codec", codecClass, 
-                  CompressionCodec.class);
+    setCompressOutput(job, true);
+    job.getConfiguration().setClass("mapred.output.compression.codec", 
+                                    codecClass, 
+                                    CompressionCodec.class);
   }
   
   /**
@@ -117,12 +119,12 @@ public abstract class FileOutputFormat<K, V> extends OutputFormat<K, V> {
   /**
    * Set the {@link Path} of the output directory for the map-reduce job.
    *
-   * @param conf The configuration of the job.
+   * @param job The job to modify
    * @param outputDir the {@link Path} of the output directory for 
    * the map-reduce job.
    */
-  public static void setOutputPath(Configuration conf, Path outputDir) {
-    conf.set("mapred.output.dir", outputDir.toString());
+  public static void setOutputPath(Job job, Path outputDir) {
+    job.getConfiguration().set("mapred.output.dir", outputDir.toString());
   }
 
   /**
@@ -203,20 +205,8 @@ public abstract class FileOutputFormat<K, V> extends OutputFormat<K, V> {
       throw new IOException("Undefined job output-path");
     }
 
-    // ${mapred.out.dir}/_temporary
-    Path jobTmpDir = new Path(outputPath, TEMP_DIR_NAME);
-    FileSystem fs = jobTmpDir.getFileSystem(conf);
-    if (!fs.exists(jobTmpDir)) {
-      throw new IOException("The temporary job-output directory " + 
-          jobTmpDir.toString() + " doesn't exist!"); 
-    }
-
     // ${mapred.out.dir}/_temporary/_${taskid}
     Path taskTmpDir = getWorkOutputPath(conf);
-    if (!fs.mkdirs(taskTmpDir)) {
-      throw new IOException("Mkdirs failed to create " 
-          + taskTmpDir.toString());
-    }
     
     // ${mapred.out.dir}/_temporary/_${taskid}/${name}
     return new Path(taskTmpDir, getOutputName(context));
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/output/SequenceFileOutputFormat.java src/mapred/org/apache/hadoop/mapreduce/lib/output/SequenceFileOutputFormat.java
index 62fb0bc..5898f72 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/output/SequenceFileOutputFormat.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/output/SequenceFileOutputFormat.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
@@ -94,10 +95,11 @@ public class SequenceFileOutputFormat <K,V> extends FileOutputFormat<K, V> {
    * @param style the {@link CompressionType} for the output
    *              {@link SequenceFile} 
    */
-  public static void setOutputCompressionType(Configuration conf, 
+  public static void setOutputCompressionType(Job job, 
 		                                          CompressionType style) {
-    setCompressOutput(conf, true);
-    conf.set("mapred.output.compression.type", style.toString());
+    setCompressOutput(job, true);
+    job.getConfiguration().set("mapred.output.compression.type", 
+                               style.toString());
   }
 
 }
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/output/TextOutputFormat.java src/mapred/org/apache/hadoop/mapreduce/lib/output/TextOutputFormat.java
index 163fdbf..d72ed0f 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/output/TextOutputFormat.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/output/TextOutputFormat.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.util.*;
 public class TextOutputFormat<K, V> extends FileOutputFormat<K, V> {
 
   protected static class LineRecordWriter<K, V>
-    implements RecordWriter<K, V> {
+    extends RecordWriter<K, V> {
     private static final String utf8 = "UTF-8";
     private static final byte[] newline;
     static {
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/reduce/IntSumReducer.java src/mapred/org/apache/hadoop/mapreduce/lib/reduce/IntSumReducer.java
index 4f2421e..d0bc0f2 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/reduce/IntSumReducer.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/reduce/IntSumReducer.java
@@ -34,7 +34,7 @@ public class IntSumReducer<Key> extends Reducer<Key,IntWritable,
       sum += val.get();
     }
     result.set(sum);
-    context.collect(key, result);
+    context.write(key, result);
   }
 
 }
\ No newline at end of file
diff --git src/mapred/org/apache/hadoop/mapreduce/lib/reduce/LongSumReducer.java src/mapred/org/apache/hadoop/mapreduce/lib/reduce/LongSumReducer.java
index 17a3a57..19a45c6 100644
--- src/mapred/org/apache/hadoop/mapreduce/lib/reduce/LongSumReducer.java
+++ src/mapred/org/apache/hadoop/mapreduce/lib/reduce/LongSumReducer.java
@@ -34,7 +34,7 @@ public class LongSumReducer<KEY> extends Reducer<KEY, LongWritable,
       sum += val.get();
     }
     result.set(sum);
-    context.collect(key, result);
+    context.write(key, result);
   }
 
 }
\ No newline at end of file
diff --git src/test/org/apache/hadoop/mapred/TestFileOutputCommitter.java src/test/org/apache/hadoop/mapred/TestFileOutputCommitter.java
index 9e88345..1662adb 100644
--- src/test/org/apache/hadoop/mapred/TestFileOutputCommitter.java
+++ src/test/org/apache/hadoop/mapred/TestFileOutputCommitter.java
@@ -37,7 +37,7 @@ public class TestFileOutputCommitter extends TestCase {
     JobConf job = new JobConf();
     job.set("mapred.task.id", attempt);
     job.setOutputCommitter(FileOutputCommitter.class);
-    JobContext jContext = new JobContext(job);
+    JobContext jContext = new JobContext(job, taskID.getJobID());
     TaskAttemptContext tContext = new TaskAttemptContext(job, taskID);
     FileOutputFormat.setOutputPath(job, outDir);
     FileOutputCommitter committer = new FileOutputCommitter();
