Bläddra i källkod

HADOOP-1230. This is a proposed patch that fills in more of the new API. Doug
suggested checking it in, because it will be easier to review and is not called
by any code. This code compiles, but has not been tested at all...


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@679845 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 17 år sedan
förälder
incheckning
adc1f57269
38 ändrade filer med 4438 tillägg och 0 borttagningar
  1. 92 0
      src/mapred/org/apache/hadoop/mapreduce/Counter.java
  2. 27 0
      src/mapred/org/apache/hadoop/mapreduce/CounterGroup.java
  3. 111 0
      src/mapred/org/apache/hadoop/mapreduce/ID.java
  4. 103 0
      src/mapred/org/apache/hadoop/mapreduce/InputFormat.java
  5. 58 0
      src/mapred/org/apache/hadoop/mapreduce/InputSplit.java
  6. 312 0
      src/mapred/org/apache/hadoop/mapreduce/Job.java
  7. 273 0
      src/mapred/org/apache/hadoop/mapreduce/JobContext.java
  8. 184 0
      src/mapred/org/apache/hadoop/mapreduce/JobID.java
  9. 43 0
      src/mapred/org/apache/hadoop/mapreduce/MapContext.java
  10. 147 0
      src/mapred/org/apache/hadoop/mapreduce/Mapper.java
  11. 70 0
      src/mapred/org/apache/hadoop/mapreduce/OutputFormat.java
  12. 47 0
      src/mapred/org/apache/hadoop/mapreduce/Partitioner.java
  13. 75 0
      src/mapred/org/apache/hadoop/mapreduce/RecordReader.java
  14. 52 0
      src/mapred/org/apache/hadoop/mapreduce/RecordWriter.java
  15. 45 0
      src/mapred/org/apache/hadoop/mapreduce/ReduceContext.java
  16. 170 0
      src/mapred/org/apache/hadoop/mapreduce/Reducer.java
  17. 71 0
      src/mapred/org/apache/hadoop/mapreduce/TaskAttemptContext.java
  18. 212 0
      src/mapred/org/apache/hadoop/mapreduce/TaskAttemptID.java
  19. 224 0
      src/mapred/org/apache/hadoop/mapreduce/TaskID.java
  20. 64 0
      src/mapred/org/apache/hadoop/mapreduce/TaskInputOutputContext.java
  21. 69 0
      src/mapred/org/apache/hadoop/mapreduce/example/WordCount.java
  22. 393 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
  23. 91 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/FileSplit.java
  24. 63 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/InvalidInputException.java
  25. 277 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
  26. 67 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileInputFormat.java
  27. 95 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileRecordReader.java
  28. 50 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/TextInputFormat.java
  29. 34 0
      src/mapred/org/apache/hadoop/mapreduce/lib/map/InverseMapper.java
  30. 196 0
      src/mapred/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java
  31. 41 0
      src/mapred/org/apache/hadoop/mapreduce/lib/map/TokenCounterMapper.java
  32. 289 0
      src/mapred/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
  33. 39 0
      src/mapred/org/apache/hadoop/mapreduce/lib/output/NullOutputFormat.java
  34. 104 0
      src/mapred/org/apache/hadoop/mapreduce/lib/output/SequenceFileOutputFormat.java
  35. 138 0
      src/mapred/org/apache/hadoop/mapreduce/lib/output/TextOutputFormat.java
  36. 32 0
      src/mapred/org/apache/hadoop/mapreduce/lib/partition/HashPartitioner.java
  37. 40 0
      src/mapred/org/apache/hadoop/mapreduce/lib/reduce/IntSumReducer.java
  38. 40 0
      src/mapred/org/apache/hadoop/mapreduce/lib/reduce/LongSumReducer.java

+ 92 - 0
src/mapred/org/apache/hadoop/mapreduce/Counter.java

@@ -0,0 +1,92 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * A named counter that tracks the progress of a map/reduce job.
+ * 
+ * <p><code>Counters</code> represent global counters, defined either by the 
+ * Map-Reduce framework or applications. Each <code>Counter</code> is named by
+ * an {@link Enum} and has a long for the value.</p>
+ * 
+ * <p><code>Counters</code> are bunched into Groups, each comprising of
+ * counters from a particular <code>Enum</code> class. 
+ */
+public class Counter implements Writable {
+
+  private String displayName;
+  private long value;
+    
+  Counter() { 
+    value = 0L;
+  }
+
+  Counter(String displayName, long value) {
+    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);
+    value = WritableUtils.readVLong(in);
+  }
+    
+  /**
+   * Write the binary representation of the counter
+   */
+  public synchronized void write(DataOutput out) throws IOException {
+    Text.writeString(out, displayName);
+    WritableUtils.writeVLong(out, value);
+  }
+    
+  /**
+   * Get the name of the counter.
+   * @return the user facing name of the counter
+   */
+  public String getDisplayName() {
+    return displayName;
+  }
+    
+  /**
+   * What is the current value of this counter?
+   * @return the current value
+   */
+  public synchronized long getValue() {
+    return value;
+  }
+    
+  /**
+   * Increment this counter by the given value
+   * @param incr the value to increase this counter by
+   */
+  public synchronized void increment(long incr) {
+    value += incr;
+  }
+}

+ 27 - 0
src/mapred/org/apache/hadoop/mapreduce/CounterGroup.java

@@ -0,0 +1,27 @@
+/**
+ * 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;
+
+/**
+ * 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();
+}

+ 111 - 0
src/mapred/org/apache/hadoop/mapreduce/ID.java

@@ -0,0 +1,111 @@
+/**
+ * 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;
+
+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}, 
+ * {@link TaskID} and {@link TaskAttemptID}.
+ * 
+ * @see JobID
+ * @see TaskID
+ * @see TaskAttemptID
+ */
+public class ID implements WritableComparable<ID> {
+  protected int id;
+
+  /** constructs an ID object from the given int */
+  public ID(int id) {
+    this.id = 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(o == null)
+      return false;
+    if (o.getClass().equals(ID.class)) {
+      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);
+  }
+
+  public static ID read(DataInput in) throws IOException {
+    ID id = new ID();
+    id.readFields(in);
+    return id;
+  }
+
+  /**
+   * Construct an ID object from given string
+   * 
+   * @return constructed Id object or null if the given String is null
+   * @throws IllegalArgumentException if the given string is malformed
+   */
+  public static ID forName(String str) throws IllegalArgumentException {
+    if (str == null)
+      return null;
+    try {
+      int id = Integer.parseInt(str);
+      return new ID(id);
+    }
+    catch (Exception ex) {
+      throw new IllegalArgumentException("Id string : " + str
+          + " is not propoerly formed");
+    }
+  }
+}

+ 103 - 0
src/mapred/org/apache/hadoop/mapreduce/InputFormat.java

@@ -0,0 +1,103 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+/** 
+ * <code>InputFormat</code> describes the input-specification for a 
+ * Map-Reduce job. 
+ * 
+ * <p>The Map-Reduce framework relies on the <code>InputFormat</code> of the
+ * job to:<p>
+ * <ol>
+ *   <li>
+ *   Validate the input-specification of the job. 
+ *   <li>
+ *   Split-up the input file(s) into logical {@link InputSplit}s, each of 
+ *   which is then assigned to an individual {@link Mapper}.
+ *   </li>
+ *   <li>
+ *   Provide the {@link RecordReader} implementation to be used to glean
+ *   input records from the logical <code>InputSplit</code> for processing by 
+ *   the {@link Mapper}.
+ *   </li>
+ * </ol>
+ * 
+ * <p>The default behavior of file-based {@link InputFormat}s, typically 
+ * sub-classes of {@link FileInputFormat}, is to split the 
+ * input into <i>logical</i> {@link InputSplit}s based on the total size, in 
+ * bytes, of the input files. However, the {@link FileSystem} blocksize of  
+ * the input files is treated as an upper bound for input splits. A lower bound 
+ * on the split size can be set via 
+ * <a href="{@docRoot}/../hadoop-default.html#mapred.min.split.size">
+ * mapred.min.split.size</a>.</p>
+ * 
+ * <p>Clearly, logical splits based on input-size is insufficient for many 
+ * applications since record boundaries are to respected. In such cases, the
+ * application has to also implement a {@link RecordReader} on whom lies the
+ * responsibility to respect record-boundaries and present a record-oriented
+ * view of the logical <code>InputSplit</code> to the individual task.
+ *
+ * @see InputSplit
+ * @see RecordReader
+ * @see FileInputFormat
+ */
+public abstract class InputFormat<K, V> {
+
+  /** 
+   * Logically split the set of input files for the job.  
+   * 
+   * <p>Each {@link InputSplit} is then assigned to an individual {@link Mapper}
+   * for processing.</p>
+   *
+   * <p><i>Note</i>: The split is a <i>logical</i> split of the inputs and the
+   * input files are not physically split into chunks. For e.g. a split could
+   * be <i>&lt;input-file-path, start, offset&gt;</i> tuple. The InputFormat
+   * also creates the {@link RecordReader} to read the {@link InputSplit}.
+   * 
+   * @param context job configuration.
+   * @return an array of {@link InputSplit}s for the job.
+   */
+  public abstract 
+    List<InputSplit> getSplits(JobContext context
+                               ) throws IOException, InterruptedException;
+  
+  /**
+   * Create a record reader for a given split. The framework will call
+   * {@link RecordReader#initialize(InputSplit, TaskAttemptContext)} before
+   * the split is used.
+   * @param split the split to be read
+   * @param context the information about the task
+   * @return a new record reader
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract 
+    RecordReader<K,V> createRecordReader(InputSplit split,
+                                         TaskAttemptContext context
+                                        ) throws IOException, 
+                                                 InterruptedException;
+
+}
+

+ 58 - 0
src/mapred/org/apache/hadoop/mapreduce/InputSplit.java

@@ -0,0 +1,58 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.RecordReader;
+
+/**
+ * <code>InputSplit</code> represents the data to be processed by an 
+ * individual {@link Mapper}. 
+ *
+ * <p>Typically, it presents a byte-oriented view on the input and is the 
+ * responsibility of {@link RecordReader} of the job to process this and present
+ * a record-oriented view. Although the type is Writable, only the InputFormat
+ * 
+ * @see InputFormat
+ * @see RecordReader
+ */
+public abstract class InputSplit implements Writable {
+  /**
+   * Get the size of the split, so that the input splits can be sorted by size.
+   * @return the number of bytes in the split
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract long getLength() throws IOException, InterruptedException;
+
+  /**
+   * Get the list of nodes by name where the data for the split would be local.
+   * The locations do not need to be serialized by calls to 
+   * {@link Writable#write(java.io.DataOutput)}
+   * @return a new array of the node nodes.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract 
+    String[] getLocations() throws IOException, InterruptedException;
+}

+ 312 - 0
src/mapred/org/apache/hadoop/mapreduce/Job.java

@@ -0,0 +1,312 @@
+/**
+ * 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;
+
+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.TaskCompletionEvent;
+
+/**
+ * The job submitter's view of the Job. It allows the user to configure the
+ * job, submit it, control its execution, and query the state.
+ */
+public class Job extends JobContext {  
+  
+  public Job() {
+    this(new Configuration());
+  }
+
+  public Job(Configuration conf) {
+    super(conf, null);
+  }
+
+  public Job(Configuration conf, String jobName) {
+    this(conf);
+    setJobName(jobName);
+  }
+
+  /**
+   * Set the number of reduce tasks for the job.
+   * @param tasks the number of reduce tasks
+   */
+  public void setNumReduceTasks(int tasks) {
+    conf.setInt(NUM_REDUCES_ATTR, tasks);
+  }
+
+  /**
+   * Set the current working directory for the default file system.
+   * 
+   * @param dir the new current working directory.
+   */
+  public void setWorkingDirectory(Path dir) throws IOException {
+    dir = dir.makeQualified(FileSystem.get(conf));
+    conf.set(WORKING_DIR_ATTR, dir.toString());
+  }
+
+  /**
+   * Set the {@link InputFormat} for the job.
+   * @param cls the <code>InputFormat</code> to use
+   */
+  public void setInputFormatClass(Class<? extends InputFormat<?,?>> cls) {
+    conf.setClass(INPUT_FORMAT_CLASS_ATTR, cls, InputFormat.class);
+  }
+
+  /**
+   * Set the {@link OutputFormat} for the job.
+   * @param cls the <code>OutputFormat</code> to use
+   */
+  public void setOutputFormatClass(Class<? extends OutputFormat<?,?>> cls) {
+    conf.setClass(OUTPUT_FORMAT_CLASS_ATTR, cls, OutputFormat.class);
+  }
+
+  /**
+   * Set the {@link Mapper} for the job.
+   * @param cls the <code>Mapper</code> to use
+   */
+  public void setMapperClass(Class<? extends Mapper<?,?,?,?>> cls) {
+    conf.setClass(MAP_CLASS_ATTR, cls, Mapper.class);
+  }
+
+  /**
+   * Set the combiner class for the job.
+   * @param cls the combiner to use
+   */
+  public void setCombinerClass(Class<? extends Reducer<?,?,?,?>> cls) {
+    conf.setClass(COMBINE_CLASS_ATTR, cls, Reducer.class);
+  }
+
+  /**
+   * Set the {@link Reducer} for the job.
+   * @param cls the <code>Reducer</code> to use
+   */
+  public void setReducerClass(Class<? extends Reducer<?,?,?,?>> cls) {
+    conf.setClass(REDUCE_CLASS_ATTR, cls, Reducer.class);
+  }
+
+  /**
+   * Set the {@link Partitioner} for the job.
+   * @param cls the <code>Partitioner</code> to use
+   */
+  public void setPartitionerClass(Class<? extends Partitioner<?,?>> cls) {
+    conf.setClass(PARTITIONER_CLASS_ATTR, cls, Partitioner.class);
+  }
+
+  /**
+   * Set the key class for the map output data. This allows the user to
+   * specify the map output key class to be different than the final output
+   * value class.
+   * 
+   * @param theClass the map output key class.
+   */
+  public void setMapOutputKeyClass(Class<?> theClass) {
+    conf.setClass(MAP_OUTPUT_KEY_CLASS_ATTR, theClass, Object.class);
+  }
+
+  /**
+   * Set the value class for the map output data. This allows the user to
+   * specify the map output value class to be different than the final output
+   * value class.
+   * 
+   * @param theClass the map output value class.
+   */
+  public void setMapOutputValueClass(Class<?> theClass) {
+    conf.setClass(MAP_OUTPUT_VALUE_CLASS_ATTR, theClass, Object.class);
+  }
+
+  /**
+   * Set the key class for the job output data.
+   * 
+   * @param theClass the key class for the job output data.
+   */
+  public void setOutputKeyClass(Class<?> theClass) {
+    conf.setClass(OUTPUT_KEY_CLASS_ATTR, theClass, Object.class);
+  }
+
+  /**
+   * Set the value class for job outputs.
+   * 
+   * @param theClass the value class for job outputs.
+   */
+  public void setOutputValueClass(Class<?> theClass) {
+    conf.setClass(OUTPUT_VALUE_CLASS_ATTR, theClass, Object.class);
+  }
+
+  /**
+   * Define the comparator that controls how the keys are sorted before they
+   * are passed to the {@link Reducer}.
+   * @param cls the raw comparator
+   */
+  public void setSortComparatorClass(Class<? extends RawComparator<?>> cls) {
+    conf.setClass(SORT_COMPARATOR_ATTR, cls, RawComparator.class);
+  }
+
+  /**
+   * 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)}
+   * @param cls the raw comparator to use
+   */
+  public void setGroupingComparatorClass(Class<? extends RawComparator<?>> cls){
+    conf.setClass(GROUPING_COMPARATOR_ATTR, cls, RawComparator.class);
+  }
+
+  /**
+   * Set the user-specified job name.
+   * 
+   * @param name the job's new name.
+   */
+  public void setJobName(String name) {
+    conf.set(JOB_NAME_ATTR, name);
+  }
+
+  /**
+   * Get the URL where some job progress information will be displayed.
+   * 
+   * @return the URL where some job progress information will be displayed.
+   */
+  public String getTrackingURL() {
+    // TODO
+    return null;
+  }
+
+  /**
+   * Get the <i>progress</i> of the job's map-tasks, as a float between 0.0 
+   * and 1.0.  When all map tasks have completed, the function returns 1.0.
+   * 
+   * @return the progress of the job's map-tasks.
+   * @throws IOException
+   */
+  public float mapProgress() throws IOException {
+    // TODO
+    return 0.0f;
+  }
+
+  /**
+   * Get the <i>progress</i> of the job's reduce-tasks, as a float between 0.0 
+   * and 1.0.  When all reduce tasks have completed, the function returns 1.0.
+   * 
+   * @return the progress of the job's reduce-tasks.
+   * @throws IOException
+   */
+  public float reduceProgress() throws IOException {
+    // TODO
+    return 0.0f;
+  }
+
+  /**
+   * Check if the job is finished or not. 
+   * This is a non-blocking call.
+   * 
+   * @return <code>true</code> if the job is complete, else <code>false</code>.
+   * @throws IOException
+   */
+  public boolean isComplete() throws IOException {
+    // TODO
+    return false;
+  }
+
+  /**
+   * Check if the job completed successfully. 
+   * 
+   * @return <code>true</code> if the job succeeded, else <code>false</code>.
+   * @throws IOException
+   */
+  public boolean isSuccessful() throws IOException {
+    // TODO
+    return false;
+  }
+
+  /**
+   * Kill the running job.  Blocks until all job tasks have been
+   * killed as well.  If the job is no longer running, it simply returns.
+   * 
+   * @throws IOException
+   */
+  public void killJob() throws IOException {
+    // TODO
+  }
+    
+  /**
+   * Get events indicating completion (success/failure) of component tasks.
+   *  
+   * @param startFrom index to start fetching events from
+   * @return an array of {@link TaskCompletionEvent}s
+   * @throws IOException
+   */
+  public TaskCompletionEvent[] getTaskCompletionEvents(int startFrom
+                                                       ) throws IOException {
+    // TODO
+    return null;
+  }
+  
+  /**
+   * Kill indicated task attempt.
+   * 
+   * @param taskId the id of the task to be terminated.
+   * @throws IOException
+   */
+  public void killTask(TaskAttemptID taskId) throws IOException {
+    // TODO
+  }
+
+  /**
+   * Fail indicated task attempt.
+   * 
+   * @param taskId the id of the task to be terminated.
+   * @throws IOException
+   */
+  public void failTask(TaskAttemptID taskId) throws IOException {
+    // TODO
+  }
+
+  /**
+   * Gets the counters for this job.
+   * 
+   * @return the counters for this job.
+   * @throws IOException
+   */
+  public Iterable<CounterGroup> getCounters() throws IOException {
+    // TODO
+    return null;
+  }
+
+  /**
+   * Submit the job to the cluster and return immediately.
+   * @throws IOException
+   */
+  public void submit() throws IOException {
+    // TODO
+  }
+  
+  /**
+   * Submit the job to the cluster and wait for it to finish.
+   * @return true if the job succeeded
+   * @throws IOException thrown if the communication with the 
+   *         <code>JobTracker</code> is lost
+   */
+  public boolean waitForCompletion() throws IOException {
+    // TODO
+    return false;
+  }
+}

+ 273 - 0
src/mapred/org/apache/hadoop/mapreduce/JobContext.java

@@ -0,0 +1,273 @@
+/**
+ * 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;
+
+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;
+
+/**
+ * A read-only view of the job that is provided to the tasks while they
+ * are running.
+ */
+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 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;
+  private final JobID jobId;
+  
+  public JobContext(Configuration conf, JobID jobId) {
+    this.conf = conf;
+    this.jobId = jobId;
+  }
+
+  /**
+   * Return the configuration for the job.
+   * @return the shared configuration object
+   */
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  /**
+   * Get the unique ID for the job.
+   * @return the object with the job id
+   */
+  public JobID getJobID() {
+    return jobId;
+  }
+  
+  /**
+   * Get configured the number of reduce tasks for this job. Defaults to 
+   * <code>1</code>.
+   * @return the number of reduce tasks for this job.
+   */
+  public int getNumReduceTasks() {
+    return conf.getInt(NUM_REDUCES_ATTR, 1);
+  }
+  
+  /**
+   * Get the current working directory for the default file system.
+   * 
+   * @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;
+    }
+  }
+
+  /**
+   * Get the key class for the job output data.
+   * @return the key class for the job output data.
+   */
+  public Class<?> getOutputKeyClass() {
+    return conf.getClass(OUTPUT_KEY_CLASS_ATTR,
+                         LongWritable.class, Object.class);
+  }
+  
+  /**
+   * Get the value class for job outputs.
+   * @return the value class for job outputs.
+   */
+  public Class<?> getOutputValueClass() {
+    return conf.getClass(OUTPUT_VALUE_CLASS_ATTR, Text.class, Object.class);
+  }
+
+  /**
+   * Get the key class for the map output data. If it is not set, use the
+   * (final) output key class. This allows the map output key class to be
+   * different than the final output key class.
+   * @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;
+  }
+
+  /**
+   * Get the value class for the map output data. If it is not set, use the
+   * (final) output value class This allows the map output value class to be
+   * different than the final output value class.
+   *  
+   * @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;
+  }
+
+  /**
+   * Get the user-specified job name. This is only used to identify the 
+   * job to the user.
+   * 
+   * @return the job's name, defaulting to "".
+   */
+  public String getJobName() {
+    return conf.get(JOB_NAME_ATTR, "");
+  }
+
+  /**
+   * Get the {@link InputFormat} class for the job.
+   * 
+   * @return the {@link InputFormat} class for the job.
+   */
+  @SuppressWarnings("unchecked")
+  public Class<? extends InputFormat<?,?>> getInputFormatClass() 
+     throws ClassNotFoundException {
+    return (Class<? extends InputFormat<?,?>>) 
+      conf.getClass(INPUT_FORMAT_CLASS_ATTR, InputFormat.class);
+  }
+
+  /**
+   * Get the {@link Mapper} class for the job.
+   * 
+   * @return the {@link Mapper} class for the job.
+   */
+  @SuppressWarnings("unchecked")
+  public Class<? extends Mapper<?,?,?,?>> getMapperClass() 
+     throws ClassNotFoundException {
+    return (Class<? extends Mapper<?,?,?,?>>) 
+      conf.getClass(MAP_CLASS_ATTR, Mapper.class);
+  }
+
+  /**
+   * Get the combiner class for the job.
+   * 
+   * @return the combiner class for the job.
+   */
+  @SuppressWarnings("unchecked")
+  public Class<? extends Reducer<?,?,?,?>> getCombinerClass() 
+     throws ClassNotFoundException {
+    return (Class<? extends Reducer<?,?,?,?>>) 
+      conf.getClass(COMBINE_CLASS_ATTR, Reducer.class);
+  }
+
+  /**
+   * Get the {@link Reducer} class for the job.
+   * 
+   * @return the {@link Reducer} class for the job.
+   */
+  @SuppressWarnings("unchecked")
+  public Class<? extends Reducer<?,?,?,?>> getReducerClass() 
+     throws ClassNotFoundException {
+    return (Class<? extends Reducer<?,?,?,?>>) 
+      conf.getClass(REDUCE_CLASS_ATTR, Reducer.class);
+  }
+
+  /**
+   * Get the {@link OutputFormat} class for the job.
+   * 
+   * @return the {@link OutputFormat} class for the job.
+   */
+  @SuppressWarnings("unchecked")
+  public Class<? extends OutputFormat<?,?>> getOutputFormatClass() 
+     throws ClassNotFoundException {
+    return (Class<? extends OutputFormat<?,?>>) 
+      conf.getClass(OUTPUT_FORMAT_CLASS_ATTR, OutputFormat.class);
+  }
+
+  /**
+   * Get the {@link Partitioner} class for the job.
+   * 
+   * @return the {@link Partitioner} class for the job.
+   */
+  @SuppressWarnings("unchecked")
+  public Class<? extends Partitioner<?,?>> getPartitionerClass() 
+     throws ClassNotFoundException {
+    return (Class<? extends Partitioner<?,?>>) 
+      conf.getClass(PARTITIONER_CLASS_ATTR, Partitioner.class);
+  }
+
+  /**
+   * Get the {@link RawComparator} comparator used to compare keys.
+   * 
+   * @return the {@link RawComparator} comparator used to compare keys.
+   */
+  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(getMapOutputKeyClass());
+  }
+
+  /** 
+   * Get the user defined {@link WritableComparable} comparator for 
+   * grouping keys of inputs to the reduce.
+   * 
+   * @return comparator set by the user for grouping values.
+   * @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);
+  }
+
+}

+ 184 - 0
src/mapred/org/apache/hadoop/mapreduce/JobID.java

@@ -0,0 +1,184 @@
+/**
+ * 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;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+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 
+ * the job. JobID consists of two parts. First part 
+ * represents the jobtracker identifier, so that jobID to jobtracker map 
+ * is defined. For cluster setup this string is the jobtracker 
+ * start time, for local setting, it is "local".
+ * Second part of the JobID is the job number. <br> 
+ * An example JobID is : 
+ * <code>job_200707121733_0003</code> , which represents the third job 
+ * running at the jobtracker started at <code>200707121733</code>. 
+ * <p>
+ * Applications should never construct or parse JobID strings, but rather 
+ * use appropriate constructors or {@link #forName(String)} method. 
+ * 
+ * @see TaskID
+ * @see TaskAttemptID
+ * @see JobTracker#getNewJobId()
+ * @see JobTracker#getStartTime()
+ */
+public class JobID extends ID {
+  private static final String JOB = "job";
+  private String jtIdentifier;
+  private static char UNDERSCORE = '_';
+  
+  private static NumberFormat idFormat = NumberFormat.getInstance();
+  static {
+    idFormat.setGroupingUsed(false);
+    idFormat.setMinimumIntegerDigits(4);
+  }
+  
+  /**
+   * Constructs a JobID object 
+   * @param jtIdentifier jobTracker identifier
+   * @param id job number
+   */
+  public JobID(String jtIdentifier, int id) {
+    super(id);
+    this.jtIdentifier = jtIdentifier;
+  }
+  
+  private JobID() { }
+  
+  public String getJtIdentifier() {
+    return jtIdentifier;
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if(o == null)
+      return false;
+    if(o.getClass().equals(JobID.class)) {
+      JobID that = (JobID)o;
+      return this.id==that.id
+        && this.jtIdentifier.equals(that.jtIdentifier);
+    }
+    else return false;
+  }
+  
+  /**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() {
+    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();
+    return builder;
+  }
+  
+  @Override
+  public int hashCode() {
+    return toStringWOPrefix().toString().hashCode();
+  }
+  
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    this.jtIdentifier = Text.readString(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;
+  }
+  
+  /** Construct a JobId object from given string 
+   * @return constructed JobId object or null if the given String is null
+   * @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("_");
+      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");
+  }
+  
+  /** 
+   * 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;
+  }
+  
+}

+ 43 - 0
src/mapred/org/apache/hadoop/mapreduce/MapContext.java

@@ -0,0 +1,43 @@
+/**
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * The context that is given to the {@link Mapper}.
+ * @param <KEYIN> the key input type to the Mapper
+ * @param <VALUEIN> the value input type to the Mapper
+ * @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> 
+  extends TaskInputOutputContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
+
+  public MapContext(Configuration conf, TaskAttemptID taskid) {
+    super(conf, taskid);
+  }
+
+  /**
+   * Get the input split for this map.
+   */
+  public abstract InputSplit getInputSplit();
+
+}
+     

+ 147 - 0
src/mapred/org/apache/hadoop/mapreduce/Mapper.java

@@ -0,0 +1,147 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+
+/** 
+ * Maps input key/value pairs to a set of intermediate key/value pairs.  
+ * 
+ * <p>Maps are the individual tasks which transform input records into a 
+ * intermediate records. The transformed intermediate records need not be of 
+ * the same type as the input records. A given input pair may map to zero or 
+ * many output pairs.</p> 
+ * 
+ * <p>The Hadoop Map-Reduce framework spawns one map task for each 
+ * {@link InputSplit} generated by the {@link InputFormat} for the job.
+ * <code>Mapper</code> implementations can access the {@link Configuration} for 
+ * the job via the {@link JobContext#getConfiguration()}.
+ * 
+ * <p>The framework first calls 
+ * {@link #setup(org.apache.hadoop.mapreduce.Mapper.Context)}, followed by
+ * {@link #map(Object, Object, Context)} 
+ * for each key/value pair in the <code>InputSplit</code>. Finally 
+ * {@link #cleanup(Context)} is called.</p>
+ * 
+ * <p>All intermediate values associated with a given output key are 
+ * subsequently grouped by the framework, and passed to a {@link Reducer} to  
+ * determine the final output. Users can control the sorting and grouping by 
+ * specifying two key {@link RawComparator} classes.</p>
+ *
+ * <p>The <code>Mapper</code> outputs are partitioned per 
+ * <code>Reducer</code>. Users can control which keys (and hence records) go to 
+ * which <code>Reducer</code> by implementing a custom {@link Partitioner}.
+ * 
+ * <p>Users can optionally specify a <code>combiner</code>, via 
+ * {@link Job#setCombinerClass(Class)}, to perform local aggregation of the 
+ * intermediate outputs, which helps to cut down the amount of data transferred 
+ * from the <code>Mapper</code> to the <code>Reducer</code>.
+ * 
+ * <p>Applications can specify if and how the intermediate
+ * outputs are to be compressed and which {@link CompressionCodec}s are to be
+ * used via the <code>Configuration</code>.</p>
+ *  
+ * <p>If the job has zero
+ * reduces then the output of the <code>Mapper</code> is directly written
+ * to the {@link OutputFormat} without sorting by keys.</p>
+ * 
+ * <p>Example:</p>
+ * <p><blockquote><pre>
+ * public class TokenCounterMapper 
+ *     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 {
+ *     StringTokenizer itr = new StringTokenizer(value.toString());
+ *     while (itr.hasMoreTokens()) {
+ *       word.set(itr.nextToken());
+ *       context.collect(word, one);
+ *     }
+ *   }
+ * }
+ * </pre></blockquote></p>
+ *
+ * <p>Applications may override the {@link #run(Context)} method to exert 
+ * greater control on map processing e.g. multi-threaded <code>Mapper</code>s 
+ * etc.</p>
+ * 
+ * @see InputFormat
+ * @see JobContext
+ * @see Partitioner  
+ * @see Reducer
+ */
+public class Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
+
+  public abstract class Context 
+    extends MapContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
+    public Context(Configuration conf, TaskAttemptID taskid) {
+      super(conf, taskid);
+    }
+  }
+  
+  /**
+   * Called once at the beginning of the task.
+   */
+  protected void setup(Context context
+                       ) throws IOException, InterruptedException {
+    // NOTHING
+  }
+
+  /**
+   * Called once for each key/value pair in the input split. Most applications
+   * should override this, but the default is the identity function.
+   */
+  @SuppressWarnings("unchecked")
+  protected void map(KEYIN key, VALUEIN value, 
+                     Context context) throws IOException, InterruptedException {
+    context.collect((KEYOUT) key, (VALUEOUT) value);
+  }
+
+  /**
+   * Called once at the end of the task.
+   */
+  protected void cleanup(Context context
+                         ) throws IOException, InterruptedException {
+    // NOTHING
+  }
+  
+  /**
+   * Expert users can override this method for more complete control over the
+   * execution of the Mapper.
+   * @param context
+   * @throws IOException
+   */
+  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);
+    }
+    cleanup(context);
+  }
+}

+ 70 - 0
src/mapred/org/apache/hadoop/mapreduce/OutputFormat.java

@@ -0,0 +1,70 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/** 
+ * <code>OutputFormat</code> describes the output-specification for a 
+ * Map-Reduce job.
+ *
+ * <p>The Map-Reduce framework relies on the <code>OutputFormat</code> of the
+ * job to:<p>
+ * <ol>
+ *   <li>
+ *   Validate the output-specification of the job. For e.g. check that the 
+ *   output directory doesn't already exist. 
+ *   <li>
+ *   Provide the {@link RecordWriter} implementation to be used to write out
+ *   the output files of the job. Output files are stored in a 
+ *   {@link FileSystem}.
+ *   </li>
+ * </ol>
+ * 
+ * @see RecordWriter
+ */
+public interface OutputFormat<K, V> {
+
+  /** 
+   * Get the {@link RecordWriter} for the given task.
+   *
+   * @param context the information about the current task.
+   * @return a {@link RecordWriter} to write the output for the job.
+   * @throws IOException
+   */
+  RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
+  throws IOException, InterruptedException;
+
+  /** 
+   * Check for validity of the output-specification for the job.
+   *  
+   * <p>This is to validate the output specification for the job when it is
+   * a job is submitted.  Typically checks that it does not already exist,
+   * throwing an exception when it already exists, so that output is not
+   * overwritten.</p>
+   *
+   * @param context information about the job
+   * @throws IOException when output should not be attempted
+   */
+  void checkOutputSpecs(JobContext context
+                        ) throws IOException, InterruptedException;
+}
+

+ 47 - 0
src/mapred/org/apache/hadoop/mapreduce/Partitioner.java

@@ -0,0 +1,47 @@
+/**
+ * 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;
+
+/** 
+ * Partitions the key space.
+ * 
+ * <p><code>Partitioner</code> controls the partitioning of the keys of the 
+ * intermediate map-outputs. The key (or a subset of the key) is used to derive
+ * the partition, typically by a hash function. The total number of partitions
+ * is the same as the number of reduce tasks for the job. Hence this controls
+ * which of the <code>m</code> reduce tasks the intermediate key (and hence the 
+ * record) is sent for reduction.</p>
+ * 
+ * @see Reducer
+ */
+public interface Partitioner<KEY, VALUE> {
+  
+  /** 
+   * Get the partition number for a given key (hence record) given the total 
+   * number of partitions i.e. number of reduce-tasks for the job.
+   *   
+   * <p>Typically a hash function on a all or a subset of the key.</p>
+   *
+   * @param key the key to be partioned.
+   * @param value the entry value.
+   * @param numPartitions the total number of partitions.
+   * @return the partition number for the <code>key</code>.
+   */
+  int getPartition(KEY key, VALUE value, int numPartitions);
+}

+ 75 - 0
src/mapred/org/apache/hadoop/mapreduce/RecordReader.java

@@ -0,0 +1,75 @@
+/**
+ * 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * The record reader breaks the data into key/value pairs for input to the
+ * {@link Mapper}.
+ * @param <KEYIN>
+ * @param <VALUEIN>
+ */
+public abstract class RecordReader<KEYIN, VALUEIN> implements Closeable {
+
+  /**
+   * Called once at initialization.
+   * @param split the split that defines the range of records to read
+   * @param context the information about the task
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract void initialize(InputSplit split,
+                                  TaskAttemptContext context
+                                  ) 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
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract KEYIN nextKey(KEYIN key
+                                ) 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
+   * @return the object that was read
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract VALUEIN nextValue(VALUEIN value
+                                    ) throws IOException, InterruptedException;
+  
+  /**
+   * The current progress of the record reader through its data.
+   * @return a number between 0.0 and 1.0 that is the fraction of the data read
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract float getProgress() throws IOException, InterruptedException;
+  
+  /**
+   * Close the record reader.
+   */
+  public abstract void close() throws IOException;
+}

+ 52 - 0
src/mapred/org/apache/hadoop/mapreduce/RecordWriter.java

@@ -0,0 +1,52 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * <code>RecordWriter</code> writes the output &lt;key, value&gt; pairs 
+ * to an output file.
+ 
+ * <p><code>RecordWriter</code> implementations write the job outputs to the
+ * {@link FileSystem}.
+ * 
+ * @see OutputFormat
+ */
+public interface RecordWriter<K, V> {
+  /** 
+   * Writes a key/value pair.
+   *
+   * @param key the key to write.
+   * @param value the value to write.
+   * @throws IOException
+   */      
+  void write(K key, V value) throws IOException, InterruptedException;
+
+  /** 
+   * Close this <code>RecordWriter</code> to future operations.
+   * 
+   * @param context the context of the task
+   * @throws IOException
+   */ 
+  void close(TaskAttemptContext context
+             ) throws IOException, InterruptedException;
+}

+ 45 - 0
src/mapred/org/apache/hadoop/mapreduce/ReduceContext.java

@@ -0,0 +1,45 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * The context passed to the {@link Reducer}.
+ * @param <KEYIN> the class of the input keys
+ * @param <VALUEIN> the class of the input values
+ * @param <KEYOUT> the class of the output keys
+ * @param <VALUEOUT> the class of the output values
+ */
+public abstract class ReduceContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
+    extends TaskInputOutputContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
+
+  public ReduceContext(Configuration conf, TaskAttemptID taskid) {
+    super(conf, taskid);
+  }
+
+  /**
+   * Iterate through the values for the current key.
+   */
+  public abstract 
+  Iterable<VALUEIN> getValues() throws IOException, InterruptedException;
+
+}

+ 170 - 0
src/mapred/org/apache/hadoop/mapreduce/Reducer.java

@@ -0,0 +1,170 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Partitioner;
+
+/** 
+ * Reduces a set of intermediate values which share a key to a smaller set of
+ * values.  
+ * 
+ * <p><code>Reducer</code> implementations 
+ * can access the {@link Configuration} for the job via the 
+ * {@link JobContext#getConfiguration()} method.</p>
+
+ * <p><code>Reducer</code> has 3 primary phases:</p>
+ * <ol>
+ *   <li>
+ *   
+ *   <h4 id="Shuffle">Shuffle</h4>
+ *   
+ *   <p>The <code>Reducer</code> copies the sorted output from each 
+ *   {@link Mapper} using HTTP across the network.</p>
+ *   </li>
+ *   
+ *   <li>
+ *   <h4 id="Sort">Sort</h4>
+ *   
+ *   <p>The framework merge sorts <code>Reducer</code> inputs by 
+ *   <code>key</code>s 
+ *   (since different <code>Mapper</code>s may have output the same key).</p>
+ *   
+ *   <p>The shuffle and sort phases occur simultaneously i.e. while outputs are
+ *   being fetched they are merged.</p>
+ *      
+ *   <h5 id="SecondarySort">SecondarySort</h5>
+ *   
+ *   <p>To achieve a secondary sort on the values returned by the value 
+ *   iterator, the application should extend the key with the secondary
+ *   key and define a grouping comparator. The keys will be sorted using the
+ *   entire key, but will be grouped using the grouping comparator to decide
+ *   which keys and values are sent in the same call to reduce.The grouping 
+ *   comparator is specified via 
+ *   {@link Job#setGroupingComparatorClass(Class)}. The sort order is
+ *   controlled by 
+ *   {@link Job#setSortComparatorClass(Class)}.</p>
+ *   
+ *   
+ *   For example, say that you want to find duplicate web pages and tag them 
+ *   all with the url of the "best" known example. You would set up the job 
+ *   like:
+ *   <ul>
+ *     <li>Map Input Key: url</li>
+ *     <li>Map Input Value: document</li>
+ *     <li>Map Output Key: document checksum, url pagerank</li>
+ *     <li>Map Output Value: url</li>
+ *     <li>Partitioner: by checksum</li>
+ *     <li>OutputKeyComparator: by checksum and then decreasing pagerank</li>
+ *     <li>OutputValueGroupingComparator: by checksum</li>
+ *   </ul>
+ *   </li>
+ *   
+ *   <li>   
+ *   <h4 id="Reduce">Reduce</h4>
+ *   
+ *   <p>In this phase the 
+ *   {@link #reduce(Object, Iterable, Context)}
+ *   method is called for each <code>&lt;key, (collection of values)></code> in
+ *   the sorted inputs.</p>
+ *   <p>The output of the reduce task is typically written to a 
+ *   {@link RecordWriter} via 
+ *   {@link Context#collect(Object, Object)}.</p>
+ *   </li>
+ * </ol>
+ * 
+ * <p>The output of the <code>Reducer</code> is <b>not re-sorted</b>.</p>
+ * 
+ * <p>Example:</p>
+ * <p><blockquote><pre>
+ * public class IntSumReducer<Key> extends Reducer<Key,IntWritable,
+ *                                                 Key,IntWritable> {
+ *   private IntWritable result = new IntWritable();
+ * 
+ *   public void reduce(Key key, Iterable<IntWritable> values, 
+ *                      Context context) throws IOException {
+ *     int sum = 0;
+ *     for (IntWritable val : values) {
+ *       sum += val.get();
+ *     }
+ *     result.set(sum);
+ *     context.collect(key, result);
+ *   }
+ * }
+ * </pre></blockquote></p>
+ * 
+ * @see Mapper
+ * @see Partitioner
+ */
+public abstract class Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
+
+  protected abstract class Context 
+    extends ReduceContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
+    public Context(Configuration conf, TaskAttemptID taskid) {
+      super(conf, taskid);
+    }
+  }
+
+  /**
+   * Called once at the start of the task.
+   */
+  protected void setup(Context context
+                       ) throws IOException, InterruptedException {
+    // NOTHING
+  }
+
+  /**
+   * This method is called once for each key. Most applications will define
+   * their reduce class by overriding this method. The default implementation
+   * is an identity function.
+   */
+  @SuppressWarnings("unchecked")
+  protected void reduce(KEYIN key, Iterable<VALUEIN> values, Context context
+                        ) throws IOException, InterruptedException {
+    for(VALUEIN value: values) {
+      context.collect((KEYOUT) key, (VALUEOUT) value);
+    }
+  }
+
+  /**
+   * Called once at the end of the task.
+   */
+  protected void cleanup(Context context
+                         ) throws IOException, InterruptedException {
+    // NOTHING
+  }
+
+  /**
+   * Advanced application writers can use the 
+   * {@link #run(org.apache.hadoop.mapreduce.Reducer.Context)} method to
+   * control how the reduce task works.
+   */
+  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);
+    }
+    cleanup(context);
+  }
+}

+ 71 - 0
src/mapred/org/apache/hadoop/mapreduce/TaskAttemptContext.java

@@ -0,0 +1,71 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * The context for task attempts.
+ */
+public abstract class TaskAttemptContext extends JobContext 
+    implements Progressable {
+  private final TaskAttemptID taskId;
+  private String status = "";
+  
+  public TaskAttemptContext(Configuration conf, TaskAttemptID taskId) {
+    super(conf, taskId.getJobID());
+    this.taskId = taskId;
+  }
+
+  /**
+   * Get the unique name for this task attempt.
+   */
+  public TaskAttemptID getTaskAttemptId() {
+    return taskId;
+  }
+
+  /**
+   * Set the current status of the task to the given string.
+   */
+  public void setStatus(String msg) throws IOException {
+    status = msg;
+  }
+
+  /**
+   * Get the last set status message.
+   * @return the current status message
+   */
+  public String getStatus() {
+    return status;
+  }
+
+  /**
+   * Lookup a counter by an enum.
+   */
+  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);
+}

+ 212 - 0
src/mapred/org/apache/hadoop/mapreduce/TaskAttemptID.java

@@ -0,0 +1,212 @@
+/**
+ * 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;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * TaskAttemptID represents the immutable and unique identifier for 
+ * a task attempt. Each task attempt is one particular instance of a Map or
+ * Reduce Task identified by its TaskID. 
+ * 
+ * TaskAttemptID consists of 2 parts. First part is the 
+ * {@link TaskID}, that this TaskAttemptID belongs to.
+ * Second part is the task attempt number. <br> 
+ * 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>. 
+ * <p>
+ * Applications should never construct or parse TaskAttemptID strings
+ * , but rather use appropriate constructors or {@link #forName(String)} 
+ * method. 
+ * 
+ * @see JobID
+ * @see TaskID
+ */
+public class TaskAttemptID extends ID {
+  private static final String ATTEMPT = "attempt";
+  private TaskID taskId;
+  private static final char UNDERSCORE = '_';
+  
+  /**
+   * Constructs a TaskAttemptID object from given {@link TaskID}.  
+   * @param taskId TaskID that this task belongs to  
+   * @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;
+  }
+  
+  /**
+   * Constructs a TaskId object from given parts.
+   * @param jtIdentifier jobTracker identifier
+   * @param jobId job number 
+   * @param isMap whether the tip is a map 
+   * @param taskId taskId number
+   * @param id the task attempt number
+   */
+  public TaskAttemptID(String jtIdentifier, int jobId, boolean isMap
+      , int taskId, int id) {
+    this(new TaskID(jtIdentifier, jobId, isMap, taskId), id);
+  }
+  
+  private TaskAttemptID() { }
+  
+  /** 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(o == null)
+      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() { 
+    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();
+  }
+  
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    this.taskId = TaskID.read(in);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    taskId.write(out);
+  }
+  
+  public static TaskAttemptID read(DataInput in) throws IOException {
+    TaskAttemptID taskId = new TaskAttemptID();
+    taskId.readFields(in);
+    return taskId;
+  }
+  
+  /** 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 {
+    if(str == null)
+      return null;
+    try {
+      String[] parts = str.split("_");
+      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");
+  }
+  
+  /** 
+   * 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;
+  }
+  
+}

+ 224 - 0
src/mapred/org/apache/hadoop/mapreduce/TaskID.java

@@ -0,0 +1,224 @@
+/**
+ * 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;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.text.NumberFormat;
+
+/**
+ * TaskID represents the immutable and unique identifier for 
+ * a Map or Reduce Task. Each TaskID encompasses multiple attempts made to
+ * execute the Map or Reduce Task, each of which are uniquely indentified by
+ * their TaskAttemptID.
+ * 
+ * TaskID consists of 3 parts. First part is the {@link JobID}, that this 
+ * TaskInProgress belongs to. Second part of the TaskID is either 'm' or 'r' 
+ * representing whether the task is a map task or a reduce task. 
+ * And the third part is the task number. <br> 
+ * An example TaskID is : 
+ * <code>task_200707121733_0003_m_000005</code> , which represents the
+ * fifth map task in the third job running at the jobtracker 
+ * started at <code>200707121733</code>. 
+ * <p>
+ * Applications should never construct or parse TaskID strings
+ * , but rather use appropriate constructors or {@link #forName(String)} 
+ * method. 
+ * 
+ * @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();
+  static {
+    idFormat.setGroupingUsed(false);
+    idFormat.setMinimumIntegerDigits(6);
+  }
+  
+  private JobID jobId;
+  private boolean isMap;
+
+  /**
+   * Constructs a TaskID object from given {@link JobID}.  
+   * @param jobId JobID that this tip belongs to 
+   * @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;
+  }
+  
+  /**
+   * Constructs a TaskInProgressId object from given parts.
+   * @param jtIdentifier jobTracker identifier
+   * @param jobId job number 
+   * @param isMap whether the tip is a map 
+   * @param id the tip number
+   */
+  public TaskID(String jtIdentifier, int jobId, boolean isMap, int id) {
+    this(new JobID(jtIdentifier, jobId), isMap, id);
+  }
+  
+  private TaskID() { }
+  
+  /** 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(o == null)
+      return false;
+    if(o.getClass().equals(TaskID.class)) {
+      TaskID that = (TaskID)o;
+      return this.id==that.id
+        && this.isMap == that.isMap
+        && this.jobId.equals(that.jobId);
+    }
+    else return false;
+  }
+
+  /**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() { 
+    StringBuilder builder = new StringBuilder();
+    return builder.append(TASK).append(UNDERSCORE)
+      .append(toStringWOPrefix()).toString();
+  }
+
+  StringBuilder toStringWOPrefix() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(jobId.toStringWOPrefix())
+      .append(isMap ? "_m_" : "_r_");
+    return builder.append(idFormat.format(id));
+  }
+  
+  @Override
+  public int hashCode() {
+    return toStringWOPrefix().toString().hashCode();
+  }
+  
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    this.jobId = JobID.read(in);
+    this.isMap = in.readBoolean();
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    jobId.write(out);
+    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
+   */
+  public static TaskID forName(String str) 
+    throws IllegalArgumentException {
+    if(str == null)
+      return null;
+    try {
+      String[] parts = str.split("_");
+      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");
+  }
+  
+  /** 
+   * 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;
+  }
+  
+}

+ 64 - 0
src/mapred/org/apache/hadoop/mapreduce/TaskInputOutputContext.java

@@ -0,0 +1,64 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A context object that allows input and output from the task. It is only
+ * supplied to the {@link Mapper} or {@link Reducer}.
+ * @param <KEYIN> the input key type for the task
+ * @param <VALUEIN> the input value type for the task
+ * @param <KEYOUT> the output key type for the task
+ * @param <VALUEOUT> the output value type for the task
+ */
+public abstract class TaskInputOutputContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
+    extends TaskAttemptContext {
+
+  public TaskInputOutputContext(Configuration conf, TaskAttemptID taskid) {
+    super(conf, taskid);
+  }
+
+  /**
+   * 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
+   */
+  public abstract KEYIN nextKey(KEYIN key
+                                ) throws IOException, InterruptedException;
+  
+  /**
+   * Read the next value. Must be called after nextKey.
+   * @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;
+
+  /**
+   * Generate an output key/value pair.
+   */
+  public abstract void collect(KEYOUT key, VALUEOUT value
+                               ) throws IOException, InterruptedException;
+
+}

+ 69 - 0
src/mapred/org/apache/hadoop/mapreduce/example/WordCount.java

@@ -0,0 +1,69 @@
+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);
+  }
+}

+ 393 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java

@@ -0,0 +1,393 @@
+/**
+ * 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.lib.input;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.JobContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+
+/** 
+ * A base class for file-based {@link InputFormat}s.
+ * 
+ * <p><code>FileInputFormat</code> is the base class for all file-based 
+ * <code>InputFormat</code>s. This provides a generic implementation of
+ * {@link #getSplits(JobContext)}.
+ * Subclasses of <code>FileInputFormat</code> can also override the 
+ * {@link #isSplitable(JobContext, Path)} method to ensure input-files are
+ * not split-up and are processed as a whole by {@link Mapper}s.
+ */
+public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
+
+  public static final Log LOG = LogFactory.getLog(FileInputFormat.class);
+
+  private static final double SPLIT_SLOP = 1.1;   // 10% slop
+
+  private static final PathFilter hiddenFileFilter = new PathFilter(){
+      public boolean accept(Path p){
+        String name = p.getName(); 
+        return !name.startsWith("_") && !name.startsWith("."); 
+      }
+    }; 
+
+  /**
+   * Proxy PathFilter that accepts a path only if all filters given in the
+   * constructor do. Used by the listPaths() to apply the built-in
+   * hiddenFileFilter together with a user provided one (if any).
+   */
+  private static class MultiPathFilter implements PathFilter {
+    private List<PathFilter> filters;
+
+    public MultiPathFilter(List<PathFilter> filters) {
+      this.filters = filters;
+    }
+
+    public boolean accept(Path path) {
+      for (PathFilter filter : filters) {
+        if (!filter.accept(path)) {
+          return false;
+        }
+      }
+      return true;
+    }
+  }
+
+  /**
+   * Get the lower bound on split size imposed by the format.
+   * @return the number of bytes of the minimal split for this format
+   */
+  protected long getFormatMinSplitSize() {
+    return 1;
+  }
+
+  /**
+   * Is the given filename splitable? Usually, true, but if the file is
+   * stream compressed, it will not be.
+   * 
+   * <code>FileInputFormat</code> implementations can override this and return
+   * <code>false</code> to ensure that individual input files are never split-up
+   * so that {@link Mapper}s process entire files.
+   * 
+   * @param context the job context
+   * @param filename the file name to check
+   * @return is this file splitable?
+   */
+  protected boolean isSplitable(JobContext context, Path filename) {
+    return true;
+  }
+
+  /**
+   * Set a PathFilter to be applied to the input paths for the map-reduce job.
+   *
+   * @param filter the PathFilter class use for filtering the input paths.
+   */
+  public static void setInputPathFilter(Configuration conf,
+                                        Class<? extends PathFilter> filter) {
+    conf.setClass("mapred.input.pathFilter.class", filter, PathFilter.class);
+  }
+
+  public static void setMinInputSplitSize(Configuration conf,
+                                          long size) {
+    conf.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,
+                                          long size) {
+    conf.setLong("mapred.max.split.size", size);
+  }
+
+  public static long getMaxSplitSize(Configuration conf) {
+    return conf.getLong("mapred.max.split.size", Long.MAX_VALUE);
+  }
+
+  /**
+   * Get a PathFilter instance of the filter set for the input paths.
+   *
+   * @return the PathFilter instance set for the job, NULL if none has been set.
+   */
+  public static PathFilter getInputPathFilter(Configuration conf) {
+    Class<?> filterClass = conf.getClass("mapred.input.pathFilter.class", null,
+        PathFilter.class);
+    return (filterClass != null) ?
+        (PathFilter) ReflectionUtils.newInstance(filterClass, conf) : null;
+  }
+
+  /** List input directories.
+   * Subclasses may override to, e.g., select only files matching a regular
+   * expression. 
+   * 
+   * @param job the job to list input paths for
+   * @return array of FileStatus objects
+   * @throws IOException if zero items.
+   */
+  protected List<FileStatus> listStatus(Configuration job
+                                              ) throws IOException {
+    List<FileStatus> result = new ArrayList<FileStatus>();
+    Path[] dirs = getInputPaths(job);
+    if (dirs.length == 0) {
+      throw new IOException("No input paths specified in job");
+    }
+
+    List<IOException> errors = new ArrayList<IOException>();
+    
+    // creates a MultiPathFilter with the hiddenFileFilter and the
+    // user provided one (if any).
+    List<PathFilter> filters = new ArrayList<PathFilter>();
+    filters.add(hiddenFileFilter);
+    PathFilter jobFilter = getInputPathFilter(job);
+    if (jobFilter != null) {
+      filters.add(jobFilter);
+    }
+    PathFilter inputFilter = new MultiPathFilter(filters);
+    
+    for (int i=0; i < dirs.length; ++i) {
+      Path p = dirs[i];
+      FileSystem fs = p.getFileSystem(job); 
+      FileStatus[] matches = fs.globStatus(p, inputFilter);
+      if (matches == null) {
+        errors.add(new IOException("Input path does not exist: " + p));
+      } else if (matches.length == 0) {
+        errors.add(new IOException("Input Pattern " + p + " matches 0 files"));
+      } else {
+        for (FileStatus globStat: matches) {
+          if (globStat.isDir()) {
+            for(FileStatus stat: fs.listStatus(globStat.getPath(),
+                inputFilter)) {
+              result.add(stat);
+            }          
+          } else {
+            result.add(globStat);
+          }
+        }
+      }
+    }
+
+    if (!errors.isEmpty()) {
+      throw new InvalidInputException(errors);
+    }
+    LOG.info("Total input paths to process : " + result.size()); 
+    return result;
+  }
+  
+
+  /** Splits files returned by {@link #listStatus(Configuration)} when
+   * they're too big.*/ 
+  public List<InputSplit> getSplits(JobContext context
+                                    ) throws IOException {
+    Configuration job = context.getConfiguration();    
+    long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
+    long maxSize = getMaxSplitSize(job);
+
+    // generate splits
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+    for (FileStatus file: listStatus(job)) {
+      Path path = file.getPath();
+      FileSystem fs = path.getFileSystem(context.getConfiguration());
+      long length = file.getLen();
+      BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length);
+      if ((length != 0) && isSplitable(context, path)) { 
+        long blockSize = file.getBlockSize();
+        long splitSize = computeSplitSize(blockSize, minSize, maxSize);
+
+        long bytesRemaining = length;
+        while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) {
+          int blkIndex = getBlockIndex(blkLocations, length-bytesRemaining);
+          splits.add(new FileSplit(path, length-bytesRemaining, splitSize, 
+                                   blkLocations[blkIndex].getHosts()));
+          bytesRemaining -= splitSize;
+        }
+        
+        if (bytesRemaining != 0) {
+          splits.add(new FileSplit(path, length-bytesRemaining, bytesRemaining, 
+                     blkLocations[blkLocations.length-1].getHosts()));
+        }
+      } else if (length != 0) {
+        splits.add(new FileSplit(path, 0, length, blkLocations[0].getHosts()));
+      } else { 
+        //Create empty hosts array for zero length files
+        splits.add(new FileSplit(path, 0, length, new String[0]));
+      }
+    }
+    LOG.debug("Total # of splits: " + splits.size());
+    return splits;
+  }
+
+  protected long computeSplitSize(long blockSize, long minSize,
+                                  long maxSize) {
+    return Math.max(minSize, Math.min(maxSize, blockSize));
+  }
+
+  protected int getBlockIndex(BlockLocation[] blkLocations, 
+                              long offset) {
+    for (int i = 0 ; i < blkLocations.length; i++) {
+      // is the offset inside this block?
+      if ((blkLocations[i].getOffset() <= offset) &&
+          (offset < blkLocations[i].getOffset() + blkLocations[i].getLength())){
+        return i;
+      }
+    }
+    BlockLocation last = blkLocations[blkLocations.length -1];
+    long fileLength = last.getOffset() + last.getLength() -1;
+    throw new IllegalArgumentException("Offset " + offset + 
+                                       " is outside of file (0.." +
+                                       fileLength + ")");
+  }
+
+  /**
+   * Sets the given comma separated paths as the list of inputs 
+   * for the map-reduce job.
+   * 
+   * @param conf Configuration of 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, 
+                                   String commaSeparatedPaths
+                                   ) throws IOException {
+    setInputPaths(conf, StringUtils.stringToPath(
+                        getPathStrings(commaSeparatedPaths)));
+  }
+
+  /**
+   * Add the given comma separated paths to the list of inputs for
+   *  the map-reduce job.
+   * 
+   * @param conf The configuration of the job 
+   * @param commaSeparatedPaths Comma separated paths to be added to
+   *        the list of inputs for the map-reduce job.
+   */
+  public static void addInputPaths(Configuration conf, 
+                                   String commaSeparatedPaths
+                                   ) throws IOException {
+    for (String str : getPathStrings(commaSeparatedPaths)) {
+      addInputPath(conf, new Path(str));
+    }
+  }
+
+  /**
+   * Set the array of {@link Path}s as the list of inputs
+   * for the map-reduce job.
+   * 
+   * @param conf Configuration of the job. 
+   * @param inputPaths the {@link Path}s of the input directories/files 
+   * for the map-reduce job.
+   */ 
+  public static void setInputPaths(Configuration conf, 
+                                   Path... inputPaths) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    Path path = inputPaths[0].makeQualified(fs);
+    StringBuffer str = new StringBuffer(StringUtils.escapeString(path.toString()));
+    for(int i = 1; i < inputPaths.length;i++) {
+      str.append(StringUtils.COMMA_STR);
+      path = inputPaths[i].makeQualified(fs);
+      str.append(StringUtils.escapeString(path.toString()));
+    }
+    conf.set("mapred.input.dir", str.toString());
+  }
+
+  /**
+   * Add a {@link Path} to the list of inputs for the map-reduce job.
+   * 
+   * @param conf The configuration of the job 
+   * @param path {@link Path} to be added to the list of inputs for 
+   *            the map-reduce job.
+   */
+  public static void addInputPath(Configuration conf, 
+                                  Path path) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    path = path.makeQualified(fs);
+    String dirStr = StringUtils.escapeString(path.toString());
+    String dirs = conf.get("mapred.input.dir");
+    conf.set("mapred.input.dir", dirs == null ? dirStr : dirs + "," + dirStr);
+  }
+  
+  // This method escapes commas in the glob pattern of the given paths.
+  private static String[] getPathStrings(String commaSeparatedPaths) {
+    int length = commaSeparatedPaths.length();
+    int curlyOpen = 0;
+    int pathStart = 0;
+    boolean globPattern = false;
+    List<String> pathStrings = new ArrayList<String>();
+    
+    for (int i=0; i<length; i++) {
+      char ch = commaSeparatedPaths.charAt(i);
+      switch(ch) {
+        case '{' : {
+          curlyOpen++;
+          if (!globPattern) {
+            globPattern = true;
+          }
+          break;
+        }
+        case '}' : {
+          curlyOpen--;
+          if (curlyOpen == 0 && globPattern) {
+            globPattern = false;
+          }
+          break;
+        }
+        case ',' : {
+          if (!globPattern) {
+            pathStrings.add(commaSeparatedPaths.substring(pathStart, i));
+            pathStart = i + 1 ;
+          }
+          break;
+        }
+      }
+    }
+    pathStrings.add(commaSeparatedPaths.substring(pathStart, length));
+    
+    return pathStrings.toArray(new String[0]);
+  }
+  
+  /**
+   * Get the list of input {@link Path}s for the map-reduce job.
+   * 
+   * @param conf The configuration of the job 
+   * @return the list of input {@link Path}s for the map-reduce job.
+   */
+  public static Path[] getInputPaths(Configuration conf) {
+    String dirs = conf.get("mapred.input.dir", "");
+    String [] list = StringUtils.split(dirs);
+    Path[] result = new Path[list.length];
+    for (int i = 0; i < list.length; i++) {
+      result[i] = new Path(StringUtils.unEscapeString(list[i]));
+    }
+    return result;
+  }
+
+}

+ 91 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/FileSplit.java

@@ -0,0 +1,91 @@
+/**
+ * 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.lib.input;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+
+/** A section of an input file.  Returned by {@link
+ * InputFormat#getSplits(JobContext)} and passed to
+ * {@link InputFormat#createRecordReader(InputSplit,TaskAttemptContext)}. */
+public class FileSplit extends InputSplit {
+  private Path file;
+  private long start;
+  private long length;
+  private String[] hosts;
+
+  FileSplit() {}
+
+  /** Constructs a split with host information
+   *
+   * @param file the file name
+   * @param start the position of the first byte in the file to process
+   * @param length the number of bytes in the file to process
+   * @param hosts the list of hosts containing the block, possibly null
+   */
+  public FileSplit(Path file, long start, long length, String[] hosts) {
+    this.file = file;
+    this.start = start;
+    this.length = length;
+    this.hosts = hosts;
+  }
+ 
+  /** The file containing this split's data. */
+  public Path getPath() { return file; }
+  
+  /** The position of the first byte in the file to process. */
+  public long getStart() { return start; }
+  
+  /** The number of bytes in the file to process. */
+  public long getLength() { return length; }
+
+  public String toString() { return file + ":" + start + "+" + length; }
+
+  ////////////////////////////////////////////
+  // Writable methods
+  ////////////////////////////////////////////
+
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, file.toString());
+    out.writeLong(start);
+    out.writeLong(length);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    file = new Path(Text.readString(in));
+    start = in.readLong();
+    length = in.readLong();
+    hosts = null;
+  }
+
+  public String[] getLocations() throws IOException {
+    if (this.hosts == null) {
+      return new String[]{};
+    } else {
+      return this.hosts;
+    }
+  }
+}

+ 63 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/InvalidInputException.java

@@ -0,0 +1,63 @@
+/**
+ * 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.lib.input;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Iterator;
+
+/**
+ * This class wraps a list of problems with the input, so that the user
+ * can get a list of problems together instead of finding and fixing them one 
+ * by one.
+ */
+public class InvalidInputException extends IOException {
+  private List<IOException> problems;
+  
+  /**
+   * Create the exception with the given list.
+   * @param probs the list of problems to report. this list is not copied.
+   */
+  public InvalidInputException(List<IOException> probs) {
+    problems = probs;
+  }
+  
+  /**
+   * Get the complete list of the problems reported.
+   * @return the list of problems, which must not be modified
+   */
+  public List<IOException> getProblems() {
+    return problems;
+  }
+  
+  /**
+   * Get a summary message of the problems found.
+   * @return the concatenated messages from all of the problems.
+   */
+  public String getMessage() {
+    StringBuffer result = new StringBuffer();
+    Iterator<IOException> itr = problems.iterator();
+    while(itr.hasNext()) {
+      result.append(itr.next().getMessage());
+      if (itr.hasNext()) {
+        result.append("\n");
+      }
+    }
+    return result.toString();
+  }
+}

+ 277 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java

@@ -0,0 +1,277 @@
+/**
+ * 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.lib.input;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+
+/**
+ * Treats keys as offset in file and value as line. 
+ */
+public class LineRecordReader extends RecordReader<LongWritable, Text> {
+  private static final Log LOG = LogFactory.getLog(LineRecordReader.class);
+
+  private CompressionCodecFactory compressionCodecs = null;
+  private long start;
+  private long pos;
+  private long end;
+  private LineReader in;
+  int maxLineLength;
+
+  /**
+   * A class that provides a line reader from an input stream.
+   */
+  public static class LineReader {
+    private static final int DEFAULT_BUFFER_SIZE = 64 * 1024;
+    private int bufferSize = DEFAULT_BUFFER_SIZE;
+    private InputStream in;
+    private byte[] buffer;
+    // the number of bytes of real data in the buffer
+    private int bufferLength = 0;
+    // the current position in the buffer
+    private int bufferPosn = 0;
+
+    /**
+     * Create a line reader that reads from the given stream using the 
+     * given buffer-size.
+     * @param in
+     * @throws IOException
+     */
+    LineReader(InputStream in, int bufferSize) {
+      this.in = in;
+      this.bufferSize = bufferSize;
+      this.buffer = new byte[this.bufferSize];
+    }
+
+    /**
+     * Create a line reader that reads from the given stream using the
+     * <code>io.file.buffer.size</code> specified in the given
+     * <code>Configuration</code>.
+     * @param in input stream
+     * @param conf configuration
+     * @throws IOException
+     */
+    public LineReader(InputStream in, Configuration conf) throws IOException {
+      this(in, conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE));
+    }
+
+    /**
+     * Fill the buffer with more data.
+     * @return was there more data?
+     * @throws IOException
+     */
+    boolean backfill() throws IOException {
+      bufferPosn = 0;
+      bufferLength = in.read(buffer);
+      return bufferLength > 0;
+    }
+    
+    /**
+     * Close the underlying stream.
+     * @throws IOException
+     */
+    public void close() throws IOException {
+      in.close();
+    }
+    
+    /**
+     * Read from the InputStream into the given Text.
+     * @param str the object to store the given line
+     * @param maxLineLength the maximum number of bytes to store into str.
+     * @param maxBytesToConsume the maximum number of bytes to consume in this call.
+     * @return the number of bytes read including the newline
+     * @throws IOException if the underlying stream throws
+     */
+    public int readLine(Text str, int maxLineLength,
+                        int maxBytesToConsume) throws IOException {
+      str.clear();
+      boolean hadFinalNewline = false;
+      boolean hadFinalReturn = false;
+      boolean hitEndOfFile = false;
+      int startPosn = bufferPosn;
+      long bytesConsumed = 0;
+      outerLoop: while (true) {
+        if (bufferPosn >= bufferLength) {
+          if (!backfill()) {
+            hitEndOfFile = true;
+            break;
+          }
+        }
+        startPosn = bufferPosn;
+        for(; bufferPosn < bufferLength; ++bufferPosn) {
+          switch (buffer[bufferPosn]) {
+          case '\n':
+            hadFinalNewline = true;
+            bufferPosn += 1;
+            break outerLoop;
+          case '\r':
+            if (hadFinalReturn) {
+              // leave this \r in the stream, so we'll get it next time
+              break outerLoop;
+            }
+            hadFinalReturn = true;
+            break;
+          default:
+            if (hadFinalReturn) {
+              break outerLoop;
+            }
+          }        
+        }
+        bytesConsumed += bufferPosn - startPosn;
+        int length = bufferPosn - startPosn - (hadFinalReturn ? 1 : 0);
+        length = (int)Math.min(length, maxLineLength - str.getLength());
+        if (length >= 0) {
+          str.append(buffer, startPosn, length);
+        }
+        if (bytesConsumed >= maxBytesToConsume) {
+          return (int)Math.min(bytesConsumed, (long)Integer.MAX_VALUE);
+        }
+      }
+      int newlineLength = (hadFinalNewline ? 1 : 0) + (hadFinalReturn ? 1 : 0);
+      if (!hitEndOfFile) {
+        bytesConsumed += bufferPosn - startPosn;
+        int length = bufferPosn - startPosn - newlineLength;
+        length = (int)Math.min(length, maxLineLength - str.getLength());
+        if (length > 0) {
+          str.append(buffer, startPosn, length);
+        }
+      }
+      return (int)Math.min(bytesConsumed, (long)Integer.MAX_VALUE);
+    }
+
+    /**
+     * Read from the InputStream into the given Text.
+     * @param str the object to store the given line
+     * @param maxLineLength the maximum number of bytes to store into str.
+     * @return the number of bytes read including the newline
+     * @throws IOException if the underlying stream throws
+     */
+    public int readLine(Text str, int maxLineLength) throws IOException {
+      return readLine(str, maxLineLength, Integer.MAX_VALUE);
+  }
+
+    /**
+     * Read from the InputStream into the given Text.
+     * @param str the object to store the given line
+     * @return the number of bytes read including the newline
+     * @throws IOException if the underlying stream throws
+     */
+    public int readLine(Text str) throws IOException {
+      return readLine(str, Integer.MAX_VALUE, Integer.MAX_VALUE);
+    }
+
+  }
+
+  public void initialize(InputSplit genericSplit,
+                         TaskAttemptContext context) throws IOException {
+    FileSplit split = (FileSplit) genericSplit;
+    Configuration job = context.getConfiguration();
+    this.maxLineLength = job.getInt("mapred.linerecordreader.maxlength",
+                                    Integer.MAX_VALUE);
+    start = split.getStart();
+    end = start + split.getLength();
+    final Path file = split.getPath();
+    compressionCodecs = new CompressionCodecFactory(job);
+    final CompressionCodec codec = compressionCodecs.getCodec(file);
+
+    // open the file and seek to the start of the split
+    FileSystem fs = file.getFileSystem(job);
+    FSDataInputStream fileIn = fs.open(split.getPath());
+    boolean skipFirstLine = false;
+    if (codec != null) {
+      in = new LineReader(codec.createInputStream(fileIn), job);
+      end = Long.MAX_VALUE;
+    } else {
+      if (start != 0) {
+        skipFirstLine = true;
+        --start;
+        fileIn.seek(start);
+      }
+      in = new LineReader(fileIn, job);
+    }
+    if (skipFirstLine) {  // skip first line and re-establish "start".
+      start += in.readLine(new Text(), 0,
+                           (int)Math.min((long)Integer.MAX_VALUE, end - start));
+    }
+    this.pos = start;
+  }
+  
+  public LongWritable nextKey(LongWritable key) 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();
+    }
+    while (pos < end) {
+      int newSize = in.readLine(value, maxLineLength,
+                                Math.max((int)Math.min(Integer.MAX_VALUE, 
+                                                       end-pos),
+                                         maxLineLength));
+      if (newSize == 0) {
+        return null;
+      }
+      pos += newSize;
+      if (newSize < maxLineLength) {
+        break;
+      }
+
+      // line too long. try again
+      LOG.info("Skipped line of size " + newSize + " at pos " + 
+               (pos - newSize));
+    }
+    return value;
+  }
+
+  /**
+   * Get the progress within the split
+   */
+  public float getProgress() {
+    if (start == end) {
+      return 0.0f;
+    } else {
+      return Math.min(1.0f, (pos - start) / (float)(end - start));
+    }
+  }
+  
+  public synchronized void close() throws IOException {
+    if (in != null) {
+      in.close(); 
+    }
+  }
+}

+ 67 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileInputFormat.java

@@ -0,0 +1,67 @@
+/**
+ * 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.lib.input;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.MapFile;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/** An {@link InputFormat} for {@link SequenceFile}s. */
+public class SequenceFileInputFormat<K, V> extends FileInputFormat<K, V> {
+
+  @Override
+  public RecordReader<K, V> createRecordReader(InputSplit split,
+                                               TaskAttemptContext context
+                                               ) throws IOException {
+    return new SequenceFileRecordReader<K,V>();
+  }
+
+  protected long getFormatMinSplitSize() {
+    return SequenceFile.SYNC_INTERVAL;
+  }
+
+  protected List<FileStatus> listStatus(Configuration job
+                                              )throws IOException {
+
+    List<FileStatus> files = super.listStatus(job);
+    int len = files.size();
+    for(int i=0; i < len; ++i) {
+      FileStatus file = files.get(i);
+      if (file.isDir()) {     // it's a MapFile
+        Path p = file.getPath();
+        FileSystem fs = p.getFileSystem(job);
+        // use the data file
+        files.set(i, fs.getFileStatus(new Path(p, MapFile.DATA_FILE_NAME)));
+      }
+    }
+    return files;
+  }
+}
+

+ 95 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileRecordReader.java

@@ -0,0 +1,95 @@
+/**
+ * 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.lib.input;
+
+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.*;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/** An {@link RecordReader} for {@link SequenceFile}s. */
+public class SequenceFileRecordReader<K, V> extends RecordReader<K, V> {
+  private SequenceFile.Reader in;
+  private long start;
+  private long end;
+  private boolean more = true;
+  protected Configuration conf;
+  
+  @Override
+  public void initialize(InputSplit split, 
+                         TaskAttemptContext context
+                         ) throws IOException, InterruptedException {
+    FileSplit fileSplit = (FileSplit) split;
+    conf = context.getConfiguration();    
+    Path path = fileSplit.getPath();
+    FileSystem fs = path.getFileSystem(conf);
+    this.in = new SequenceFile.Reader(fs, path, conf);
+    this.end = fileSplit.getStart() + fileSplit.getLength();
+
+    if (fileSplit.getStart() > in.getPosition()) {
+      in.sync(fileSplit.getStart());                  // sync to start
+    }
+
+    this.start = in.getPosition();
+    more = start < end;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public K nextKey(K key) throws IOException, InterruptedException {
+    if (!more) {
+      return null;
+    }
+    long pos = in.getPosition();
+    K result = (K) in.next(key);
+    if (result == null || (pos >= end && in.syncSeen())) {
+      more = false;
+      result = null;
+    }
+    return result;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public V nextValue(V value) throws IOException, InterruptedException {
+    return (V) in.getCurrentValue(value);
+  }
+  
+  /**
+   * Return the progress within the input split
+   * @return 0.0 to 1.0 of the input byte range
+   */
+  public float getProgress() throws IOException {
+    if (end == start) {
+      return 0.0f;
+    } else {
+      return Math.min(1.0f, (in.getPosition() - start) / (float)(end - start));
+    }
+  }
+  
+  public synchronized void close() throws IOException { in.close(); }
+  
+}
+

+ 50 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/TextInputFormat.java

@@ -0,0 +1,50 @@
+/**
+ * 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.lib.input;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/** An {@link InputFormat} for plain text files.  Files are broken into lines.
+ * Either linefeed or carriage-return are used to signal end of line.  Keys are
+ * the position in the file, and values are the line of text.. */
+public class TextInputFormat extends FileInputFormat<LongWritable, Text> {
+
+  @Override
+  public RecordReader<LongWritable, Text> 
+    createRecordReader(InputSplit split,
+                       TaskAttemptContext context) {
+    return new LineRecordReader();
+  }
+
+  protected boolean isSplitable(JobContext context, Path file) {
+    CompressionCodec codec = 
+      new CompressionCodecFactory(context.getConfiguration()).getCodec(file);
+    return codec == null;
+  }
+
+}

+ 34 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/map/InverseMapper.java

@@ -0,0 +1,34 @@
+/**
+ * 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.lib.map;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+
+/** A {@link Mapper} that swaps keys and values. */
+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);
+  }
+  
+}

+ 196 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java

@@ -0,0 +1,196 @@
+/**
+ * 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.lib.map;
+
+import org.apache.hadoop.util.ReflectionUtils;
+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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+
+/**
+ * Multithreaded implementation for @link org.apache.hadoop.mapreduce.Mapper.
+ * <p>
+ * It can be used instead of the default implementation,
+ * @link org.apache.hadoop.mapred.MapRunner, when the Map operation is not CPU
+ * bound in order to improve throughput.
+ * <p>
+ * Mapper implementations using this MapRunnable must be thread-safe.
+ * <p>
+ * The Map-Reduce job has to be configured with the mapper to use via 
+ * {@link #setMapperClass(Configuration, Class)} and
+ * the number of thread the thread-pool can use with the
+ * {@link #getNumberOfThreads(Configuration) method. The default
+ * value is 10 threads.
+ * <p>
+ */
+public class MultithreadedMapper<K1, V1, K2, V2> 
+  extends Mapper<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;
+
+  public static int getNumberOfThreads(Configuration conf) {
+    return conf.getInt("mapred.map.multithreadedrunner.threads", 10);
+  }
+
+  public static void setNumberOfThreads(Configuration conf, int threads) {
+    conf.setInt("mapred.map.multithreadedrunner.threads", threads);
+  }
+
+  @SuppressWarnings("unchecked")
+  public static <K1,V1,K2,V2>
+  Class<Mapper<K1,V1,K2,V2>> getMapperClass(Configuration conf) {
+    return (Class<Mapper<K1,V1,K2,V2>>) 
+           conf.getClass("mapred.map.multithreadedrunner.class",
+                         Mapper.class);
+  }
+  
+  public static <K1,V1,K2,V2> 
+  void setMapperClass(Configuration conf, 
+                      Class<Mapper<K1,V1,K2,V2>> cls) {
+    if (MultithreadedMapper.class.isAssignableFrom(cls)) {
+      throw new IllegalArgumentException("Can't have recursive " + 
+                                         "MultithreadedMapper instances.");
+    }
+    conf.setClass("mapred.map.multithreadedrunner.class", cls, Mapper.class);
+  }
+
+  public void run(Context context) throws IOException, InterruptedException {
+    Configuration conf = context.getConfiguration();
+    outer = context;
+    int numberOfThreads = getNumberOfThreads(conf);
+    mapClass = getMapperClass(conf);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Configuring multithread runner to use " + numberOfThreads + 
+                " threads");
+    }
+    
+    runners = (MapRunner[]) new Object[numberOfThreads];
+    for(int i=0; i < numberOfThreads; ++i) {
+      runners[i] = new MapRunner();
+      runners[i].start();
+    }
+    for(int i=0; i < numberOfThreads; ++i) {
+      runners[i].join();
+      Throwable th = runners[i].throwable;
+      if (th != null) {
+        if (th instanceof IOException) {
+          throw (IOException) th;
+        } else if (th instanceof InterruptedException) {
+          throw (InterruptedException) th;
+        } else {
+          throw (RuntimeException) th;
+        }
+      }
+    }
+  }
+
+  private class SubMapContext extends Context {
+    private K1 key;
+    private V1 value;
+    
+    SubMapContext() {
+      super(outer.getConfiguration(), outer.getTaskAttemptId());
+    }
+
+    @Override
+    public InputSplit getInputSplit() {
+      synchronized (outer) {
+        return outer.getInputSplit();
+      }
+    }
+
+    @Override
+    public Counter getCounter(Enum<?> counterName) {
+      synchronized (outer) {
+        return outer.getCounter(counterName);
+      }
+    }
+
+    @Override
+    public Counter getCounter(String groupName, String counterName) {
+      synchronized (outer) {
+        return outer.getCounter(groupName, counterName);
+      }
+    }
+
+    @Override
+    public void progress() {
+      synchronized (outer) {
+        outer.progress();
+      }
+    }
+
+    @Override
+    public void collect(K2 key, V2 value) throws IOException,
+                                         InterruptedException {
+      synchronized (outer) {
+        outer.collect(key, value);
+      }
+    }
+
+    @Override
+    public K1 nextKey(K1 k) throws IOException, InterruptedException {
+      synchronized (outer) {
+        key = outer.nextKey(key);
+        if (key != null) {
+          value = outer.nextValue(value);
+        }
+        return key;
+      }
+    }
+    
+    public V1 nextValue(V1 v) throws IOException, InterruptedException {
+      return value;
+    }
+  }
+
+  private class MapRunner extends Thread {
+    private Mapper<K1,V1,K2,V2> mapper;
+    private Context context;
+    private Throwable throwable;
+
+    @SuppressWarnings("unchecked")
+    MapRunner() {
+      mapper = (Mapper<K1,V1,K2,V2>) 
+        ReflectionUtils.newInstance(mapClass, context.getConfiguration());
+      context = new SubMapContext();
+    }
+
+    public Throwable getThrowable() {
+      return throwable;
+    }
+
+    public void run() {
+      try {
+        mapper.run(context);
+      } catch (Throwable ie) {
+        throwable = ie;
+      }
+    }
+  }
+
+}

+ 41 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/map/TokenCounterMapper.java

@@ -0,0 +1,41 @@
+/**
+ * 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.lib.map;
+
+import java.io.IOException;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+public class TokenCounterMapper 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);
+    }
+  }
+}

+ 289 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java

@@ -0,0 +1,289 @@
+/**
+ * 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.lib.output;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.JobContext;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+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> implements OutputFormat<K, V> {
+
+  private static final String TEMP_DIR_NAME = "_temp";
+  /**
+   * Set whether the output of the job is compressed.
+   * @param conf the {@link Configuration} 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);
+  }
+  
+  /**
+   * Is the job output compressed?
+   * @param conf the {@link Configuration} to look in
+   * @return <code>true</code> if the job output should be compressed,
+   *         <code>false</code> otherwise
+   */
+  public static boolean getCompressOutput(Configuration conf) {
+    return conf.getBoolean("mapred.output.compress", false);
+  }
+  
+  /**
+   * Set the {@link CompressionCodec} to be used to compress job outputs.
+   * @param conf the {@link Configuration} to modify
+   * @param codecClass the {@link CompressionCodec} to be used to
+   *                   compress the job outputs
+   */
+  public static void 
+  setOutputCompressorClass(Configuration conf, 
+                           Class<? extends CompressionCodec> codecClass) {
+    setCompressOutput(conf, true);
+    conf.setClass("mapred.output.compression.codec", codecClass, 
+                  CompressionCodec.class);
+  }
+  
+  /**
+   * Get the {@link CompressionCodec} for compressing the job outputs.
+   * @param conf the {@link Configuration} to look in
+   * @param defaultValue the {@link CompressionCodec} to return if not set
+   * @return the {@link CompressionCodec} to be used to compress the 
+   *         job outputs
+   * @throws IllegalArgumentException if the class was specified, but not found
+   */
+  public static Class<? extends CompressionCodec> 
+  getOutputCompressorClass(Configuration conf, 
+		                       Class<? extends CompressionCodec> defaultValue) {
+    Class<? extends CompressionCodec> codecClass = defaultValue;
+    
+    String name = conf.get("mapred.output.compression.codec");
+    if (name != null) {
+      try {
+        codecClass = 
+        	conf.getClassByName(name).asSubclass(CompressionCodec.class);
+      } catch (ClassNotFoundException e) {
+        throw new IllegalArgumentException("Compression codec " + name + 
+                                           " was not found.", e);
+      }
+    }
+    return codecClass;
+  }
+  
+  public abstract 
+    RecordWriter<K, V> getRecordWriter(TaskAttemptContext context
+                                       ) throws IOException;
+
+  public void checkOutputSpecs(JobContext context) 
+    throws FileAlreadyExistsException, 
+           InvalidJobConfException, IOException {
+    // Ensure that the output directory is set and not already there
+    Configuration job = context.getConfiguration();
+    Path outDir = getOutputPath(job);
+    if (outDir == null && context.getNumReduceTasks() != 0) {
+      throw new InvalidJobConfException("Output directory not set in JobConf.");
+    }
+    if (outDir != null && outDir.getFileSystem(job).exists(outDir)) {
+      throw new FileAlreadyExistsException("Output directory " + outDir + 
+                                           " already exists");
+    }
+  }
+
+  /**
+   * Set the {@link Path} of the output directory for the map-reduce job.
+   *
+   * @param conf The configuration of the job.
+   * @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());
+  }
+
+  /**
+   * Get the {@link Path} to the output directory for the map-reduce job.
+   * 
+   * @return the {@link Path} to the output directory for the map-reduce job.
+   * @see FileOutputFormat#getWorkOutputPath(Configuration)
+   */
+  public static Path getOutputPath(Configuration conf) {
+    String name = conf.get("mapred.output.dir");
+    return name == null ? null: new Path(name);
+  }
+  
+  /**
+   *  Get the {@link Path} to the task's temporary output directory 
+   *  for the map-reduce job
+   *  
+   * <h4 id="SideEffectFiles">Tasks' Side-Effect Files</h4>
+   * 
+   * <p>Some applications need to create/write-to side-files, which differ from
+   * the actual job-outputs.
+   * 
+   * <p>In such cases there could be issues with 2 instances of the same TIP 
+   * (running simultaneously e.g. speculative tasks) trying to open/write-to the
+   * same file (path) on HDFS. Hence the application-writer will have to pick 
+   * unique names per task-attempt (e.g. using the attemptid, say 
+   * <tt>attempt_200709221812_0001_m_000000_0</tt>), not just per TIP.</p> 
+   * 
+   * <p>To get around this the Map-Reduce framework helps the application-writer 
+   * out by maintaining a special 
+   * <tt>${mapred.output.dir}/_temporary/_${taskid}</tt> 
+   * sub-directory for each task-attempt on HDFS where the output of the 
+   * task-attempt goes. On successful completion of the task-attempt the files 
+   * in the <tt>${mapred.output.dir}/_temporary/_${taskid}</tt> (only) 
+   * are <i>promoted</i> to <tt>${mapred.output.dir}</tt>. Of course, the 
+   * framework discards the sub-directory of unsuccessful task-attempts. This 
+   * is completely transparent to the application.</p>
+   * 
+   * <p>The application-writer can take advantage of this by creating any 
+   * side-files required in <tt>${mapred.work.output.dir}</tt> during execution 
+   * of his reduce-task i.e. via {@link #getWorkOutputPath(Configuration)}, and
+   * the framework will move them out similarly - thus she doesn't have to pick 
+   * unique paths per task-attempt.</p>
+   * 
+   * <p><i>Note</i>: the value of <tt>${mapred.work.output.dir}</tt> during 
+   * execution of a particular task-attempt is actually 
+   * <tt>${mapred.output.dir}/_temporary/_{$taskid}</tt>, and this value is 
+   * set by the map-reduce framework. So, just create any side-files in the 
+   * path  returned by {@link #getWorkOutputPath(Configuration)} from map/reduce 
+   * task to take advantage of this feature.</p>
+   * 
+   * <p>The entire discussion holds true for maps of jobs with 
+   * reducer=NONE (i.e. 0 reduces) since output of the map, in that case, 
+   * goes directly to HDFS.</p> 
+   * 
+   * @return the {@link Path} to the task's temporary output directory 
+   * for the map-reduce job.
+   */
+  public static Path getWorkOutputPath(Configuration conf) {
+    String name = conf.get("mapred.work.output.dir");
+    return name == null ? null: new Path(name);
+  }
+
+  /**
+   * Helper function to create the task's temporary output directory and 
+   * return the path to the task's output file.
+   * 
+   * @param context the task's context
+   * @return path to the task's temporary output file
+   * @throws IOException
+   */
+  protected static Path getTaskOutputPath(TaskAttemptContext context
+                                          ) throws IOException {
+    // ${mapred.job.dir}
+    Configuration conf = context.getConfiguration();
+    Path outputPath = getOutputPath(conf);
+    if (outputPath == null) {
+      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));
+  } 
+
+  /**
+   * Helper function to generate a name that is unique for the task.
+   *
+   * <p>The generated name can be used to create custom files from within the
+   * different tasks for the job, the names for different tasks will not collide
+   * with each other.</p>
+   *
+   * <p>The given name is postfixed with the task type, 'm' for maps, 'r' for
+   * reduces and the task partition number. For example, give a name 'test'
+   * running on the first map o the job the generated name will be
+   * 'test-m-00000'.</p>
+   *
+   * @param conf the configuration for the job.
+   * @param name the name to make unique.
+   * @return a unique name accross all tasks of the job.
+   */
+  public static String getUniqueName(Configuration conf, String name) {
+    int partition = conf.getInt("mapred.task.partition", -1);
+    if (partition == -1) {
+      throw new IllegalArgumentException(
+        "This method can only be called from within a Job");
+    }
+
+    String taskType = (conf.getBoolean("mapred.task.is.map", true)) ? "m" : "r";
+
+    NumberFormat numberFormat = NumberFormat.getInstance();
+    numberFormat.setMinimumIntegerDigits(5);
+    numberFormat.setGroupingUsed(false);
+
+    return name + "-" + taskType + "-" + numberFormat.format(partition);
+  }
+
+  /**
+   * Helper function to generate a {@link Path} for a file that is unique for
+   * the task within the job output directory.
+   *
+   * <p>The path can be used to create custom files from within the map and
+   * reduce tasks. The path name will be unique for each task. The path parent
+   * will be the job output directory.</p>ls
+   *
+   * <p>This method uses the {@link #getUniqueName} method to make the file name
+   * unique for the task.</p>
+   *
+   * @param conf the configuration for the job.
+   * @param name the name for the file.
+   * @return a unique path accross all tasks of the job.
+   */
+  public static Path getPathForCustomFile(Configuration conf, String name) {
+    return new Path(getWorkOutputPath(conf), getUniqueName(conf, name));
+  }
+
+  /** Construct output file names so that, when an output directory listing is
+   * sorted lexicographically, positions correspond to output partitions.*/
+  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
+  static {
+    NUMBER_FORMAT.setMinimumIntegerDigits(5);
+    NUMBER_FORMAT.setGroupingUsed(false);
+  }
+
+  protected static synchronized 
+  String getOutputName(TaskAttemptContext context) {
+    return "part-" + NUMBER_FORMAT.format(context.getTaskAttemptId().getId());
+  }
+}
+

+ 39 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/output/NullOutputFormat.java

@@ -0,0 +1,39 @@
+/**
+ * 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.lib.output;
+
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * Consume all outputs and put them in /dev/null. 
+ */
+public class NullOutputFormat<K, V> implements OutputFormat<K, V> {
+  
+  public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context) {
+    return new RecordWriter<K, V>(){
+        public void write(K key, V value) { }
+        public void close(TaskAttemptContext context) { }
+      };
+  }
+  
+  public void checkOutputSpecs(JobContext context) { }
+}

+ 104 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/output/SequenceFileOutputFormat.java

@@ -0,0 +1,104 @@
+/**
+ * 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.lib.output;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+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.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.conf.Configuration;
+
+/** An {@link OutputFormat} that writes {@link SequenceFile}s. */
+public class SequenceFileOutputFormat <K,V> extends FileOutputFormat<K, V> {
+
+  public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
+    throws IOException {
+    // get the path of the temporary output file 
+    Path file = FileOutputFormat.getTaskOutputPath(context);
+    Configuration conf = context.getConfiguration();
+    
+    FileSystem fs = file.getFileSystem(conf);
+    CompressionCodec codec = null;
+    CompressionType compressionType = CompressionType.NONE;
+    if (getCompressOutput(conf)) {
+      // find the kind of compression to do
+      compressionType = getOutputCompressionType(conf);
+
+      // find the right codec
+      Class<?> codecClass = getOutputCompressorClass(conf, DefaultCodec.class);
+      codec = (CompressionCodec) 
+        ReflectionUtils.newInstance(codecClass, conf);
+    }
+    final SequenceFile.Writer out = 
+      SequenceFile.createWriter(fs, conf, file,
+                                context.getOutputKeyClass(),
+                                context.getOutputValueClass(),
+                                compressionType,
+                                codec,
+                                context);
+
+    return new RecordWriter<K, V>() {
+
+        public void write(K key, V value)
+          throws IOException {
+
+          out.append(key, value);
+        }
+
+        public void close(TaskAttemptContext context) throws IOException { 
+          out.close();
+        }
+      };
+  }
+
+  /**
+   * Get the {@link CompressionType} for the output {@link SequenceFile}.
+   * @param conf the {@link Configuration}
+   * @return the {@link CompressionType} for the output {@link SequenceFile}, 
+   *         defaulting to {@link CompressionType#RECORD}
+   */
+  public static CompressionType getOutputCompressionType(Configuration conf) {
+    String val = conf.get("mapred.output.compression.type", 
+                          CompressionType.RECORD.toString());
+    return CompressionType.valueOf(val);
+  }
+  
+  /**
+   * Set the {@link CompressionType} for the output {@link SequenceFile}.
+   * @param conf the {@link Configuration} to modify
+   * @param style the {@link CompressionType} for the output
+   *              {@link SequenceFile} 
+   */
+  public static void setOutputCompressionType(Configuration conf, 
+		                                          CompressionType style) {
+    setCompressOutput(conf, true);
+    conf.set("mapred.output.compression.type", style.toString());
+  }
+
+}
+

+ 138 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/output/TextOutputFormat.java

@@ -0,0 +1,138 @@
+/**
+ * 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.lib.output;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataOutputStream;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.*;
+
+/** An {@link OutputFormat} that writes plain text files. */
+public class TextOutputFormat<K, V> extends FileOutputFormat<K, V> {
+
+  protected static class LineRecordWriter<K, V>
+    implements RecordWriter<K, V> {
+    private static final String utf8 = "UTF-8";
+    private static final byte[] newline;
+    static {
+      try {
+        newline = "\n".getBytes(utf8);
+      } catch (UnsupportedEncodingException uee) {
+        throw new IllegalArgumentException("can't find " + utf8 + " encoding");
+      }
+    }
+
+    protected DataOutputStream out;
+    private final byte[] keyValueSeparator;
+
+    public LineRecordWriter(DataOutputStream out, String keyValueSeparator) {
+      this.out = out;
+      try {
+        this.keyValueSeparator = keyValueSeparator.getBytes(utf8);
+      } catch (UnsupportedEncodingException uee) {
+        throw new IllegalArgumentException("can't find " + utf8 + " encoding");
+      }
+    }
+
+    public LineRecordWriter(DataOutputStream out) {
+      this(out, "\t");
+    }
+
+    /**
+     * Write the object to the byte stream, handling Text as a special
+     * case.
+     * @param o the object to print
+     * @throws IOException if the write throws, we pass it on
+     */
+    private void writeObject(Object o) throws IOException {
+      if (o instanceof Text) {
+        Text to = (Text) o;
+        out.write(to.getBytes(), 0, to.getLength());
+      } else {
+        out.write(o.toString().getBytes(utf8));
+      }
+    }
+
+    public synchronized void write(K key, V value)
+      throws IOException {
+
+      boolean nullKey = key == null || key instanceof NullWritable;
+      boolean nullValue = value == null || value instanceof NullWritable;
+      if (nullKey && nullValue) {
+        return;
+      }
+      if (!nullKey) {
+        writeObject(key);
+      }
+      if (!(nullKey || nullValue)) {
+        out.write(keyValueSeparator);
+      }
+      if (!nullValue) {
+        writeObject(value);
+      }
+      out.write(newline);
+    }
+
+    public synchronized 
+    void close(TaskAttemptContext context) throws IOException {
+      out.close();
+    }
+  }
+
+  public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
+    throws IOException {
+    Configuration job = context.getConfiguration();
+    boolean isCompressed = getCompressOutput(job);
+    String keyValueSeparator = job.get("mapred.textoutputformat.separator", 
+                                       "\t");
+    Path file = FileOutputFormat.getTaskOutputPath(context);
+    if (!isCompressed) {
+      FileSystem fs = file.getFileSystem(job);
+      FSDataOutputStream fileOut = fs.create(file, context);
+      return new LineRecordWriter<K, V>(fileOut, keyValueSeparator);
+    } else {
+      Class<? extends CompressionCodec> codecClass =
+        getOutputCompressorClass(job, GzipCodec.class);
+      // create the named codec
+      CompressionCodec codec = (CompressionCodec)
+        ReflectionUtils.newInstance(codecClass, job);
+      // build the filename including the extension
+      file = new Path(file + codec.getDefaultExtension());
+      FileSystem fs = file.getFileSystem(job);
+      FSDataOutputStream fileOut = fs.create(file, context);
+      return new LineRecordWriter<K, V>(new DataOutputStream
+                                        (codec.createOutputStream(fileOut)),
+                                        keyValueSeparator);
+    }
+  }
+}
+

+ 32 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/partition/HashPartitioner.java

@@ -0,0 +1,32 @@
+/**
+ * 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.lib.partition;
+
+import org.apache.hadoop.mapreduce.Partitioner;
+
+/** Partition keys by their {@link Object#hashCode()}. */
+public class HashPartitioner<K, V> implements Partitioner<K, V> {
+
+  /** Use {@link Object#hashCode()} to partition. */
+  public int getPartition(K key, V value,
+                          int numReduceTasks) {
+    return (key.hashCode() & Integer.MAX_VALUE) % numReduceTasks;
+  }
+
+}

+ 40 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/reduce/IntSumReducer.java

@@ -0,0 +1,40 @@
+/**
+ * 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.lib.reduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+
+public class IntSumReducer<Key> extends Reducer<Key,IntWritable,
+                                                Key,IntWritable> {
+  private IntWritable result = new IntWritable();
+
+  public void reduce(Key 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);
+  }
+
+}

+ 40 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/reduce/LongSumReducer.java

@@ -0,0 +1,40 @@
+/**
+ * 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.lib.reduce;
+
+import java.io.IOException;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+
+public class LongSumReducer<KEY> extends Reducer<KEY, LongWritable,
+                                                 KEY,LongWritable> {
+
+  private LongWritable result = new LongWritable();
+
+  public void reduce(KEY key, Iterable<LongWritable> values,
+                     Context context) throws IOException, InterruptedException {
+    long sum = 0;
+    for (LongWritable val : values) {
+      sum += val.get();
+    }
+    result.set(sum);
+    context.collect(key, result);
+  }
+
+}