Browse Source

Merge -r 1235547:1235548 from branch-1 to branch-1.0. Fixes: MAPREDUCE-3607

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1.0@1235551 13f79535-47bb-0310-9956-ffa450edef68
Thomas White 13 years ago
parent
commit
ee6e86af0e
80 changed files with 14130 additions and 5 deletions
  1. 2 0
      CHANGES.txt
  2. 26 0
      src/core/org/apache/hadoop/conf/Configuration.java
  3. 8 3
      src/core/org/apache/hadoop/io/WritableComparator.java
  4. 35 0
      src/mapred/org/apache/hadoop/mapreduce/Job.java
  5. 148 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/BigDecimalSplitter.java
  6. 71 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/BooleanSplitter.java
  7. 265 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/DBConfiguration.java
  8. 358 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/DBInputFormat.java
  9. 237 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/DBOutputFormat.java
  10. 275 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/DBRecordReader.java
  11. 47 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/DBSplitter.java
  12. 95 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/DBWritable.java
  13. 330 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBInputFormat.java
  14. 137 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBRecordReader.java
  15. 177 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/DateSplitter.java
  16. 104 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/FloatSplitter.java
  17. 131 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/IntegerSplitter.java
  18. 50 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/MySQLDBRecordReader.java
  19. 51 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/MySQLDataDrivenDBRecordReader.java
  20. 145 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/OracleDBRecordReader.java
  21. 93 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBInputFormat.java
  22. 49 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBRecordReader.java
  23. 42 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/OracleDateSplitter.java
  24. 220 0
      src/mapred/org/apache/hadoop/mapreduce/lib/db/TextSplitter.java
  25. 227 0
      src/mapred/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionHelper.java
  26. 110 0
      src/mapred/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionMapper.java
  27. 106 0
      src/mapred/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionReducer.java
  28. 693 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
  29. 169 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReader.java
  30. 200 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileSplit.java
  31. 131 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/DelegatingInputFormat.java
  32. 58 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/DelegatingMapper.java
  33. 92 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/DelegatingRecordReader.java
  34. 1 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
  35. 128 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/KeyValueLineRecordReader.java
  36. 59 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/KeyValueTextInputFormat.java
  37. 150 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/MultipleInputs.java
  38. 155 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java
  39. 154 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileAsBinaryInputFormat.java
  40. 49 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileAsTextInputFormat.java
  41. 95 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileAsTextRecordReader.java
  42. 325 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileInputFilter.java
  43. 159 0
      src/mapred/org/apache/hadoop/mapreduce/lib/input/TaggedInputSplit.java
  44. 345 0
      src/mapred/org/apache/hadoop/mapreduce/lib/jobcontrol/ControlledJob.java
  45. 284 0
      src/mapred/org/apache/hadoop/mapreduce/lib/jobcontrol/JobControl.java
  46. 19 2
      src/mapred/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
  47. 111 0
      src/mapred/org/apache/hadoop/mapreduce/lib/output/FilterOutputFormat.java
  48. 124 0
      src/mapred/org/apache/hadoop/mapreduce/lib/output/LazyOutputFormat.java
  49. 468 0
      src/mapred/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java
  50. 198 0
      src/mapred/org/apache/hadoop/mapreduce/lib/output/SequenceFileAsBinaryOutputFormat.java
  51. 142 0
      src/mapred/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java
  52. 418 0
      src/mapred/org/apache/hadoop/mapreduce/lib/partition/InputSampler.java
  53. 374 0
      src/mapred/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.java
  54. 156 0
      src/mapred/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedPartitioner.java
  55. 296 0
      src/mapred/org/apache/hadoop/mapreduce/lib/partition/KeyFieldHelper.java
  56. 411 0
      src/mapred/org/apache/hadoop/mapreduce/lib/partition/TotalOrderPartitioner.java
  57. 189 0
      src/test/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java
  58. 39 0
      src/test/org/apache/hadoop/mapreduce/lib/db/TestDBJob.java
  59. 69 0
      src/test/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java
  60. 222 0
      src/test/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
  61. 106 0
      src/test/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java
  62. 128 0
      src/test/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java
  63. 124 0
      src/test/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java
  64. 1174 0
      src/test/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
  65. 115 0
      src/test/org/apache/hadoop/mapreduce/lib/input/TestDelegatingInputFormat.java
  66. 227 0
      src/test/org/apache/hadoop/mapreduce/lib/input/TestKeyValueTextInputFormat.java
  67. 115 0
      src/test/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java
  68. 118 0
      src/test/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java
  69. 194 0
      src/test/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java
  70. 72 0
      src/test/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
  71. 114 0
      src/test/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java
  72. 46 0
      src/test/org/apache/hadoop/mapreduce/lib/jobcontrol/TestControlledJob.java
  73. 142 0
      src/test/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java
  74. 212 0
      src/test/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java
  75. 232 0
      src/test/org/apache/hadoop/mapreduce/lib/output/TestMultipleOutputs.java
  76. 103 0
      src/test/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java
  77. 149 0
      src/test/org/apache/hadoop/mapreduce/lib/partition/TestInputSampler.java
  78. 425 0
      src/test/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java
  79. 125 0
      src/test/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java
  80. 187 0
      src/test/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java

+ 2 - 0
CHANGES.txt

@@ -6,6 +6,8 @@ Release 1.0.1 - unreleased
 
   IMPROVEMENTS
 
+    MAPREDUCE-3607. Port missing new API mapreduce lib classes to 1.x. (tomwhite)
+
   BUG FIXES
 
     HADOOP-7964. Deadlock in NetUtils and SecurityUtil class initialization.

+ 26 - 0
src/core/org/apache/hadoop/conf/Configuration.java

@@ -61,6 +61,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonGenerator;
@@ -899,6 +900,31 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
 
+  /**
+   * Get the value of the <code>name</code> property as a <code>List</code>
+   * of objects implementing the interface specified by <code>xface</code>.
+   * 
+   * An exception is thrown if any of the classes does not exist, or if it does
+   * not implement the named interface.
+   * 
+   * @param name the property name.
+   * @param xface the interface implemented by the classes named by
+   *        <code>name</code>.
+   * @return a <code>List</code> of objects implementing <code>xface</code>.
+   */
+  @SuppressWarnings("unchecked")
+  public <U> List<U> getInstances(String name, Class<U> xface) {
+    List<U> ret = new ArrayList<U>();
+    Class<?>[] classes = getClasses(name);
+    for (Class<?> cl: classes) {
+      if (!xface.isAssignableFrom(cl)) {
+        throw new RuntimeException(cl + " does not implement " + xface);
+      }
+      ret.add((U)ReflectionUtils.newInstance(cl, this));
+    }
+    return ret;
+  }
+
   /** 
    * Set the value of the <code>name</code> property to the name of a 
    * <code>theClass</code> implementing the given interface <code>xface</code>.

+ 8 - 3
src/core/org/apache/hadoop/io/WritableComparator.java

@@ -133,14 +133,19 @@ public class WritableComparator implements RawComparator {
     }
     return l1 - l2;
   }
-
+  
   /** Compute hash for binary data. */
-  public static int hashBytes(byte[] bytes, int length) {
+  public static int hashBytes(byte[] bytes, int offset, int length) {
     int hash = 1;
-    for (int i = 0; i < length; i++)
+    for (int i = offset; i < offset + length; i++)
       hash = (31 * hash) + (int)bytes[i];
     return hash;
   }
+  
+  /** Compute hash for binary data. */
+  public static int hashBytes(byte[] bytes, int length) {
+    return hashBytes(bytes, 0, length);
+  }
 
   /** Parse an unsigned short from a byte array. */
   public static int readUnsignedShort(byte[] bytes, int start) {

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

@@ -262,6 +262,41 @@ public class Job extends JobContext {
     ensureState(JobState.DEFINE);
     conf.setJobName(name);
   }
+  
+  /**
+   * Turn speculative execution on or off for this job. 
+   * 
+   * @param speculativeExecution <code>true</code> if speculative execution 
+   *                             should be turned on, else <code>false</code>.
+   */
+  public void setSpeculativeExecution(boolean speculativeExecution) {
+    ensureState(JobState.DEFINE);
+    conf.setSpeculativeExecution(speculativeExecution);
+  }
+
+  /**
+   * Turn speculative execution on or off for this job for map tasks. 
+   * 
+   * @param speculativeExecution <code>true</code> if speculative execution 
+   *                             should be turned on for map tasks,
+   *                             else <code>false</code>.
+   */
+  public void setMapSpeculativeExecution(boolean speculativeExecution) {
+    ensureState(JobState.DEFINE);
+    conf.setMapSpeculativeExecution(speculativeExecution);
+  }
+
+  /**
+   * Turn speculative execution on or off for this job for reduce tasks. 
+   * 
+   * @param speculativeExecution <code>true</code> if speculative execution 
+   *                             should be turned on for reduce tasks,
+   *                             else <code>false</code>.
+   */
+  public void setReduceSpeculativeExecution(boolean speculativeExecution) {
+    ensureState(JobState.DEFINE);
+    conf.setReduceSpeculativeExecution(speculativeExecution);
+  }
 
   /**
    * Get the URL where some job progress information will be displayed.

+ 148 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/BigDecimalSplitter.java

@@ -0,0 +1,148 @@
+/**
+ * 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.db;
+
+import java.math.BigDecimal;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+/**
+ * Implement DBSplitter over BigDecimal values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BigDecimalSplitter implements DBSplitter {
+  private static final Log LOG = LogFactory.getLog(BigDecimalSplitter.class);
+
+  public List<InputSplit> split(Configuration conf, ResultSet results, String colName)
+      throws SQLException {
+
+    BigDecimal minVal = results.getBigDecimal(1);
+    BigDecimal maxVal = results.getBigDecimal(2);
+
+    String lowClausePrefix = colName + " >= ";
+    String highClausePrefix = colName + " < ";
+
+    BigDecimal numSplits = new BigDecimal(conf.getInt("mapred.map.tasks", 1));
+
+    if (minVal == null && maxVal == null) {
+      // Range is null to null. Return a null split accordingly.
+      List<InputSplit> splits = new ArrayList<InputSplit>();
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+      return splits;
+    }
+
+    if (minVal == null || maxVal == null) {
+      // Don't know what is a reasonable min/max value for interpolation. Fail.
+      LOG.error("Cannot find a range for NUMERIC or DECIMAL fields with one end NULL.");
+      return null;
+    }
+
+    // Get all the split points together.
+    List<BigDecimal> splitPoints = split(numSplits, minVal, maxVal);
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+
+    // Turn the split points into a set of intervals.
+    BigDecimal start = splitPoints.get(0);
+    for (int i = 1; i < splitPoints.size(); i++) {
+      BigDecimal end = splitPoints.get(i);
+
+      if (i == splitPoints.size() - 1) {
+        // This is the last one; use a closed interval.
+        splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+            lowClausePrefix + start.toString(),
+            colName + " <= " + end.toString()));
+      } else {
+        // Normal open-interval case.
+        splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+            lowClausePrefix + start.toString(),
+            highClausePrefix + end.toString()));
+      }
+
+      start = end;
+    }
+
+    return splits;
+  }
+
+  private static final BigDecimal MIN_INCREMENT = new BigDecimal(10000 * Double.MIN_VALUE);
+
+  /**
+   * Divide numerator by denominator. If impossible in exact mode, use rounding.
+   */
+  protected BigDecimal tryDivide(BigDecimal numerator, BigDecimal denominator) {
+    try {
+      return numerator.divide(denominator);
+    } catch (ArithmeticException ae) {
+      return numerator.divide(denominator, BigDecimal.ROUND_HALF_UP);
+    }
+  }
+
+  /**
+   * Returns a list of BigDecimals one element longer than the list of input splits.
+   * This represents the boundaries between input splits.
+   * All splits are open on the top end, except the last one.
+   *
+   * So the list [0, 5, 8, 12, 18] would represent splits capturing the intervals:
+   *
+   * [0, 5)
+   * [5, 8)
+   * [8, 12)
+   * [12, 18] note the closed interval for the last split.
+   */
+  List<BigDecimal> split(BigDecimal numSplits, BigDecimal minVal, BigDecimal maxVal)
+      throws SQLException {
+
+    List<BigDecimal> splits = new ArrayList<BigDecimal>();
+
+    // Use numSplits as a hint. May need an extra task if the size doesn't
+    // divide cleanly.
+
+    BigDecimal splitSize = tryDivide(maxVal.subtract(minVal), (numSplits));
+    if (splitSize.compareTo(MIN_INCREMENT) < 0) {
+      splitSize = MIN_INCREMENT;
+      LOG.warn("Set BigDecimal splitSize to MIN_INCREMENT");
+    }
+
+    BigDecimal curVal = minVal;
+
+    while (curVal.compareTo(maxVal) <= 0) {
+      splits.add(curVal);
+      curVal = curVal.add(splitSize);
+    }
+
+    if (splits.get(splits.size() - 1).compareTo(maxVal) != 0 || splits.size() == 1) {
+      // We didn't end on the maxVal. Add that to the end of the list.
+      splits.add(maxVal);
+    }
+
+    return splits;
+  }
+}

+ 71 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/BooleanSplitter.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.lib.db;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+/**
+ * Implement DBSplitter over boolean values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BooleanSplitter implements DBSplitter {
+  public List<InputSplit> split(Configuration conf, ResultSet results, String colName)
+      throws SQLException {
+
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+
+    if (results.getString(1) == null && results.getString(2) == null) {
+      // Range is null to null. Return a null split accordingly.
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+      return splits;
+    }
+
+    boolean minVal = results.getBoolean(1);
+    boolean maxVal = results.getBoolean(2);
+
+    // Use one or two splits.
+    if (!minVal) {
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " = FALSE", colName + " = FALSE"));
+    }
+
+    if (maxVal) {
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " = TRUE", colName + " = TRUE"));
+    }
+
+    if (results.getString(1) == null || results.getString(2) == null) {
+      // Include a null value.
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+    }
+
+    return splits;
+  }
+}

+ 265 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/DBConfiguration.java

@@ -0,0 +1,265 @@
+/**
+ * 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.db;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.db.DBInputFormat.NullDBWritable;
+
+/**
+ * A container for configuration property names for jobs with DB input/output.
+ *  
+ * The job can be configured using the static methods in this class, 
+ * {@link DBInputFormat}, and {@link DBOutputFormat}. 
+ * Alternatively, the properties can be set in the configuration with proper
+ * values. 
+ *   
+ * @see DBConfiguration#configureDB(Configuration, String, String, String, String)
+ * @see DBInputFormat#setInput(Job, Class, String, String)
+ * @see DBInputFormat#setInput(Job, Class, String, String, String, String...)
+ * @see DBOutputFormat#setOutput(Job, String, String...)
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DBConfiguration {
+
+  /** The JDBC Driver class name */
+  public static final String DRIVER_CLASS_PROPERTY = 
+    "mapreduce.jdbc.driver.class";
+  
+  /** JDBC Database access URL */
+  public static final String URL_PROPERTY = "mapreduce.jdbc.url";
+
+  /** User name to access the database */
+  public static final String USERNAME_PROPERTY = "mapreduce.jdbc.username";
+  
+  /** Password to access the database */
+  public static final String PASSWORD_PROPERTY = "mapreduce.jdbc.password";
+
+  /** Input table name */
+  public static final String INPUT_TABLE_NAME_PROPERTY = 
+    "mapreduce.jdbc.input.table.name";
+
+  /** Field names in the Input table */
+  public static final String INPUT_FIELD_NAMES_PROPERTY = 
+    "mapreduce.jdbc.input.field.names";
+
+  /** WHERE clause in the input SELECT statement */
+  public static final String INPUT_CONDITIONS_PROPERTY = 
+    "mapreduce.jdbc.input.conditions";
+  
+  /** ORDER BY clause in the input SELECT statement */
+  public static final String INPUT_ORDER_BY_PROPERTY = 
+    "mapreduce.jdbc.input.orderby";
+  
+  /** Whole input query, exluding LIMIT...OFFSET */
+  public static final String INPUT_QUERY = "mapreduce.jdbc.input.query";
+  
+  /** Input query to get the count of records */
+  public static final String INPUT_COUNT_QUERY = 
+    "mapreduce.jdbc.input.count.query";
+  
+  /** Input query to get the max and min values of the jdbc.input.query */
+  public static final String INPUT_BOUNDING_QUERY =
+      "mapred.jdbc.input.bounding.query";
+  
+  /** Class name implementing DBWritable which will hold input tuples */
+  public static final String INPUT_CLASS_PROPERTY = 
+    "mapreduce.jdbc.input.class";
+
+  /** Output table name */
+  public static final String OUTPUT_TABLE_NAME_PROPERTY = 
+    "mapreduce.jdbc.output.table.name";
+
+  /** Field names in the Output table */
+  public static final String OUTPUT_FIELD_NAMES_PROPERTY = 
+    "mapreduce.jdbc.output.field.names";  
+
+  /** Number of fields in the Output table */
+  public static final String OUTPUT_FIELD_COUNT_PROPERTY = 
+    "mapreduce.jdbc.output.field.count";  
+  
+  /**
+   * Sets the DB access related fields in the {@link Configuration}.  
+   * @param conf the configuration
+   * @param driverClass JDBC Driver class name
+   * @param dbUrl JDBC DB access URL. 
+   * @param userName DB access username 
+   * @param passwd DB access passwd
+   */
+  public static void configureDB(Configuration conf, String driverClass, 
+      String dbUrl, String userName, String passwd) {
+
+    conf.set(DRIVER_CLASS_PROPERTY, driverClass);
+    conf.set(URL_PROPERTY, dbUrl);
+    if (userName != null) {
+      conf.set(USERNAME_PROPERTY, userName);
+    }
+    if (passwd != null) {
+      conf.set(PASSWORD_PROPERTY, passwd);
+    }
+  }
+
+  /**
+   * Sets the DB access related fields in the JobConf.  
+   * @param job the job
+   * @param driverClass JDBC Driver class name
+   * @param dbUrl JDBC DB access URL. 
+   */
+  public static void configureDB(Configuration job, String driverClass,
+      String dbUrl) {
+    configureDB(job, driverClass, dbUrl, null, null);
+  }
+
+  private Configuration conf;
+
+  public DBConfiguration(Configuration job) {
+    this.conf = job;
+  }
+
+  /** Returns a connection object o the DB 
+   * @throws ClassNotFoundException 
+   * @throws SQLException */
+  public Connection getConnection() 
+      throws ClassNotFoundException, SQLException {
+
+    Class.forName(conf.get(DBConfiguration.DRIVER_CLASS_PROPERTY));
+
+    if(conf.get(DBConfiguration.USERNAME_PROPERTY) == null) {
+      return DriverManager.getConnection(
+               conf.get(DBConfiguration.URL_PROPERTY));
+    } else {
+      return DriverManager.getConnection(
+          conf.get(DBConfiguration.URL_PROPERTY), 
+          conf.get(DBConfiguration.USERNAME_PROPERTY), 
+          conf.get(DBConfiguration.PASSWORD_PROPERTY));
+    }
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+  
+  public String getInputTableName() {
+    return conf.get(DBConfiguration.INPUT_TABLE_NAME_PROPERTY);
+  }
+
+  public void setInputTableName(String tableName) {
+    conf.set(DBConfiguration.INPUT_TABLE_NAME_PROPERTY, tableName);
+  }
+
+  public String[] getInputFieldNames() {
+    return conf.getStrings(DBConfiguration.INPUT_FIELD_NAMES_PROPERTY);
+  }
+
+  public void setInputFieldNames(String... fieldNames) {
+    conf.setStrings(DBConfiguration.INPUT_FIELD_NAMES_PROPERTY, fieldNames);
+  }
+
+  public String getInputConditions() {
+    return conf.get(DBConfiguration.INPUT_CONDITIONS_PROPERTY);
+  }
+
+  public void setInputConditions(String conditions) {
+    if (conditions != null && conditions.length() > 0)
+      conf.set(DBConfiguration.INPUT_CONDITIONS_PROPERTY, conditions);
+  }
+
+  public String getInputOrderBy() {
+    return conf.get(DBConfiguration.INPUT_ORDER_BY_PROPERTY);
+  }
+  
+  public void setInputOrderBy(String orderby) {
+    if(orderby != null && orderby.length() >0) {
+      conf.set(DBConfiguration.INPUT_ORDER_BY_PROPERTY, orderby);
+    }
+  }
+  
+  public String getInputQuery() {
+    return conf.get(DBConfiguration.INPUT_QUERY);
+  }
+  
+  public void setInputQuery(String query) {
+    if(query != null && query.length() >0) {
+      conf.set(DBConfiguration.INPUT_QUERY, query);
+    }
+  }
+  
+  public String getInputCountQuery() {
+    return conf.get(DBConfiguration.INPUT_COUNT_QUERY);
+  }
+  
+  public void setInputCountQuery(String query) {
+    if(query != null && query.length() > 0) {
+      conf.set(DBConfiguration.INPUT_COUNT_QUERY, query);
+    }
+  }
+
+  public void setInputBoundingQuery(String query) {
+    if (query != null && query.length() > 0) {
+      conf.set(DBConfiguration.INPUT_BOUNDING_QUERY, query);
+    }
+  }
+
+  public String getInputBoundingQuery() {
+    return conf.get(DBConfiguration.INPUT_BOUNDING_QUERY);
+  }
+
+  public Class<?> getInputClass() {
+    return conf.getClass(DBConfiguration.INPUT_CLASS_PROPERTY,
+                         NullDBWritable.class);
+  }
+
+  public void setInputClass(Class<? extends DBWritable> inputClass) {
+    conf.setClass(DBConfiguration.INPUT_CLASS_PROPERTY, inputClass,
+                  DBWritable.class);
+  }
+
+  public String getOutputTableName() {
+    return conf.get(DBConfiguration.OUTPUT_TABLE_NAME_PROPERTY);
+  }
+
+  public void setOutputTableName(String tableName) {
+    conf.set(DBConfiguration.OUTPUT_TABLE_NAME_PROPERTY, tableName);
+  }
+
+  public String[] getOutputFieldNames() {
+    return conf.getStrings(DBConfiguration.OUTPUT_FIELD_NAMES_PROPERTY);
+  }
+
+  public void setOutputFieldNames(String... fieldNames) {
+    conf.setStrings(DBConfiguration.OUTPUT_FIELD_NAMES_PROPERTY, fieldNames);
+  }
+
+  public void setOutputFieldCount(int fieldCount) {
+    conf.setInt(DBConfiguration.OUTPUT_FIELD_COUNT_PROPERTY, fieldCount);
+  }
+  
+  public int getOutputFieldCount() {
+    return conf.getInt(OUTPUT_FIELD_COUNT_PROPERTY, 0);
+  }
+  
+}
+

+ 358 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/DBInputFormat.java

@@ -0,0 +1,358 @@
+/**
+ * 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.db;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+/**
+ * A InputFormat that reads input data from an SQL table.
+ * <p>
+ * DBInputFormat emits LongWritables containing the record number as 
+ * key and DBWritables as value. 
+ * 
+ * The SQL query, and input class can be using one of the two 
+ * setInput methods.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DBInputFormat<T extends DBWritable>
+    extends InputFormat<LongWritable, T> implements Configurable {
+
+  private String dbProductName = "DEFAULT";
+
+  /**
+   * A Class that does nothing, implementing DBWritable
+   */
+  @InterfaceStability.Evolving
+  public static class NullDBWritable implements DBWritable, Writable {
+    @Override
+    public void readFields(DataInput in) throws IOException { }
+    @Override
+    public void readFields(ResultSet arg0) throws SQLException { }
+    @Override
+    public void write(DataOutput out) throws IOException { }
+    @Override
+    public void write(PreparedStatement arg0) throws SQLException { }
+  }
+  
+  /**
+   * A InputSplit that spans a set of rows
+   */
+  @InterfaceStability.Evolving
+  public static class DBInputSplit extends InputSplit implements Writable {
+
+    private long end = 0;
+    private long start = 0;
+
+    /**
+     * Default Constructor
+     */
+    public DBInputSplit() {
+    }
+
+    /**
+     * Convenience Constructor
+     * @param start the index of the first row to select
+     * @param end the index of the last row to select
+     */
+    public DBInputSplit(long start, long end) {
+      this.start = start;
+      this.end = end;
+    }
+
+    /** {@inheritDoc} */
+    public String[] getLocations() throws IOException {
+      // TODO Add a layer to enable SQL "sharding" and support locality
+      return new String[] {};
+    }
+
+    /**
+     * @return The index of the first row to select
+     */
+    public long getStart() {
+      return start;
+    }
+
+    /**
+     * @return The index of the last row to select
+     */
+    public long getEnd() {
+      return end;
+    }
+
+    /**
+     * @return The total row count in this split
+     */
+    public long getLength() throws IOException {
+      return end - start;
+    }
+
+    /** {@inheritDoc} */
+    public void readFields(DataInput input) throws IOException {
+      start = input.readLong();
+      end = input.readLong();
+    }
+
+    /** {@inheritDoc} */
+    public void write(DataOutput output) throws IOException {
+      output.writeLong(start);
+      output.writeLong(end);
+    }
+  }
+
+  private String conditions;
+
+  private Connection connection;
+
+  private String tableName;
+
+  private String[] fieldNames;
+
+  private DBConfiguration dbConf;
+
+  /** {@inheritDoc} */
+  public void setConf(Configuration conf) {
+
+    dbConf = new DBConfiguration(conf);
+
+    try {
+      getConnection();
+
+      DatabaseMetaData dbMeta = connection.getMetaData();
+      this.dbProductName = dbMeta.getDatabaseProductName().toUpperCase();
+    }
+    catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+
+    tableName = dbConf.getInputTableName();
+    fieldNames = dbConf.getInputFieldNames();
+    conditions = dbConf.getInputConditions();
+  }
+
+  public Configuration getConf() {
+    return dbConf.getConf();
+  }
+  
+  public DBConfiguration getDBConf() {
+    return dbConf;
+  }
+
+  public Connection getConnection() {
+    try {
+      if (null == this.connection) {
+        // The connection was closed; reinstantiate it.
+        this.connection = dbConf.getConnection();
+        this.connection.setAutoCommit(false);
+        this.connection.setTransactionIsolation(
+            Connection.TRANSACTION_SERIALIZABLE);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return connection;
+  }
+
+  public String getDBProductName() {
+    return dbProductName;
+  }
+
+  protected RecordReader<LongWritable, T> createDBRecordReader(DBInputSplit split,
+      Configuration conf) throws IOException {
+
+    @SuppressWarnings("unchecked")
+    Class<T> inputClass = (Class<T>) (dbConf.getInputClass());
+    try {
+      // use database product name to determine appropriate record reader.
+      if (dbProductName.startsWith("ORACLE")) {
+        // use Oracle-specific db reader.
+        return new OracleDBRecordReader<T>(split, inputClass,
+            conf, getConnection(), getDBConf(), conditions, fieldNames,
+            tableName);
+      } else if (dbProductName.startsWith("MYSQL")) {
+        // use MySQL-specific db reader.
+        return new MySQLDBRecordReader<T>(split, inputClass,
+            conf, getConnection(), getDBConf(), conditions, fieldNames,
+            tableName);
+      } else {
+        // Generic reader.
+        return new DBRecordReader<T>(split, inputClass,
+            conf, getConnection(), getDBConf(), conditions, fieldNames,
+            tableName);
+      }
+    } catch (SQLException ex) {
+      throw new IOException(ex.getMessage());
+    }
+  }
+
+  /** {@inheritDoc} */
+  @SuppressWarnings("unchecked")
+  public RecordReader<LongWritable, T> createRecordReader(InputSplit split,
+      TaskAttemptContext context) throws IOException, InterruptedException {  
+
+    return createDBRecordReader((DBInputSplit) split, context.getConfiguration());
+  }
+
+  /** {@inheritDoc} */
+  public List<InputSplit> getSplits(JobContext job) throws IOException {
+
+    ResultSet results = null;  
+    Statement statement = null;
+    try {
+      statement = connection.createStatement();
+
+      results = statement.executeQuery(getCountQuery());
+      results.next();
+
+      long count = results.getLong(1);
+      int chunks = job.getConfiguration().getInt("mapred.map.tasks", 1);
+      long chunkSize = (count / chunks);
+
+      results.close();
+      statement.close();
+
+      List<InputSplit> splits = new ArrayList<InputSplit>();
+
+      // Split the rows into n-number of chunks and adjust the last chunk
+      // accordingly
+      for (int i = 0; i < chunks; i++) {
+        DBInputSplit split;
+
+        if ((i + 1) == chunks)
+          split = new DBInputSplit(i * chunkSize, count);
+        else
+          split = new DBInputSplit(i * chunkSize, (i * chunkSize)
+              + chunkSize);
+
+        splits.add(split);
+      }
+
+      connection.commit();
+      return splits;
+    } catch (SQLException e) {
+      throw new IOException("Got SQLException", e);
+    } finally {
+      try {
+        if (results != null) { results.close(); }
+      } catch (SQLException e1) {}
+      try {
+        if (statement != null) { statement.close(); }
+      } catch (SQLException e1) {}
+
+      closeConnection();
+    }
+  }
+
+  /** Returns the query for getting the total number of rows, 
+   * subclasses can override this for custom behaviour.*/
+  protected String getCountQuery() {
+    
+    if(dbConf.getInputCountQuery() != null) {
+      return dbConf.getInputCountQuery();
+    }
+    
+    StringBuilder query = new StringBuilder();
+    query.append("SELECT COUNT(*) FROM " + tableName);
+
+    if (conditions != null && conditions.length() > 0)
+      query.append(" WHERE " + conditions);
+    return query.toString();
+  }
+
+  /**
+   * Initializes the map-part of the job with the appropriate input settings.
+   * 
+   * @param job The map-reduce job
+   * @param inputClass the class object implementing DBWritable, which is the 
+   * Java object holding tuple fields.
+   * @param tableName The table to read data from
+   * @param conditions The condition which to select data with, 
+   * eg. '(updated > 20070101 AND length > 0)'
+   * @param orderBy the fieldNames in the orderBy clause.
+   * @param fieldNames The field names in the table
+   * @see #setInput(Job, Class, String, String)
+   */
+  public static void setInput(Job job, 
+      Class<? extends DBWritable> inputClass,
+      String tableName,String conditions, 
+      String orderBy, String... fieldNames) {
+    job.setInputFormatClass(DBInputFormat.class);
+    DBConfiguration dbConf = new DBConfiguration(job.getConfiguration());
+    dbConf.setInputClass(inputClass);
+    dbConf.setInputTableName(tableName);
+    dbConf.setInputFieldNames(fieldNames);
+    dbConf.setInputConditions(conditions);
+    dbConf.setInputOrderBy(orderBy);
+  }
+  
+  /**
+   * Initializes the map-part of the job with the appropriate input settings.
+   * 
+   * @param job The map-reduce job
+   * @param inputClass the class object implementing DBWritable, which is the 
+   * Java object holding tuple fields.
+   * @param inputQuery the input query to select fields. Example : 
+   * "SELECT f1, f2, f3 FROM Mytable ORDER BY f1"
+   * @param inputCountQuery the input query that returns 
+   * the number of records in the table. 
+   * Example : "SELECT COUNT(f1) FROM Mytable"
+   * @see #setInput(Job, Class, String, String, String, String...)
+   */
+  public static void setInput(Job job,
+      Class<? extends DBWritable> inputClass,
+      String inputQuery, String inputCountQuery) {
+    job.setInputFormatClass(DBInputFormat.class);
+    DBConfiguration dbConf = new DBConfiguration(job.getConfiguration());
+    dbConf.setInputClass(inputClass);
+    dbConf.setInputQuery(inputQuery);
+    dbConf.setInputCountQuery(inputCountQuery);
+  }
+
+  protected void closeConnection() {
+    try {
+      if (null != this.connection) {
+        this.connection.close();
+        this.connection = null;
+      }
+    } catch (SQLException sqlE) { } // ignore exception on close.
+  }
+}

+ 237 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/DBOutputFormat.java

@@ -0,0 +1,237 @@
+/**
+ * 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.db;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * A OutputFormat that sends the reduce output to a SQL table.
+ * <p> 
+ * {@link DBOutputFormat} accepts &lt;key,value&gt; pairs, where 
+ * key has a type extending DBWritable. Returned {@link RecordWriter} 
+ * writes <b>only the key</b> to the database with a batch SQL query.  
+ * 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DBOutputFormat<K  extends DBWritable, V> 
+extends OutputFormat<K,V> {
+
+  private static final Log LOG = LogFactory.getLog(DBOutputFormat.class);
+  public void checkOutputSpecs(JobContext context) 
+      throws IOException, InterruptedException {}
+
+  public OutputCommitter getOutputCommitter(TaskAttemptContext context) 
+      throws IOException, InterruptedException {
+    return new FileOutputCommitter(FileOutputFormat.getOutputPath(context),
+                                   context);
+  }
+
+  /**
+   * A RecordWriter that writes the reduce output to a SQL table
+   */
+  @InterfaceStability.Evolving
+  public class DBRecordWriter 
+      extends RecordWriter<K, V> {
+
+    private Connection connection;
+    private PreparedStatement statement;
+
+    public DBRecordWriter() throws SQLException {
+    }
+
+    public DBRecordWriter(Connection connection
+        , PreparedStatement statement) throws SQLException {
+      this.connection = connection;
+      this.statement = statement;
+      this.connection.setAutoCommit(false);
+    }
+
+    public Connection getConnection() {
+      return connection;
+    }
+    
+    public PreparedStatement getStatement() {
+      return statement;
+    }
+    
+    /** {@inheritDoc} */
+    public void close(TaskAttemptContext context) throws IOException {
+      try {
+        statement.executeBatch();
+        connection.commit();
+      } catch (SQLException e) {
+        try {
+          connection.rollback();
+        }
+        catch (SQLException ex) {
+          LOG.warn(StringUtils.stringifyException(ex));
+        }
+        throw new IOException(e.getMessage());
+      } finally {
+        try {
+          statement.close();
+          connection.close();
+        }
+        catch (SQLException ex) {
+          throw new IOException(ex.getMessage());
+        }
+      }
+    }
+
+    /** {@inheritDoc} */
+    public void write(K key, V value) throws IOException {
+      try {
+        key.write(statement);
+        statement.addBatch();
+      } catch (SQLException e) {
+        e.printStackTrace();
+      }
+    }
+  }
+
+  /**
+   * Constructs the query used as the prepared statement to insert data.
+   * 
+   * @param table
+   *          the table to insert into
+   * @param fieldNames
+   *          the fields to insert into. If field names are unknown, supply an
+   *          array of nulls.
+   */
+  public String constructQuery(String table, String[] fieldNames) {
+    if(fieldNames == null) {
+      throw new IllegalArgumentException("Field names may not be null");
+    }
+
+    StringBuilder query = new StringBuilder();
+    query.append("INSERT INTO ").append(table);
+
+    if (fieldNames.length > 0 && fieldNames[0] != null) {
+      query.append(" (");
+      for (int i = 0; i < fieldNames.length; i++) {
+        query.append(fieldNames[i]);
+        if (i != fieldNames.length - 1) {
+          query.append(",");
+        }
+      }
+      query.append(")");
+    }
+    query.append(" VALUES (");
+
+    for (int i = 0; i < fieldNames.length; i++) {
+      query.append("?");
+      if(i != fieldNames.length - 1) {
+        query.append(",");
+      }
+    }
+    query.append(");");
+
+    return query.toString();
+  }
+
+  /** {@inheritDoc} */
+  public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context) 
+      throws IOException {
+    DBConfiguration dbConf = new DBConfiguration(context.getConfiguration());
+    String tableName = dbConf.getOutputTableName();
+    String[] fieldNames = dbConf.getOutputFieldNames();
+    
+    if(fieldNames == null) {
+      fieldNames = new String[dbConf.getOutputFieldCount()];
+    }
+    
+    try {
+      Connection connection = dbConf.getConnection();
+      PreparedStatement statement = null;
+  
+      statement = connection.prepareStatement(
+                    constructQuery(tableName, fieldNames));
+      return new DBRecordWriter(connection, statement);
+    } catch (Exception ex) {
+      throw new IOException(ex.getMessage());
+    }
+  }
+
+  /**
+   * Initializes the reduce-part of the job with 
+   * the appropriate output settings
+   * 
+   * @param job The job
+   * @param tableName The table to insert data into
+   * @param fieldNames The field names in the table.
+   */
+  public static void setOutput(Job job, String tableName, 
+      String... fieldNames) throws IOException {
+    if(fieldNames.length > 0 && fieldNames[0] != null) {
+      DBConfiguration dbConf = setOutput(job, tableName);
+      dbConf.setOutputFieldNames(fieldNames);
+    } else {
+      if (fieldNames.length > 0) {
+        setOutput(job, tableName, fieldNames.length);
+      }
+      else { 
+        throw new IllegalArgumentException(
+          "Field names must be greater than 0");
+      }
+    }
+  }
+  
+  /**
+   * Initializes the reduce-part of the job 
+   * with the appropriate output settings
+   * 
+   * @param job The job
+   * @param tableName The table to insert data into
+   * @param fieldCount the number of fields in the table.
+   */
+  public static void setOutput(Job job, String tableName, 
+      int fieldCount) throws IOException {
+    DBConfiguration dbConf = setOutput(job, tableName);
+    dbConf.setOutputFieldCount(fieldCount);
+  }
+  
+  private static DBConfiguration setOutput(Job job,
+      String tableName) throws IOException {
+    job.setOutputFormatClass(DBOutputFormat.class);
+    job.setReduceSpeculativeExecution(false);
+
+    DBConfiguration dbConf = new DBConfiguration(job.getConfiguration());
+    
+    dbConf.setOutputTableName(tableName);
+    return dbConf;
+  }
+}

+ 275 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/DBRecordReader.java

@@ -0,0 +1,275 @@
+/**
+ * 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.db;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A RecordReader that reads records from a SQL table.
+ * Emits LongWritables containing the record number as 
+ * key and DBWritables as value.  
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class DBRecordReader<T extends DBWritable> extends
+    RecordReader<LongWritable, T> {
+
+  private static final Log LOG = LogFactory.getLog(DBRecordReader.class);
+
+  private ResultSet results = null;
+
+  private Class<T> inputClass;
+
+  private Configuration conf;
+
+  private DBInputFormat.DBInputSplit split;
+
+  private long pos = 0;
+  
+  private LongWritable key = null;
+  
+  private T value = null;
+
+  private Connection connection;
+
+  protected PreparedStatement statement;
+
+  private DBConfiguration dbConf;
+
+  private String conditions;
+
+  private String [] fieldNames;
+
+  private String tableName;
+
+  /**
+   * @param split The InputSplit to read data for
+   * @throws SQLException 
+   */
+  public DBRecordReader(DBInputFormat.DBInputSplit split, 
+      Class<T> inputClass, Configuration conf, Connection conn, DBConfiguration dbConfig,
+      String cond, String [] fields, String table)
+      throws SQLException {
+    this.inputClass = inputClass;
+    this.split = split;
+    this.conf = conf;
+    this.connection = conn;
+    this.dbConf = dbConfig;
+    this.conditions = cond;
+    this.fieldNames = fields;
+    this.tableName = table;
+  }
+
+  protected ResultSet executeQuery(String query) throws SQLException {
+    this.statement = connection.prepareStatement(query,
+        ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+    return statement.executeQuery();
+  }
+
+  /** Returns the query for selecting the records, 
+   * subclasses can override this for custom behaviour.*/
+  protected String getSelectQuery() {
+    StringBuilder query = new StringBuilder();
+
+    // Default codepath for MySQL, HSQLDB, etc. Relies on LIMIT/OFFSET for splits.
+    if(dbConf.getInputQuery() == null) {
+      query.append("SELECT ");
+  
+      for (int i = 0; i < fieldNames.length; i++) {
+        query.append(fieldNames[i]);
+        if (i != fieldNames.length -1) {
+          query.append(", ");
+        }
+      }
+
+      query.append(" FROM ").append(tableName);
+      query.append(" AS ").append(tableName); //in hsqldb this is necessary
+      if (conditions != null && conditions.length() > 0) {
+        query.append(" WHERE (").append(conditions).append(")");
+      }
+
+      String orderBy = dbConf.getInputOrderBy();
+      if (orderBy != null && orderBy.length() > 0) {
+        query.append(" ORDER BY ").append(orderBy);
+      }
+    } else {
+      //PREBUILT QUERY
+      query.append(dbConf.getInputQuery());
+    }
+        
+    try {
+      query.append(" LIMIT ").append(split.getLength());
+      query.append(" OFFSET ").append(split.getStart());
+    } catch (IOException ex) {
+      // Ignore, will not throw.
+    }		
+
+    return query.toString();
+  }
+
+  /** {@inheritDoc} */
+  public void close() throws IOException {
+    try {
+      if (null != results) {
+        results.close();
+      }
+      if (null != statement) {
+        statement.close();
+      }
+      if (null != connection) {
+        connection.commit();
+        connection.close();
+      }
+    } catch (SQLException e) {
+      throw new IOException(e.getMessage());
+    }
+  }
+
+  public void initialize(InputSplit split, TaskAttemptContext context) 
+      throws IOException, InterruptedException {
+    //do nothing
+  }
+
+  /** {@inheritDoc} */
+  public LongWritable getCurrentKey() {
+    return key;  
+  }
+
+  /** {@inheritDoc} */
+  public T getCurrentValue() {
+    return value;
+  }
+
+  /**
+   * @deprecated 
+   */
+  @Deprecated
+  public T createValue() {
+    return ReflectionUtils.newInstance(inputClass, conf);
+  }
+
+  /**
+   * @deprecated 
+   */
+  @Deprecated
+  public long getPos() throws IOException {
+    return pos;
+  }
+
+  /**
+   * @deprecated Use {@link #nextKeyValue()}
+   */
+  @Deprecated
+  public boolean next(LongWritable key, T value) throws IOException {
+    this.key = key;
+    this.value = value;
+    return nextKeyValue();
+  }
+
+  /** {@inheritDoc} */
+  public float getProgress() throws IOException {
+    return pos / (float)split.getLength();
+  }
+
+  /** {@inheritDoc} */
+  public boolean nextKeyValue() throws IOException {
+    try {
+      if (key == null) {
+        key = new LongWritable();
+      }
+      if (value == null) {
+        value = createValue();
+      }
+      if (null == this.results) {
+        // First time into this method, run the query.
+        this.results = executeQuery(getSelectQuery());
+      }
+      if (!results.next())
+        return false;
+
+      // Set the key field value as the output key value
+      key.set(pos + split.getStart());
+
+      value.readFields(results);
+
+      pos ++;
+    } catch (SQLException e) {
+      throw new IOException("SQLException in nextKeyValue", e);
+    }
+    return true;
+  }
+
+  protected DBInputFormat.DBInputSplit getSplit() {
+    return split;
+  }
+
+  protected String [] getFieldNames() {
+    return fieldNames;
+  }
+
+  protected String getTableName() {
+    return tableName;
+  }
+
+  protected String getConditions() {
+    return conditions;
+  }
+
+  protected DBConfiguration getDBConf() {
+    return dbConf;
+  }
+
+  protected Connection getConnection() {
+    return connection;
+  }
+
+  protected PreparedStatement getStatement() {
+    return statement;
+  }
+
+  protected void setStatement(PreparedStatement stmt) {
+    this.statement = stmt;
+  }
+}

+ 47 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/DBSplitter.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.lib.db;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+/**
+ * DBSplitter will generate DBInputSplits to use with DataDrivenDBInputFormat.
+ * DataDrivenDBInputFormat needs to interpolate between two values that
+ * represent the lowest and highest valued records to import. Depending
+ * on the data-type of the column, this requires different behavior.
+ * DBSplitter implementations should perform this for a data type or family
+ * of data types.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface DBSplitter {
+  /**
+   * Given a ResultSet containing one record (and already advanced to that record)
+   * with two columns (a low value, and a high value, both of the same type), determine
+   * a set of splits that span the given values.
+   */
+  List<InputSplit> split(Configuration conf, ResultSet results, String colName) throws SQLException;
+}

+ 95 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/DBWritable.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.db;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Objects that are read from/written to a database should implement
+ * <code>DBWritable</code>. DBWritable, is similar to {@link Writable} 
+ * except that the {@link #write(PreparedStatement)} method takes a 
+ * {@link PreparedStatement}, and {@link #readFields(ResultSet)} 
+ * takes a {@link ResultSet}. 
+ * <p>
+ * Implementations are responsible for writing the fields of the object 
+ * to PreparedStatement, and reading the fields of the object from the 
+ * ResultSet. 
+ * 
+ * <p>Example:</p>
+ * If we have the following table in the database :
+ * <pre>
+ * CREATE TABLE MyTable (
+ *   counter        INTEGER NOT NULL,
+ *   timestamp      BIGINT  NOT NULL,
+ * );
+ * </pre>
+ * then we can read/write the tuples from/to the table with :
+ * <p><pre>
+ * public class MyWritable implements Writable, DBWritable {
+ *   // Some data     
+ *   private int counter;
+ *   private long timestamp;
+ *       
+ *   //Writable#write() implementation
+ *   public void write(DataOutput out) throws IOException {
+ *     out.writeInt(counter);
+ *     out.writeLong(timestamp);
+ *   }
+ *       
+ *   //Writable#readFields() implementation
+ *   public void readFields(DataInput in) throws IOException {
+ *     counter = in.readInt();
+ *     timestamp = in.readLong();
+ *   }
+ *       
+ *   public void write(PreparedStatement statement) throws SQLException {
+ *     statement.setInt(1, counter);
+ *     statement.setLong(2, timestamp);
+ *   }
+ *       
+ *   public void readFields(ResultSet resultSet) throws SQLException {
+ *     counter = resultSet.getInt(1);
+ *     timestamp = resultSet.getLong(2);
+ *   } 
+ * }
+ * </pre></p>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public interface DBWritable {
+
+  /**
+   * Sets the fields of the object in the {@link PreparedStatement}.
+   * @param statement the statement that the fields are put into.
+   * @throws SQLException
+   */
+	public void write(PreparedStatement statement) throws SQLException;
+	
+	/**
+	 * Reads the fields of the object from the {@link ResultSet}. 
+	 * @param resultSet the {@link ResultSet} to get the fields from.
+	 * @throws SQLException
+	 */
+	public void readFields(ResultSet resultSet) throws SQLException ; 
+}

+ 330 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBInputFormat.java

@@ -0,0 +1,330 @@
+/**
+ * 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.db;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A InputFormat that reads input data from an SQL table.
+ * Operates like DBInputFormat, but instead of using LIMIT and OFFSET to demarcate
+ * splits, it tries to generate WHERE clauses which separate the data into roughly
+ * equivalent shards.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class DataDrivenDBInputFormat<T extends DBWritable>
+    extends DBInputFormat<T> implements Configurable {
+
+  private static final Log LOG = LogFactory.getLog(DataDrivenDBInputFormat.class);
+
+  /** If users are providing their own query, the following string is expected to
+      appear in the WHERE clause, which will be substituted with a pair of conditions
+      on the input to allow input splits to parallelise the import. */
+  public static final String SUBSTITUTE_TOKEN = "$CONDITIONS";
+
+  /**
+   * A InputSplit that spans a set of rows
+   */
+  @InterfaceStability.Evolving
+  public static class DataDrivenDBInputSplit extends DBInputFormat.DBInputSplit {
+
+    private String lowerBoundClause;
+    private String upperBoundClause;
+
+    /**
+     * Default Constructor
+     */
+    public DataDrivenDBInputSplit() {
+    }
+
+    /**
+     * Convenience Constructor
+     * @param lower the string to be put in the WHERE clause to guard on the 'lower' end
+     * @param upper the string to be put in the WHERE clause to guard on the 'upper' end
+     */
+    public DataDrivenDBInputSplit(final String lower, final String upper) {
+      this.lowerBoundClause = lower;
+      this.upperBoundClause = upper;
+    }
+
+
+    /**
+     * @return The total row count in this split
+     */
+    public long getLength() throws IOException {
+      return 0; // unfortunately, we don't know this.
+    }
+
+    /** {@inheritDoc} */
+    public void readFields(DataInput input) throws IOException {
+      this.lowerBoundClause = Text.readString(input);
+      this.upperBoundClause = Text.readString(input);
+    }
+
+    /** {@inheritDoc} */
+    public void write(DataOutput output) throws IOException {
+      Text.writeString(output, this.lowerBoundClause);
+      Text.writeString(output, this.upperBoundClause);
+    }
+
+    public String getLowerClause() {
+      return lowerBoundClause;
+    }
+
+    public String getUpperClause() {
+      return upperBoundClause;
+    }
+  }
+
+  /**
+   * @return the DBSplitter implementation to use to divide the table/query into InputSplits.
+   */
+  protected DBSplitter getSplitter(int sqlDataType) {
+    switch (sqlDataType) {
+    case Types.NUMERIC:
+    case Types.DECIMAL:
+      return new BigDecimalSplitter();
+
+    case Types.BIT:
+    case Types.BOOLEAN:
+      return new BooleanSplitter();
+
+    case Types.INTEGER:
+    case Types.TINYINT:
+    case Types.SMALLINT:
+    case Types.BIGINT:
+      return new IntegerSplitter();
+
+    case Types.REAL:
+    case Types.FLOAT:
+    case Types.DOUBLE:
+      return new FloatSplitter();
+
+    case Types.CHAR:
+    case Types.VARCHAR:
+    case Types.LONGVARCHAR:
+      return new TextSplitter();
+
+    case Types.DATE:
+    case Types.TIME:
+    case Types.TIMESTAMP:
+      return new DateSplitter();
+
+    default:
+      // TODO: Support BINARY, VARBINARY, LONGVARBINARY, DISTINCT, CLOB, BLOB, ARRAY
+      // STRUCT, REF, DATALINK, and JAVA_OBJECT.
+      return null;
+    }
+  }
+
+  /** {@inheritDoc} */
+  public List<InputSplit> getSplits(JobContext job) throws IOException {
+
+    int targetNumTasks = job.getConfiguration().getInt("mapred.map.tasks", 1);
+    if (1 == targetNumTasks) {
+      // There's no need to run a bounding vals query; just return a split
+      // that separates nothing. This can be considerably more optimal for a
+      // large table with no index.
+      List<InputSplit> singletonSplit = new ArrayList<InputSplit>();
+      singletonSplit.add(new DataDrivenDBInputSplit("1=1", "1=1"));
+      return singletonSplit;
+    }
+
+    ResultSet results = null;
+    Statement statement = null;
+    Connection connection = getConnection();
+    try {
+      statement = connection.createStatement();
+
+      results = statement.executeQuery(getBoundingValsQuery());
+      results.next();
+
+      // Based on the type of the results, use a different mechanism
+      // for interpolating split points (i.e., numeric splits, text splits,
+      // dates, etc.)
+      int sqlDataType = results.getMetaData().getColumnType(1);
+      DBSplitter splitter = getSplitter(sqlDataType);
+      if (null == splitter) {
+        throw new IOException("Unknown SQL data type: " + sqlDataType);
+      }
+
+      return splitter.split(job.getConfiguration(), results, getDBConf().getInputOrderBy());
+    } catch (SQLException e) {
+      throw new IOException(e.getMessage());
+    } finally {
+      // More-or-less ignore SQL exceptions here, but log in case we need it.
+      try {
+        if (null != results) {
+          results.close();
+        }
+      } catch (SQLException se) {
+        LOG.debug("SQLException closing resultset: " + se.toString());
+      }
+
+      try {
+        if (null != statement) {
+          statement.close();
+        }
+      } catch (SQLException se) {
+        LOG.debug("SQLException closing statement: " + se.toString());
+      }
+
+      try {
+        connection.commit();
+        closeConnection();
+      } catch (SQLException se) {
+        LOG.debug("SQLException committing split transaction: " + se.toString());
+      }
+    }
+  }
+
+  /**
+   * @return a query which returns the minimum and maximum values for
+   * the order-by column.
+   *
+   * The min value should be in the first column, and the
+   * max value should be in the second column of the results.
+   */
+  protected String getBoundingValsQuery() {
+    // If the user has provided a query, use that instead.
+    String userQuery = getDBConf().getInputBoundingQuery();
+    if (null != userQuery) {
+      return userQuery;
+    }
+
+    // Auto-generate one based on the table name we've been provided with.
+    StringBuilder query = new StringBuilder();
+
+    String splitCol = getDBConf().getInputOrderBy();
+    query.append("SELECT MIN(").append(splitCol).append("), ");
+    query.append("MAX(").append(splitCol).append(") FROM ");
+    query.append(getDBConf().getInputTableName());
+    String conditions = getDBConf().getInputConditions();
+    if (null != conditions) {
+      query.append(" WHERE ( " + conditions + " )");
+    }
+
+    return query.toString();
+  }
+
+  /** Set the user-defined bounding query to use with a user-defined query.
+      This *must* include the substring "$CONDITIONS"
+      (DataDrivenDBInputFormat.SUBSTITUTE_TOKEN) inside the WHERE clause,
+      so that DataDrivenDBInputFormat knows where to insert split clauses.
+      e.g., "SELECT foo FROM mytable WHERE $CONDITIONS"
+      This will be expanded to something like:
+        SELECT foo FROM mytable WHERE (id &gt; 100) AND (id &lt; 250)
+      inside each split.
+    */
+  public static void setBoundingQuery(Configuration conf, String query) {
+    if (null != query) {
+      // If the user's settng a query, warn if they don't allow conditions.
+      if (query.indexOf(SUBSTITUTE_TOKEN) == -1) {
+        LOG.warn("Could not find " + SUBSTITUTE_TOKEN + " token in query: " + query
+            + "; splits may not partition data.");
+      }
+    }
+
+    conf.set(DBConfiguration.INPUT_BOUNDING_QUERY, query);
+  }
+
+  protected RecordReader<LongWritable, T> createDBRecordReader(DBInputSplit split,
+      Configuration conf) throws IOException {
+
+    DBConfiguration dbConf = getDBConf();
+    @SuppressWarnings("unchecked")
+    Class<T> inputClass = (Class<T>) (dbConf.getInputClass());
+    String dbProductName = getDBProductName();
+
+    LOG.debug("Creating db record reader for db product: " + dbProductName);
+
+    try {
+      // use database product name to determine appropriate record reader.
+      if (dbProductName.startsWith("MYSQL")) {
+        // use MySQL-specific db reader.
+        return new MySQLDataDrivenDBRecordReader<T>(split, inputClass,
+            conf, getConnection(), dbConf, dbConf.getInputConditions(),
+            dbConf.getInputFieldNames(), dbConf.getInputTableName());
+      } else {
+        // Generic reader.
+        return new DataDrivenDBRecordReader<T>(split, inputClass,
+            conf, getConnection(), dbConf, dbConf.getInputConditions(),
+            dbConf.getInputFieldNames(), dbConf.getInputTableName(),
+            dbProductName);
+      }
+    } catch (SQLException ex) {
+      throw new IOException(ex.getMessage());
+    }
+  }
+
+  // Configuration methods override superclass to ensure that the proper
+  // DataDrivenDBInputFormat gets used.
+
+  /** Note that the "orderBy" column is called the "splitBy" in this version.
+    * We reuse the same field, but it's not strictly ordering it -- just partitioning
+    * the results.
+    */
+  public static void setInput(Job job, 
+      Class<? extends DBWritable> inputClass,
+      String tableName,String conditions, 
+      String splitBy, String... fieldNames) {
+    DBInputFormat.setInput(job, inputClass, tableName, conditions, splitBy, fieldNames);
+    job.setInputFormatClass(DataDrivenDBInputFormat.class);
+  }
+
+  /** setInput() takes a custom query and a separate "bounding query" to use
+      instead of the custom "count query" used by DBInputFormat.
+    */
+  public static void setInput(Job job,
+      Class<? extends DBWritable> inputClass,
+      String inputQuery, String inputBoundingQuery) {
+    DBInputFormat.setInput(job, inputClass, inputQuery, "");
+    job.getConfiguration().set(DBConfiguration.INPUT_BOUNDING_QUERY, inputBoundingQuery);
+    job.setInputFormatClass(DataDrivenDBInputFormat.class);
+  }
+}

+ 137 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBRecordReader.java

@@ -0,0 +1,137 @@
+/**
+ * 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.db;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A RecordReader that reads records from a SQL table,
+ * using data-driven WHERE clause splits.
+ * Emits LongWritables containing the record number as
+ * key and DBWritables as value.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class DataDrivenDBRecordReader<T extends DBWritable> extends DBRecordReader<T> {
+
+  private static final Log LOG = LogFactory.getLog(DataDrivenDBRecordReader.class);
+
+  private String dbProductName; // database manufacturer string.
+
+  /**
+   * @param split The InputSplit to read data for
+   * @throws SQLException 
+   */
+  public DataDrivenDBRecordReader(DBInputFormat.DBInputSplit split,
+      Class<T> inputClass, Configuration conf, Connection conn, DBConfiguration dbConfig,
+      String cond, String [] fields, String table, String dbProduct)
+      throws SQLException {
+    super(split, inputClass, conf, conn, dbConfig, cond, fields, table);
+    this.dbProductName = dbProduct;
+  }
+
+  /** Returns the query for selecting the records,
+   * subclasses can override this for custom behaviour.*/
+  @SuppressWarnings("unchecked")
+  protected String getSelectQuery() {
+    StringBuilder query = new StringBuilder();
+    DataDrivenDBInputFormat.DataDrivenDBInputSplit dataSplit =
+        (DataDrivenDBInputFormat.DataDrivenDBInputSplit) getSplit();
+    DBConfiguration dbConf = getDBConf();
+    String [] fieldNames = getFieldNames();
+    String tableName = getTableName();
+    String conditions = getConditions();
+
+    // Build the WHERE clauses associated with the data split first.
+    // We need them in both branches of this function.
+    StringBuilder conditionClauses = new StringBuilder();
+    conditionClauses.append("( ").append(dataSplit.getLowerClause());
+    conditionClauses.append(" ) AND ( ").append(dataSplit.getUpperClause());
+    conditionClauses.append(" )");
+
+    if(dbConf.getInputQuery() == null) {
+      // We need to generate the entire query.
+      query.append("SELECT ");
+
+      for (int i = 0; i < fieldNames.length; i++) {
+        query.append(fieldNames[i]);
+        if (i != fieldNames.length -1) {
+          query.append(", ");
+        }
+      }
+
+      query.append(" FROM ").append(tableName);
+      if (!dbProductName.startsWith("ORACLE")) {
+        // Seems to be necessary for hsqldb? Oracle explicitly does *not*
+        // use this clause.
+        query.append(" AS ").append(tableName);
+      }
+      query.append(" WHERE ");
+      if (conditions != null && conditions.length() > 0) {
+        // Put the user's conditions first.
+        query.append("( ").append(conditions).append(" ) AND ");
+      }
+
+      // Now append the conditions associated with our split.
+      query.append(conditionClauses.toString());
+
+    } else {
+      // User provided the query. We replace the special token with our WHERE clause.
+      String inputQuery = dbConf.getInputQuery();
+      if (inputQuery.indexOf(DataDrivenDBInputFormat.SUBSTITUTE_TOKEN) == -1) {
+        LOG.error("Could not find the clause substitution token "
+            + DataDrivenDBInputFormat.SUBSTITUTE_TOKEN + " in the query: ["
+            + inputQuery + "]. Parallel splits may not work correctly.");
+      }
+
+      query.append(inputQuery.replace(DataDrivenDBInputFormat.SUBSTITUTE_TOKEN,
+          conditionClauses.toString()));
+    }
+
+    LOG.debug("Using query: " + query.toString());
+
+    return query.toString();
+  }
+}

+ 177 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/DateSplitter.java

@@ -0,0 +1,177 @@
+/**
+ * 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.db;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+/**
+ * Implement DBSplitter over date/time values.
+ * Make use of logic from IntegerSplitter, since date/time are just longs
+ * in Java.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class DateSplitter extends IntegerSplitter {
+
+  private static final Log LOG = LogFactory.getLog(DateSplitter.class);
+
+  public List<InputSplit> split(Configuration conf, ResultSet results, String colName)
+      throws SQLException {
+
+    long minVal;
+    long maxVal;
+
+    int sqlDataType = results.getMetaData().getColumnType(1);
+    minVal = resultSetColToLong(results, 1, sqlDataType);
+    maxVal = resultSetColToLong(results, 2, sqlDataType);
+
+    String lowClausePrefix = colName + " >= ";
+    String highClausePrefix = colName + " < ";
+
+    int numSplits = conf.getInt("mapred.map.tasks", 1);
+    if (numSplits < 1) {
+      numSplits = 1;
+    }
+
+    if (minVal == Long.MIN_VALUE && maxVal == Long.MIN_VALUE) {
+      // The range of acceptable dates is NULL to NULL. Just create a single split.
+      List<InputSplit> splits = new ArrayList<InputSplit>();
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+      return splits;
+    }
+
+    // Gather the split point integers
+    List<Long> splitPoints = split(numSplits, minVal, maxVal);
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+
+    // Turn the split points into a set of intervals.
+    long start = splitPoints.get(0);
+    Date startDate = longToDate(start, sqlDataType);
+    if (sqlDataType == Types.TIMESTAMP) {
+      // The lower bound's nanos value needs to match the actual lower-bound nanos.
+      try {
+        ((java.sql.Timestamp) startDate).setNanos(results.getTimestamp(1).getNanos());
+      } catch (NullPointerException npe) {
+        // If the lower bound was NULL, we'll get an NPE; just ignore it and don't set nanos.
+      }
+    }
+
+    for (int i = 1; i < splitPoints.size(); i++) {
+      long end = splitPoints.get(i);
+      Date endDate = longToDate(end, sqlDataType);
+
+      if (i == splitPoints.size() - 1) {
+        if (sqlDataType == Types.TIMESTAMP) {
+          // The upper bound's nanos value needs to match the actual upper-bound nanos.
+          try {
+            ((java.sql.Timestamp) endDate).setNanos(results.getTimestamp(2).getNanos());
+          } catch (NullPointerException npe) {
+            // If the upper bound was NULL, we'll get an NPE; just ignore it and don't set nanos.
+          }
+        }
+        // This is the last one; use a closed interval.
+        splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+            lowClausePrefix + dateToString(startDate),
+            colName + " <= " + dateToString(endDate)));
+      } else {
+        // Normal open-interval case.
+        splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+            lowClausePrefix + dateToString(startDate),
+            highClausePrefix + dateToString(endDate)));
+      }
+
+      start = end;
+      startDate = endDate;
+    }
+
+    if (minVal == Long.MIN_VALUE || maxVal == Long.MIN_VALUE) {
+      // Add an extra split to handle the null case that we saw.
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+    }
+
+    return splits;
+  }
+
+  /** Retrieve the value from the column in a type-appropriate manner and return
+      its timestamp since the epoch. If the column is null, then return Long.MIN_VALUE.
+      This will cause a special split to be generated for the NULL case, but may also
+      cause poorly-balanced splits if most of the actual dates are positive time
+      since the epoch, etc.
+    */
+  private long resultSetColToLong(ResultSet rs, int colNum, int sqlDataType) throws SQLException {
+    try {
+      switch (sqlDataType) {
+      case Types.DATE:
+        return rs.getDate(colNum).getTime();
+      case Types.TIME:
+        return rs.getTime(colNum).getTime();
+      case Types.TIMESTAMP:
+        return rs.getTimestamp(colNum).getTime();
+      default:
+        throw new SQLException("Not a date-type field");
+      }
+    } catch (NullPointerException npe) {
+      // null column. return minimum long value.
+      LOG.warn("Encountered a NULL date in the split column. Splits may be poorly balanced.");
+      return Long.MIN_VALUE;
+    }
+  }
+
+  /**  Parse the long-valued timestamp into the appropriate SQL date type. */
+  private Date longToDate(long val, int sqlDataType) {
+    switch (sqlDataType) {
+    case Types.DATE:
+      return new java.sql.Date(val);
+    case Types.TIME:
+      return new java.sql.Time(val);
+    case Types.TIMESTAMP:
+      return new java.sql.Timestamp(val);
+    default: // Shouldn't ever hit this case.
+      return null;
+    }
+  }
+
+  /**
+   * Given a Date 'd', format it as a string for use in a SQL date
+   * comparison operation.
+   * @param d the date to format.
+   * @return the string representing this date in SQL with any appropriate
+   * quotation characters, etc.
+   */
+  protected String dateToString(Date d) {
+    return "'" + d.toString() + "'";
+  }
+}

+ 104 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/FloatSplitter.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.db;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+/**
+ * Implement DBSplitter over floating-point values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class FloatSplitter implements DBSplitter {
+
+  private static final Log LOG = LogFactory.getLog(FloatSplitter.class);
+
+  private static final double MIN_INCREMENT = 10000 * Double.MIN_VALUE;
+
+  public List<InputSplit> split(Configuration conf, ResultSet results, String colName)
+      throws SQLException {
+
+    LOG.warn("Generating splits for a floating-point index column. Due to the");
+    LOG.warn("imprecise representation of floating-point values in Java, this");
+    LOG.warn("may result in an incomplete import.");
+    LOG.warn("You are strongly encouraged to choose an integral split column.");
+
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+
+    if (results.getString(1) == null && results.getString(2) == null) {
+      // Range is null to null. Return a null split accordingly.
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+      return splits;
+    }
+
+    double minVal = results.getDouble(1);
+    double maxVal = results.getDouble(2);
+
+    // Use this as a hint. May need an extra task if the size doesn't
+    // divide cleanly.
+    int numSplits = conf.getInt("mapred.map.tasks", 1);
+    double splitSize = (maxVal - minVal) / (double) numSplits;
+
+    if (splitSize < MIN_INCREMENT) {
+      splitSize = MIN_INCREMENT;
+    }
+
+    String lowClausePrefix = colName + " >= ";
+    String highClausePrefix = colName + " < ";
+
+    double curLower = minVal;
+    double curUpper = curLower + splitSize;
+
+    while (curUpper < maxVal) {
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          lowClausePrefix + Double.toString(curLower),
+          highClausePrefix + Double.toString(curUpper)));
+
+      curLower = curUpper;
+      curUpper += splitSize;
+    }
+
+    // Catch any overage and create the closed interval for the last split.
+    if (curLower <= maxVal || splits.size() == 1) {
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          lowClausePrefix + Double.toString(curUpper),
+          colName + " <= " + Double.toString(maxVal)));
+    }
+
+    if (results.getString(1) == null || results.getString(2) == null) {
+      // At least one extrema is null; add a null split.
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+    }
+
+    return splits;
+  }
+}

+ 131 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/IntegerSplitter.java

@@ -0,0 +1,131 @@
+/**
+ * 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.db;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+/**
+ * Implement DBSplitter over integer values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class IntegerSplitter implements DBSplitter {
+  public List<InputSplit> split(Configuration conf, ResultSet results, String colName)
+      throws SQLException {
+
+    long minVal = results.getLong(1);
+    long maxVal = results.getLong(2);
+
+    String lowClausePrefix = colName + " >= ";
+    String highClausePrefix = colName + " < ";
+
+    int numSplits = conf.getInt("mapred.map.tasks", 1);
+    if (numSplits < 1) {
+      numSplits = 1;
+    }
+
+    if (results.getString(1) == null && results.getString(2) == null) {
+      // Range is null to null. Return a null split accordingly.
+      List<InputSplit> splits = new ArrayList<InputSplit>();
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+      return splits;
+    }
+
+    // Get all the split points together.
+    List<Long> splitPoints = split(numSplits, minVal, maxVal);
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+
+    // Turn the split points into a set of intervals.
+    long start = splitPoints.get(0);
+    for (int i = 1; i < splitPoints.size(); i++) {
+      long end = splitPoints.get(i);
+
+      if (i == splitPoints.size() - 1) {
+        // This is the last one; use a closed interval.
+        splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+            lowClausePrefix + Long.toString(start),
+            colName + " <= " + Long.toString(end)));
+      } else {
+        // Normal open-interval case.
+        splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+            lowClausePrefix + Long.toString(start),
+            highClausePrefix + Long.toString(end)));
+      }
+
+      start = end;
+    }
+
+    if (results.getString(1) == null || results.getString(2) == null) {
+      // At least one extrema is null; add a null split.
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+    }
+
+    return splits;
+  }
+
+  /**
+   * Returns a list of longs one element longer than the list of input splits.
+   * This represents the boundaries between input splits.
+   * All splits are open on the top end, except the last one.
+   *
+   * So the list [0, 5, 8, 12, 18] would represent splits capturing the intervals:
+   *
+   * [0, 5)
+   * [5, 8)
+   * [8, 12)
+   * [12, 18] note the closed interval for the last split.
+   */
+  List<Long> split(long numSplits, long minVal, long maxVal)
+      throws SQLException {
+
+    List<Long> splits = new ArrayList<Long>();
+
+    // Use numSplits as a hint. May need an extra task if the size doesn't
+    // divide cleanly.
+
+    long splitSize = (maxVal - minVal) / numSplits;
+    if (splitSize < 1) {
+      splitSize = 1;
+    }
+
+    long curVal = minVal;
+
+    while (curVal <= maxVal) {
+      splits.add(curVal);
+      curVal += splitSize;
+    }
+
+    if (splits.get(splits.size() - 1) != maxVal || splits.size() == 1) {
+      // We didn't end on the maxVal. Add that to the end of the list.
+      splits.add(maxVal);
+    }
+
+    return splits;
+  }
+}

+ 50 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/MySQLDBRecordReader.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.db;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A RecordReader that reads records from a MySQL table.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class MySQLDBRecordReader<T extends DBWritable> extends DBRecordReader<T> {
+
+  public MySQLDBRecordReader(DBInputFormat.DBInputSplit split, 
+      Class<T> inputClass, Configuration conf, Connection conn, DBConfiguration dbConfig,
+      String cond, String [] fields, String table) throws SQLException {
+    super(split, inputClass, conf, conn, dbConfig, cond, fields, table);
+  }
+
+  // Execute statements for mysql in unbuffered mode.
+  protected ResultSet executeQuery(String query) throws SQLException {
+    statement = getConnection().prepareStatement(query,
+      ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+    statement.setFetchSize(Integer.MIN_VALUE); // MySQL: read row-at-a-time.
+    return statement.executeQuery();
+  }
+}

+ 51 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/MySQLDataDrivenDBRecordReader.java

@@ -0,0 +1,51 @@
+/**
+ * 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.db;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A RecordReader that reads records from a MySQL table via DataDrivenDBRecordReader
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class MySQLDataDrivenDBRecordReader<T extends DBWritable>
+    extends DataDrivenDBRecordReader<T> {
+
+  public MySQLDataDrivenDBRecordReader(DBInputFormat.DBInputSplit split,
+      Class<T> inputClass, Configuration conf, Connection conn, DBConfiguration dbConfig,
+      String cond, String [] fields, String table) throws SQLException {
+    super(split, inputClass, conf, conn, dbConfig, cond, fields, table, "MYSQL");
+  }
+
+  // Execute statements for mysql in unbuffered mode.
+  protected ResultSet executeQuery(String query) throws SQLException {
+    statement = getConnection().prepareStatement(query,
+      ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+    statement.setFetchSize(Integer.MIN_VALUE); // MySQL: read row-at-a-time.
+    return statement.executeQuery();
+  }
+}

+ 145 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/OracleDBRecordReader.java

@@ -0,0 +1,145 @@
+/**
+ * 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.db;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.lang.reflect.Method;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * A RecordReader that reads records from an Oracle SQL table.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OracleDBRecordReader<T extends DBWritable> extends DBRecordReader<T> {
+
+  /** Configuration key to set to a timezone string. */
+  public static final String SESSION_TIMEZONE_KEY = "oracle.sessionTimeZone";
+
+  private static final Log LOG = LogFactory.getLog(OracleDBRecordReader.class);
+
+  public OracleDBRecordReader(DBInputFormat.DBInputSplit split, 
+      Class<T> inputClass, Configuration conf, Connection conn, DBConfiguration dbConfig,
+      String cond, String [] fields, String table) throws SQLException {
+    super(split, inputClass, conf, conn, dbConfig, cond, fields, table);
+    setSessionTimeZone(conf, conn);
+  }
+
+  /** Returns the query for selecting the records from an Oracle DB. */
+  protected String getSelectQuery() {
+    StringBuilder query = new StringBuilder();
+    DBConfiguration dbConf = getDBConf();
+    String conditions = getConditions();
+    String tableName = getTableName();
+    String [] fieldNames = getFieldNames();
+
+    // Oracle-specific codepath to use rownum instead of LIMIT/OFFSET.
+    if(dbConf.getInputQuery() == null) {
+      query.append("SELECT ");
+  
+      for (int i = 0; i < fieldNames.length; i++) {
+        query.append(fieldNames[i]);
+        if (i != fieldNames.length -1) {
+          query.append(", ");
+        }
+      }
+  
+      query.append(" FROM ").append(tableName);
+      if (conditions != null && conditions.length() > 0)
+        query.append(" WHERE ").append(conditions);
+      String orderBy = dbConf.getInputOrderBy();
+      if (orderBy != null && orderBy.length() > 0) {
+        query.append(" ORDER BY ").append(orderBy);
+      }
+    } else {
+      //PREBUILT QUERY
+      query.append(dbConf.getInputQuery());
+    }
+        
+    try {
+      DBInputFormat.DBInputSplit split = getSplit();
+      if (split.getLength() > 0 && split.getStart() > 0){
+        String querystring = query.toString();
+
+        query = new StringBuilder();
+        query.append("SELECT * FROM (SELECT a.*,ROWNUM dbif_rno FROM ( ");
+        query.append(querystring);
+        query.append(" ) a WHERE rownum <= ").append(split.getStart());
+        query.append(" + ").append(split.getLength());
+        query.append(" ) WHERE dbif_rno >= ").append(split.getStart());
+      }
+    } catch (IOException ex) {
+      // ignore, will not throw.
+    }		      
+
+    return query.toString();
+  }
+
+  /**
+   * Set session time zone
+   * @param conf The current configuration.
+   * We read the 'oracle.sessionTimeZone' property from here.
+   * @param conn The connection to alter the timezone properties of.
+   */
+  public static void setSessionTimeZone(Configuration conf,
+      Connection conn) throws SQLException {
+    // need to use reflection to call the method setSessionTimeZone on
+    // the OracleConnection class because oracle specific java libraries are
+    // not accessible in this context.
+    Method method;
+    try {
+      method = conn.getClass().getMethod(
+              "setSessionTimeZone", new Class [] {String.class});
+    } catch (Exception ex) {
+      LOG.error("Could not find method setSessionTimeZone in " + conn.getClass().getName(), ex);
+      // rethrow SQLException
+      throw new SQLException(ex);
+    }
+
+    // Need to set the time zone in order for Java
+    // to correctly access the column "TIMESTAMP WITH LOCAL TIME ZONE".
+    // We can't easily get the correct Oracle-specific timezone string
+    // from Java; just let the user set the timezone in a property.
+    String clientTimeZone = conf.get(SESSION_TIMEZONE_KEY, "GMT");
+    try {
+      method.setAccessible(true);
+      method.invoke(conn, clientTimeZone);
+      LOG.info("Time zone has been set to " + clientTimeZone);
+    } catch (Exception ex) {
+      LOG.warn("Time zone " + clientTimeZone +
+               " could not be set on Oracle database.");
+      LOG.warn("Setting default time zone: GMT");
+      try {
+        // "GMT" timezone is guaranteed to exist.
+        method.invoke(conn, "GMT");
+      } catch (Exception ex2) {
+        LOG.error("Could not set time zone for oracle connection", ex2);
+        // rethrow SQLException
+        throw new SQLException(ex);
+      }
+    }
+  }
+}

+ 93 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBInputFormat.java

@@ -0,0 +1,93 @@
+/**
+ * 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.db;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A InputFormat that reads input data from an SQL table in an Oracle db.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OracleDataDrivenDBInputFormat<T extends DBWritable>
+    extends DataDrivenDBInputFormat<T> implements Configurable {
+
+  /**
+   * @return the DBSplitter implementation to use to divide the table/query into InputSplits.
+   */
+  @Override
+  protected DBSplitter getSplitter(int sqlDataType) {
+    switch (sqlDataType) {
+    case Types.DATE:
+    case Types.TIME:
+    case Types.TIMESTAMP:
+      return new OracleDateSplitter();
+
+    default:
+      return super.getSplitter(sqlDataType);
+    }
+  }
+
+  @Override
+  protected RecordReader<LongWritable, T> createDBRecordReader(DBInputSplit split,
+      Configuration conf) throws IOException {
+
+    DBConfiguration dbConf = getDBConf();
+    @SuppressWarnings("unchecked")
+    Class<T> inputClass = (Class<T>) (dbConf.getInputClass());
+
+    try {
+      // Use Oracle-specific db reader
+      return new OracleDataDrivenDBRecordReader<T>(split, inputClass,
+          conf, getConnection(), dbConf, dbConf.getInputConditions(),
+          dbConf.getInputFieldNames(), dbConf.getInputTableName());
+    } catch (SQLException ex) {
+      throw new IOException(ex.getMessage());
+    }
+  }
+}

+ 49 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBRecordReader.java

@@ -0,0 +1,49 @@
+/**
+ * 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.db;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A RecordReader that reads records from a Oracle table via DataDrivenDBRecordReader
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OracleDataDrivenDBRecordReader<T extends DBWritable>
+    extends DataDrivenDBRecordReader<T> {
+
+  public OracleDataDrivenDBRecordReader(DBInputFormat.DBInputSplit split,
+      Class<T> inputClass, Configuration conf, Connection conn,
+      DBConfiguration dbConfig, String cond, String [] fields,
+      String table) throws SQLException {
+
+    super(split, inputClass, conf, conn, dbConfig, cond, fields, table,
+        "ORACLE");
+
+    // Must initialize the tz used by the connection for Oracle.
+    OracleDBRecordReader.setSessionTimeZone(conf, conn);
+  }
+}

+ 42 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/OracleDateSplitter.java

@@ -0,0 +1,42 @@
+/**
+ * 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.db;
+
+import java.util.Date;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Implement DBSplitter over date/time values returned by an Oracle db.
+ * Make use of logic from DateSplitter, since this just needs to use
+ * some Oracle-specific functions on the formatting end when generating
+ * InputSplits.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OracleDateSplitter extends DateSplitter {
+
+  @SuppressWarnings("unchecked")
+  @Override
+  protected String dateToString(Date d) {
+    // Oracle Data objects are always actually Timestamps
+    return "TO_TIMESTAMP('" + d.toString() + "', 'YYYY-MM-DD HH24:MI:SS.FF')";
+  }
+}

+ 220 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/db/TextSplitter.java

@@ -0,0 +1,220 @@
+/**
+ * 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.db;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+/**
+ * Implement DBSplitter over text strings.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class TextSplitter extends BigDecimalSplitter {
+
+  private static final Log LOG = LogFactory.getLog(TextSplitter.class);
+
+  /**
+   * This method needs to determine the splits between two user-provided strings.
+   * In the case where the user's strings are 'A' and 'Z', this is not hard; we 
+   * could create two splits from ['A', 'M') and ['M', 'Z'], 26 splits for strings
+   * beginning with each letter, etc.
+   *
+   * If a user has provided us with the strings "Ham" and "Haze", however, we need
+   * to create splits that differ in the third letter.
+   *
+   * The algorithm used is as follows:
+   * Since there are 2**16 unicode characters, we interpret characters as digits in
+   * base 65536. Given a string 's' containing characters s_0, s_1 .. s_n, we interpret
+   * the string as the number: 0.s_0 s_1 s_2.. s_n in base 65536. Having mapped the
+   * low and high strings into floating-point values, we then use the BigDecimalSplitter
+   * to establish the even split points, then map the resulting floating point values
+   * back into strings.
+   */
+  public List<InputSplit> split(Configuration conf, ResultSet results, String colName)
+      throws SQLException {
+
+    LOG.warn("Generating splits for a textual index column.");
+    LOG.warn("If your database sorts in a case-insensitive order, "
+        + "this may result in a partial import or duplicate records.");
+    LOG.warn("You are strongly encouraged to choose an integral split column.");
+
+    String minString = results.getString(1);
+    String maxString = results.getString(2);
+
+    boolean minIsNull = false;
+
+    // If the min value is null, switch it to an empty string instead for purposes
+    // of interpolation. Then add [null, null] as a special case split.
+    if (null == minString) {
+      minString = "";
+      minIsNull = true;
+    }
+
+    if (null == maxString) {
+      // If the max string is null, then the min string has to be null too.
+      // Just return a special split for this case.
+      List<InputSplit> splits = new ArrayList<InputSplit>();
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+      return splits;
+    }
+
+    // Use this as a hint. May need an extra task if the size doesn't
+    // divide cleanly.
+    int numSplits = conf.getInt("mapred.map.tasks", 1);
+
+    String lowClausePrefix = colName + " >= '";
+    String highClausePrefix = colName + " < '";
+
+    // If there is a common prefix between minString and maxString, establish it
+    // and pull it out of minString and maxString.
+    int maxPrefixLen = Math.min(minString.length(), maxString.length());
+    int sharedLen;
+    for (sharedLen = 0; sharedLen < maxPrefixLen; sharedLen++) {
+      char c1 = minString.charAt(sharedLen);
+      char c2 = maxString.charAt(sharedLen);
+      if (c1 != c2) {
+        break;
+      }
+    }
+
+    // The common prefix has length 'sharedLen'. Extract it from both.
+    String commonPrefix = minString.substring(0, sharedLen);
+    minString = minString.substring(sharedLen);
+    maxString = maxString.substring(sharedLen);
+
+    List<String> splitStrings = split(numSplits, minString, maxString, commonPrefix);
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+
+    // Convert the list of split point strings into an actual set of InputSplits.
+    String start = splitStrings.get(0);
+    for (int i = 1; i < splitStrings.size(); i++) {
+      String end = splitStrings.get(i);
+
+      if (i == splitStrings.size() - 1) {
+        // This is the last one; use a closed interval.
+        splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+            lowClausePrefix + start + "'", colName + " <= '" + end + "'"));
+      } else {
+        // Normal open-interval case.
+        splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+            lowClausePrefix + start + "'", highClausePrefix + end + "'"));
+      }
+    }
+
+    if (minIsNull) {
+      // Add the special null split at the end.
+      splits.add(new DataDrivenDBInputFormat.DataDrivenDBInputSplit(
+          colName + " IS NULL", colName + " IS NULL"));
+    }
+
+    return splits;
+  }
+
+  List<String> split(int numSplits, String minString, String maxString, String commonPrefix)
+      throws SQLException {
+
+    BigDecimal minVal = stringToBigDecimal(minString);
+    BigDecimal maxVal = stringToBigDecimal(maxString);
+
+    List<BigDecimal> splitPoints = split(new BigDecimal(numSplits), minVal, maxVal);
+    List<String> splitStrings = new ArrayList<String>();
+
+    // Convert the BigDecimal splitPoints into their string representations.
+    for (BigDecimal bd : splitPoints) {
+      splitStrings.add(commonPrefix + bigDecimalToString(bd));
+    }
+
+    // Make sure that our user-specified boundaries are the first and last entries
+    // in the array.
+    if (splitStrings.size() == 0 || !splitStrings.get(0).equals(commonPrefix + minString)) {
+      splitStrings.add(0, commonPrefix + minString);
+    }
+    if (splitStrings.size() == 1
+        || !splitStrings.get(splitStrings.size() - 1).equals(commonPrefix + maxString)) {
+      splitStrings.add(commonPrefix + maxString);
+    }
+
+    return splitStrings;
+  }
+
+  private final static BigDecimal ONE_PLACE = new BigDecimal(65536);
+
+  // Maximum number of characters to convert. This is to prevent rounding errors
+  // or repeating fractions near the very bottom from getting out of control. Note
+  // that this still gives us a huge number of possible splits.
+  private final static int MAX_CHARS = 8;
+
+  /**
+   * Return a BigDecimal representation of string 'str' suitable for use
+   * in a numerically-sorting order.
+   */
+  BigDecimal stringToBigDecimal(String str) {
+    BigDecimal result = BigDecimal.ZERO;
+    BigDecimal curPlace = ONE_PLACE; // start with 1/65536 to compute the first digit.
+
+    int len = Math.min(str.length(), MAX_CHARS);
+
+    for (int i = 0; i < len; i++) {
+      int codePoint = str.codePointAt(i);
+      result = result.add(tryDivide(new BigDecimal(codePoint), curPlace));
+      // advance to the next less significant place. e.g., 1/(65536^2) for the second char.
+      curPlace = curPlace.multiply(ONE_PLACE);
+    }
+
+    return result;
+  }
+
+  /**
+   * Return the string encoded in a BigDecimal.
+   * Repeatedly multiply the input value by 65536; the integer portion after such a multiplication
+   * represents a single character in base 65536. Convert that back into a char and create a
+   * string out of these until we have no data left.
+   */
+  String bigDecimalToString(BigDecimal bd) {
+    BigDecimal cur = bd.stripTrailingZeros();
+    StringBuilder sb = new StringBuilder();
+
+    for (int numConverted = 0; numConverted < MAX_CHARS; numConverted++) {
+      cur = cur.multiply(ONE_PLACE);
+      int curCodePoint = cur.intValue();
+      if (0 == curCodePoint) {
+        break;
+      }
+
+      cur = cur.subtract(new BigDecimal(curCodePoint));
+      sb.append(Character.toChars(curCodePoint));
+    }
+
+    return sb.toString();
+  }
+}

+ 227 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionHelper.java

@@ -0,0 +1,227 @@
+/**
+ * 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.fieldsel;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class implements a mapper/reducer class that can be used to perform
+ * field selections in a manner similar to unix cut. The input data is treated
+ * as fields separated by a user specified separator (the default value is
+ * "\t"). The user can specify a list of fields that form the map output keys,
+ * and a list of fields that form the map output values. If the inputformat is
+ * TextInputFormat, the mapper will ignore the key to the map function. and the
+ * fields are from the value only. Otherwise, the fields are the union of those
+ * from the key and those from the value.
+ * 
+ * The field separator is under attribute "mapreduce.fieldsel.data.field.separator"
+ * 
+ * The map output field list spec is under attribute 
+ * "mapreduce.fieldsel.map.output.key.value.fields.spec".
+ * The value is expected to be like "keyFieldsSpec:valueFieldsSpec"
+ * key/valueFieldsSpec are comma (,) separated field spec: fieldSpec,fieldSpec,fieldSpec ...
+ * Each field spec can be a simple number (e.g. 5) specifying a specific field, or a range
+ * (like 2-5) to specify a range of fields, or an open range (like 3-) specifying all 
+ * the fields starting from field 3. The open range field spec applies value fields only.
+ * They have no effect on the key fields.
+ * 
+ * Here is an example: "4,3,0,1:6,5,1-3,7-". It specifies to use fields 4,3,0 and 1 for keys,
+ * and use fields 6,5,1,2,3,7 and above for values.
+ * 
+ * The reduce output field list spec is under attribute 
+ * "mapreduce.fieldsel.reduce.output.key.value.fields.spec".
+ * 
+ * The reducer extracts output key/value pairs in a similar manner, except that
+ * the key is never ignored.
+ * 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class FieldSelectionHelper {
+
+  public static Text emptyText = new Text("");
+  public static final String DATA_FIELD_SEPERATOR = 
+    "mapreduce.fieldsel.data.field.separator";
+  public static final String MAP_OUTPUT_KEY_VALUE_SPEC = 
+    "mapreduce.fieldsel.map.output.key.value.fields.spec";
+  public static final String REDUCE_OUTPUT_KEY_VALUE_SPEC = 
+    "mapreduce.fieldsel.reduce.output.key.value.fields.spec";
+
+
+  /**
+   * Extract the actual field numbers from the given field specs.
+   * If a field spec is in the form of "n-" (like 3-), then n will be the 
+   * return value. Otherwise, -1 will be returned.  
+   * @param fieldListSpec an array of field specs
+   * @param fieldList an array of field numbers extracted from the specs.
+   * @return number n if some field spec is in the form of "n-", -1 otherwise.
+   */
+  private static int extractFields(String[] fieldListSpec,
+      List<Integer> fieldList) {
+    int allFieldsFrom = -1;
+    int i = 0;
+    int j = 0;
+    int pos = -1;
+    String fieldSpec = null;
+    for (i = 0; i < fieldListSpec.length; i++) {
+      fieldSpec = fieldListSpec[i];
+      if (fieldSpec.length() == 0) {
+        continue;
+      }
+      pos = fieldSpec.indexOf('-');
+      if (pos < 0) {
+        Integer fn = new Integer(fieldSpec);
+        fieldList.add(fn);
+      } else {
+        String start = fieldSpec.substring(0, pos);
+        String end = fieldSpec.substring(pos + 1);
+        if (start.length() == 0) {
+          start = "0";
+        }
+        if (end.length() == 0) {
+          allFieldsFrom = Integer.parseInt(start);
+          continue;
+        }
+        int startPos = Integer.parseInt(start);
+        int endPos = Integer.parseInt(end);
+        for (j = startPos; j <= endPos; j++) {
+          fieldList.add(j);
+        }
+      }
+    }
+    return allFieldsFrom;
+  }
+
+  private static String selectFields(String[] fields, List<Integer> fieldList,
+      int allFieldsFrom, String separator) {
+    String retv = null;
+    int i = 0;
+    StringBuffer sb = null;
+    if (fieldList != null && fieldList.size() > 0) {
+      if (sb == null) {
+        sb = new StringBuffer();
+      }
+      for (Integer index : fieldList) {
+        if (index < fields.length) {
+          sb.append(fields[index]);
+        }
+        sb.append(separator);
+      }
+    }
+    if (allFieldsFrom >= 0) {
+      if (sb == null) {
+        sb = new StringBuffer();
+      }
+      for (i = allFieldsFrom; i < fields.length; i++) {
+        sb.append(fields[i]).append(separator);
+      }
+    }
+    if (sb != null) {
+      retv = sb.toString();
+      if (retv.length() > 0) {
+        retv = retv.substring(0, retv.length() - 1);
+      }
+    }
+    return retv;
+  }
+  
+  public static int parseOutputKeyValueSpec(String keyValueSpec,
+      List<Integer> keyFieldList, List<Integer> valueFieldList) {
+    String[] keyValSpecs = keyValueSpec.split(":", -1);
+    
+    String[] keySpec = keyValSpecs[0].split(",");
+    
+    String[] valSpec = new String[0];
+    if (keyValSpecs.length > 1) {
+      valSpec = keyValSpecs[1].split(",");
+    }
+
+    FieldSelectionHelper.extractFields(keySpec, keyFieldList);
+    return FieldSelectionHelper.extractFields(valSpec, valueFieldList);
+  }
+
+  public static String specToString(String fieldSeparator, String keyValueSpec,
+      int allValueFieldsFrom, List<Integer> keyFieldList,
+      List<Integer> valueFieldList) {
+    StringBuffer sb = new StringBuffer();
+    sb.append("fieldSeparator: ").append(fieldSeparator).append("\n");
+
+    sb.append("keyValueSpec: ").append(keyValueSpec).append("\n");
+    sb.append("allValueFieldsFrom: ").append(allValueFieldsFrom);
+    sb.append("\n");
+    sb.append("keyFieldList.length: ").append(keyFieldList.size());
+    sb.append("\n");
+    for (Integer field : keyFieldList) {
+      sb.append("\t").append(field).append("\n");
+    }
+    sb.append("valueFieldList.length: ").append(valueFieldList.size());
+    sb.append("\n");
+    for (Integer field : valueFieldList) {
+      sb.append("\t").append(field).append("\n");
+    }
+    return sb.toString();
+  }
+
+  private Text key = null;
+  private Text value = null;
+  
+  public FieldSelectionHelper() {
+  }
+
+  public FieldSelectionHelper(Text key, Text val) {
+    this.key = key;
+    this.value = val;
+  }
+  
+  public Text getKey() {
+    return key;
+  }
+ 
+  public Text getValue() {
+    return value;
+  }
+
+  public void extractOutputKeyValue(String key, String val,
+      String fieldSep, List<Integer> keyFieldList, List<Integer> valFieldList,
+      int allValueFieldsFrom, boolean ignoreKey, boolean isMap) {
+    if (!ignoreKey) {
+      val = key + val;
+    }
+    String[] fields = val.split(fieldSep);
+    
+    String newKey = selectFields(fields, keyFieldList, -1, fieldSep);
+    String newVal = selectFields(fields, valFieldList, allValueFieldsFrom,
+      fieldSep);
+    if (isMap && newKey == null) {
+      newKey = newVal;
+      newVal = null;
+    }
+    
+    if (newKey != null) {
+      this.key = new Text(newKey);
+    }
+    if (newVal != null) {
+      this.value = new Text(newVal);
+    }
+  }
+}

+ 110 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionMapper.java

@@ -0,0 +1,110 @@
+/**
+ * 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.fieldsel;
+
+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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+
+/**
+ * This class implements a mapper class that can be used to perform
+ * field selections in a manner similar to unix cut. The input data is treated
+ * as fields separated by a user specified separator (the default value is
+ * "\t"). The user can specify a list of fields that form the map output keys,
+ * and a list of fields that form the map output values. If the inputformat is
+ * TextInputFormat, the mapper will ignore the key to the map function. and the
+ * fields are from the value only. Otherwise, the fields are the union of those
+ * from the key and those from the value.
+ * 
+ * The field separator is under attribute "mapreduce.fieldsel.data.field.separator"
+ * 
+ * The map output field list spec is under attribute 
+ * "mapreduce.fieldsel.map.output.key.value.fields.spec". 
+ * The value is expected to be like
+ * "keyFieldsSpec:valueFieldsSpec" key/valueFieldsSpec are comma (,) separated
+ * field spec: fieldSpec,fieldSpec,fieldSpec ... Each field spec can be a 
+ * simple number (e.g. 5) specifying a specific field, or a range (like 2-5)
+ * to specify a range of fields, or an open range (like 3-) specifying all 
+ * the fields starting from field 3. The open range field spec applies value
+ * fields only. They have no effect on the key fields.
+ * 
+ * Here is an example: "4,3,0,1:6,5,1-3,7-". It specifies to use fields
+ * 4,3,0 and 1 for keys, and use fields 6,5,1,2,3,7 and above for values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class FieldSelectionMapper<K, V>
+    extends Mapper<K, V, Text, Text> {
+
+  private String mapOutputKeyValueSpec;
+
+  private boolean ignoreInputKey;
+
+  private String fieldSeparator = "\t";
+
+  private List<Integer> mapOutputKeyFieldList = new ArrayList<Integer>();
+
+  private List<Integer> mapOutputValueFieldList = new ArrayList<Integer>();
+
+  private int allMapValueFieldsFrom = -1;
+
+  public static final Log LOG = LogFactory.getLog("FieldSelectionMapReduce");
+
+  public void setup(Context context) 
+      throws IOException, InterruptedException {
+    Configuration conf = context.getConfiguration();
+    this.fieldSeparator = 
+      conf.get(FieldSelectionHelper.DATA_FIELD_SEPERATOR, "\t");
+    this.mapOutputKeyValueSpec = 
+      conf.get(FieldSelectionHelper.MAP_OUTPUT_KEY_VALUE_SPEC, "0-:");
+    try {
+      this.ignoreInputKey = TextInputFormat.class.getCanonicalName().equals(
+        context.getInputFormatClass().getCanonicalName());
+    } catch (ClassNotFoundException e) {
+      throw new IOException("Input format class not found", e);
+    }
+    allMapValueFieldsFrom = FieldSelectionHelper.parseOutputKeyValueSpec(
+      mapOutputKeyValueSpec, mapOutputKeyFieldList, mapOutputValueFieldList);
+    LOG.info(FieldSelectionHelper.specToString(fieldSeparator,
+      mapOutputKeyValueSpec, allMapValueFieldsFrom, mapOutputKeyFieldList,
+      mapOutputValueFieldList) + "\nignoreInputKey:" + ignoreInputKey);
+  }
+
+  /**
+   * The identify function. Input key/value pair is written directly to output.
+   */
+  public void map(K key, V val, Context context) 
+      throws IOException, InterruptedException {
+    FieldSelectionHelper helper = new FieldSelectionHelper(
+      FieldSelectionHelper.emptyText, FieldSelectionHelper.emptyText);
+    helper.extractOutputKeyValue(key.toString(), val.toString(),
+      fieldSeparator, mapOutputKeyFieldList, mapOutputValueFieldList,
+      allMapValueFieldsFrom, ignoreInputKey, true);
+    context.write(helper.getKey(), helper.getValue());
+  }
+}

+ 106 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionReducer.java

@@ -0,0 +1,106 @@
+/**
+ * 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.fieldsel;
+
+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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Reducer;
+
+/**
+ * This class implements a reducer class that can be used to perform field
+ * selections in a manner similar to unix cut. 
+ * 
+ * The input data is treated as fields separated by a user specified
+ * separator (the default value is "\t"). The user can specify a list of
+ * fields that form the reduce output keys, and a list of fields that form
+ * the reduce output values. The fields are the union of those from the key
+ * and those from the value.
+ * 
+ * The field separator is under attribute "mapreduce.fieldsel.data.field.separator"
+ * 
+ * The reduce output field list spec is under attribute 
+ * "mapreduce.fieldsel.reduce.output.key.value.fields.spec". 
+ * The value is expected to be like
+ * "keyFieldsSpec:valueFieldsSpec" key/valueFieldsSpec are comma (,) 
+ * separated field spec: fieldSpec,fieldSpec,fieldSpec ... Each field spec
+ * can be a simple number (e.g. 5) specifying a specific field, or a range
+ * (like 2-5) to specify a range of fields, or an open range (like 3-) 
+ * specifying all the fields starting from field 3. The open range field
+ * spec applies value fields only. They have no effect on the key fields.
+ * 
+ * Here is an example: "4,3,0,1:6,5,1-3,7-". It specifies to use fields
+ * 4,3,0 and 1 for keys, and use fields 6,5,1,2,3,7 and above for values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class FieldSelectionReducer<K, V>
+    extends Reducer<Text, Text, Text, Text> {
+
+  private String fieldSeparator = "\t";
+
+  private String reduceOutputKeyValueSpec;
+
+  private List<Integer> reduceOutputKeyFieldList = new ArrayList<Integer>();
+
+  private List<Integer> reduceOutputValueFieldList = new ArrayList<Integer>();
+
+  private int allReduceValueFieldsFrom = -1;
+
+  public static final Log LOG = LogFactory.getLog("FieldSelectionMapReduce");
+
+  public void setup(Context context) 
+      throws IOException, InterruptedException {
+    Configuration conf = context.getConfiguration();
+    
+    this.fieldSeparator = 
+      conf.get(FieldSelectionHelper.DATA_FIELD_SEPERATOR, "\t");
+    
+    this.reduceOutputKeyValueSpec = 
+      conf.get(FieldSelectionHelper.REDUCE_OUTPUT_KEY_VALUE_SPEC, "0-:");
+    
+    allReduceValueFieldsFrom = FieldSelectionHelper.parseOutputKeyValueSpec(
+      reduceOutputKeyValueSpec, reduceOutputKeyFieldList,
+      reduceOutputValueFieldList);
+
+    LOG.info(FieldSelectionHelper.specToString(fieldSeparator,
+      reduceOutputKeyValueSpec, allReduceValueFieldsFrom,
+      reduceOutputKeyFieldList, reduceOutputValueFieldList));
+  }
+
+  public void reduce(Text key, Iterable<Text> values, Context context)
+      throws IOException, InterruptedException {
+    String keyStr = key.toString() + this.fieldSeparator;
+    
+    for (Text val : values) {
+      FieldSelectionHelper helper = new FieldSelectionHelper();
+      helper.extractOutputKeyValue(keyStr, val.toString(),
+        fieldSeparator, reduceOutputKeyFieldList,
+        reduceOutputValueFieldList, allReduceValueFieldsFrom, false, false);
+      context.write(helper.getKey(), helper.getValue());
+    }
+  }
+}

+ 693 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java

@@ -0,0 +1,693 @@
+/**
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Set;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.PathFilter;
+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;
+import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.net.NetworkTopology;
+
+/**
+ * An abstract {@link InputFormat} that returns {@link CombineFileSplit}'s in 
+ * {@link InputFormat#getSplits(JobContext)} method. 
+ * 
+ * Splits are constructed from the files under the input paths. 
+ * A split cannot have files from different pools.
+ * Each split returned may contain blocks from different files.
+ * If a maxSplitSize is specified, then blocks on the same node are
+ * combined to form a single split. Blocks that are left over are
+ * then combined with other blocks in the same rack. 
+ * If maxSplitSize is not specified, then blocks from the same rack
+ * are combined in a single split; no attempt is made to create
+ * node-local splits.
+ * If the maxSplitSize is equal to the block size, then this class
+ * is similar to the default splitting behavior in Hadoop: each
+ * block is a locally processed split.
+ * Subclasses implement 
+ * {@link InputFormat#createRecordReader(InputSplit, TaskAttemptContext)}
+ * to construct <code>RecordReader</code>'s for 
+ * <code>CombineFileSplit</code>'s.
+ * 
+ * @see CombineFileSplit
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class CombineFileInputFormat<K, V>
+  extends FileInputFormat<K, V> {
+
+  public static final String SPLIT_MINSIZE_PERNODE = 
+    "mapreduce.input.fileinputformat.split.minsize.per.node";
+  public static final String SPLIT_MINSIZE_PERRACK = 
+    "mapreduce.input.fileinputformat.split.minsize.per.rack";
+  // ability to limit the size of a single split
+  private long maxSplitSize = 0;
+  private long minSplitSizeNode = 0;
+  private long minSplitSizeRack = 0;
+
+  // A pool of input paths filters. A split cannot have blocks from files
+  // across multiple pools.
+  private ArrayList<MultiPathFilter> pools = new  ArrayList<MultiPathFilter>();
+
+  // mapping from a rack name to the set of Nodes in the rack 
+  private HashMap<String, Set<String>> rackToNodes = 
+                            new HashMap<String, Set<String>>();
+  /**
+   * Specify the maximum size (in bytes) of each split. Each split is
+   * approximately equal to the specified size.
+   */
+  protected void setMaxSplitSize(long maxSplitSize) {
+    this.maxSplitSize = maxSplitSize;
+  }
+
+  /**
+   * Specify the minimum size (in bytes) of each split per node.
+   * This applies to data that is left over after combining data on a single
+   * node into splits that are of maximum size specified by maxSplitSize.
+   * This leftover data will be combined into its own split if its size
+   * exceeds minSplitSizeNode.
+   */
+  protected void setMinSplitSizeNode(long minSplitSizeNode) {
+    this.minSplitSizeNode = minSplitSizeNode;
+  }
+
+  /**
+   * Specify the minimum size (in bytes) of each split per rack.
+   * This applies to data that is left over after combining data on a single
+   * rack into splits that are of maximum size specified by maxSplitSize.
+   * This leftover data will be combined into its own split if its size
+   * exceeds minSplitSizeRack.
+   */
+  protected void setMinSplitSizeRack(long minSplitSizeRack) {
+    this.minSplitSizeRack = minSplitSizeRack;
+  }
+
+  /**
+   * Create a new pool and add the filters to it.
+   * A split cannot have files from different pools.
+   */
+  protected void createPool(List<PathFilter> filters) {
+    pools.add(new MultiPathFilter(filters));
+  }
+
+  /**
+   * Create a new pool and add the filters to it. 
+   * A pathname can satisfy any one of the specified filters.
+   * A split cannot have files from different pools.
+   */
+  protected void createPool(PathFilter... filters) {
+    MultiPathFilter multi = new MultiPathFilter();
+    for (PathFilter f: filters) {
+      multi.add(f);
+    }
+    pools.add(multi);
+  }
+  
+  @Override
+  protected boolean isSplitable(JobContext context, Path file) {
+    final CompressionCodec codec =
+      new CompressionCodecFactory(context.getConfiguration()).getCodec(file);
+    return codec == null;
+  }
+
+  /**
+   * default constructor
+   */
+  public CombineFileInputFormat() {
+  }
+
+  @Override
+  public List<InputSplit> getSplits(JobContext job) 
+    throws IOException {
+
+    long minSizeNode = 0;
+    long minSizeRack = 0;
+    long maxSize = 0;
+    Configuration conf = job.getConfiguration();
+
+    // the values specified by setxxxSplitSize() takes precedence over the
+    // values that might have been specified in the config
+    if (minSplitSizeNode != 0) {
+      minSizeNode = minSplitSizeNode;
+    } else {
+      minSizeNode = conf.getLong(SPLIT_MINSIZE_PERNODE, 0);
+    }
+    if (minSplitSizeRack != 0) {
+      minSizeRack = minSplitSizeRack;
+    } else {
+      minSizeRack = conf.getLong(SPLIT_MINSIZE_PERRACK, 0);
+    }
+    if (maxSplitSize != 0) {
+      maxSize = maxSplitSize;
+    } else {
+      maxSize = conf.getLong("mapreduce.input.fileinputformat.split.maxsize", 0);
+    }
+    if (minSizeNode != 0 && maxSize != 0 && minSizeNode > maxSize) {
+      throw new IOException("Minimum split size pernode " + minSizeNode +
+                            " cannot be larger than maximum split size " +
+                            maxSize);
+    }
+    if (minSizeRack != 0 && maxSize != 0 && minSizeRack > maxSize) {
+      throw new IOException("Minimum split size per rack" + minSizeRack +
+                            " cannot be larger than maximum split size " +
+                            maxSize);
+    }
+    if (minSizeRack != 0 && minSizeNode > minSizeRack) {
+      throw new IOException("Minimum split size per node" + minSizeNode +
+                            " cannot be smaller than minimum split " +
+                            "size per rack " + minSizeRack);
+    }
+
+    // all the files in input set
+    Path[] paths = FileUtil.stat2Paths(
+                     listStatus(job).toArray(new FileStatus[0]));
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+    if (paths.length == 0) {
+      return splits;    
+    }
+
+    // Convert them to Paths first. This is a costly operation and 
+    // we should do it first, otherwise we will incur doing it multiple
+    // times, one time each for each pool in the next loop.
+    List<Path> newpaths = new LinkedList<Path>();
+    for (int i = 0; i < paths.length; i++) {
+      Path p = new Path(paths[i].toUri().getPath());
+      newpaths.add(p);
+    }
+    paths = null;
+
+    // In one single iteration, process all the paths in a single pool.
+    // Processing one pool at a time ensures that a split contains paths
+    // from a single pool only.
+    for (MultiPathFilter onepool : pools) {
+      ArrayList<Path> myPaths = new ArrayList<Path>();
+      
+      // pick one input path. If it matches all the filters in a pool,
+      // add it to the output set
+      for (Iterator<Path> iter = newpaths.iterator(); iter.hasNext();) {
+        Path p = iter.next();
+        if (onepool.accept(p)) {
+          myPaths.add(p); // add it to my output set
+          iter.remove();
+        }
+      }
+      // create splits for all files in this pool.
+      getMoreSplits(job, myPaths.toArray(new Path[myPaths.size()]), 
+                    maxSize, minSizeNode, minSizeRack, splits);
+    }
+
+    // create splits for all files that are not in any pool.
+    getMoreSplits(job, newpaths.toArray(new Path[newpaths.size()]), 
+                  maxSize, minSizeNode, minSizeRack, splits);
+
+    // free up rackToNodes map
+    rackToNodes.clear();
+    return splits;    
+  }
+
+  /**
+   * Return all the splits in the specified set of paths
+   */
+  private void getMoreSplits(JobContext job, Path[] paths, 
+                             long maxSize, long minSizeNode, long minSizeRack,
+                             List<InputSplit> splits)
+    throws IOException {
+    Configuration conf = job.getConfiguration();
+
+    // all blocks for all the files in input set
+    OneFileInfo[] files;
+  
+    // mapping from a rack name to the list of blocks it has
+    HashMap<String, List<OneBlockInfo>> rackToBlocks = 
+                              new HashMap<String, List<OneBlockInfo>>();
+
+    // mapping from a block to the nodes on which it has replicas
+    HashMap<OneBlockInfo, String[]> blockToNodes = 
+                              new HashMap<OneBlockInfo, String[]>();
+
+    // mapping from a node to the list of blocks that it contains
+    HashMap<String, List<OneBlockInfo>> nodeToBlocks = 
+                              new HashMap<String, List<OneBlockInfo>>();
+    
+    files = new OneFileInfo[paths.length];
+    if (paths.length == 0) {
+      return; 
+    }
+
+    // populate all the blocks for all files
+    long totLength = 0;
+    for (int i = 0; i < paths.length; i++) {
+      files[i] = new OneFileInfo(paths[i], conf, isSplitable(job, paths[i]),
+                                 rackToBlocks, blockToNodes, nodeToBlocks,
+                                 rackToNodes, maxSize);
+      totLength += files[i].getLength();
+    }
+
+    ArrayList<OneBlockInfo> validBlocks = new ArrayList<OneBlockInfo>();
+    Set<String> nodes = new HashSet<String>();
+    long curSplitSize = 0;
+
+    // process all nodes and create splits that are local
+    // to a node. 
+    for (Iterator<Map.Entry<String, 
+         List<OneBlockInfo>>> iter = nodeToBlocks.entrySet().iterator(); 
+         iter.hasNext();) {
+
+      Map.Entry<String, List<OneBlockInfo>> one = iter.next();
+      nodes.add(one.getKey());
+      List<OneBlockInfo> blocksInNode = one.getValue();
+
+      // for each block, copy it into validBlocks. Delete it from 
+      // blockToNodes so that the same block does not appear in 
+      // two different splits.
+      for (OneBlockInfo oneblock : blocksInNode) {
+        if (blockToNodes.containsKey(oneblock)) {
+          validBlocks.add(oneblock);
+          blockToNodes.remove(oneblock);
+          curSplitSize += oneblock.length;
+
+          // if the accumulated split size exceeds the maximum, then 
+          // create this split.
+          if (maxSize != 0 && curSplitSize >= maxSize) {
+            // create an input split and add it to the splits array
+            addCreatedSplit(splits, nodes, validBlocks);
+            curSplitSize = 0;
+            validBlocks.clear();
+          }
+        }
+      }
+      // if there were any blocks left over and their combined size is
+      // larger than minSplitNode, then combine them into one split.
+      // Otherwise add them back to the unprocessed pool. It is likely 
+      // that they will be combined with other blocks from the 
+      // same rack later on.
+      if (minSizeNode != 0 && curSplitSize >= minSizeNode) {
+        // create an input split and add it to the splits array
+        addCreatedSplit(splits, nodes, validBlocks);
+      } else {
+        for (OneBlockInfo oneblock : validBlocks) {
+          blockToNodes.put(oneblock, oneblock.hosts);
+        }
+      }
+      validBlocks.clear();
+      nodes.clear();
+      curSplitSize = 0;
+    }
+
+    // if blocks in a rack are below the specified minimum size, then keep them
+    // in 'overflow'. After the processing of all racks is complete, these 
+    // overflow blocks will be combined into splits.
+    ArrayList<OneBlockInfo> overflowBlocks = new ArrayList<OneBlockInfo>();
+    Set<String> racks = new HashSet<String>();
+
+    // Process all racks over and over again until there is no more work to do.
+    while (blockToNodes.size() > 0) {
+
+      // Create one split for this rack before moving over to the next rack. 
+      // Come back to this rack after creating a single split for each of the 
+      // remaining racks.
+      // Process one rack location at a time, Combine all possible blocks that
+      // reside on this rack as one split. (constrained by minimum and maximum
+      // split size).
+
+      // iterate over all racks 
+      for (Iterator<Map.Entry<String, List<OneBlockInfo>>> iter = 
+           rackToBlocks.entrySet().iterator(); iter.hasNext();) {
+
+        Map.Entry<String, List<OneBlockInfo>> one = iter.next();
+        racks.add(one.getKey());
+        List<OneBlockInfo> blocks = one.getValue();
+
+        // for each block, copy it into validBlocks. Delete it from 
+        // blockToNodes so that the same block does not appear in 
+        // two different splits.
+        boolean createdSplit = false;
+        for (OneBlockInfo oneblock : blocks) {
+          if (blockToNodes.containsKey(oneblock)) {
+            validBlocks.add(oneblock);
+            blockToNodes.remove(oneblock);
+            curSplitSize += oneblock.length;
+      
+            // if the accumulated split size exceeds the maximum, then 
+            // create this split.
+            if (maxSize != 0 && curSplitSize >= maxSize) {
+              // create an input split and add it to the splits array
+              addCreatedSplit(splits, getHosts(racks), validBlocks);
+              createdSplit = true;
+              break;
+            }
+          }
+        }
+
+        // if we created a split, then just go to the next rack
+        if (createdSplit) {
+          curSplitSize = 0;
+          validBlocks.clear();
+          racks.clear();
+          continue;
+        }
+
+        if (!validBlocks.isEmpty()) {
+          if (minSizeRack != 0 && curSplitSize >= minSizeRack) {
+            // if there is a minimum size specified, then create a single split
+            // otherwise, store these blocks into overflow data structure
+            addCreatedSplit(splits, getHosts(racks), validBlocks);
+          } else {
+            // There were a few blocks in this rack that 
+        	// remained to be processed. Keep them in 'overflow' block list. 
+        	// These will be combined later.
+            overflowBlocks.addAll(validBlocks);
+          }
+        }
+        curSplitSize = 0;
+        validBlocks.clear();
+        racks.clear();
+      }
+    }
+
+    assert blockToNodes.isEmpty();
+    assert curSplitSize == 0;
+    assert validBlocks.isEmpty();
+    assert racks.isEmpty();
+
+    // Process all overflow blocks
+    for (OneBlockInfo oneblock : overflowBlocks) {
+      validBlocks.add(oneblock);
+      curSplitSize += oneblock.length;
+
+      // This might cause an exiting rack location to be re-added,
+      // but it should be ok.
+      for (int i = 0; i < oneblock.racks.length; i++) {
+        racks.add(oneblock.racks[i]);
+      }
+
+      // if the accumulated split size exceeds the maximum, then 
+      // create this split.
+      if (maxSize != 0 && curSplitSize >= maxSize) {
+        // create an input split and add it to the splits array
+        addCreatedSplit(splits, getHosts(racks), validBlocks);
+        curSplitSize = 0;
+        validBlocks.clear();
+        racks.clear();
+      }
+    }
+
+    // Process any remaining blocks, if any.
+    if (!validBlocks.isEmpty()) {
+      addCreatedSplit(splits, getHosts(racks), validBlocks);
+    }
+  }
+
+  /**
+   * Create a single split from the list of blocks specified in validBlocks
+   * Add this new split into splitList.
+   */
+  private void addCreatedSplit(List<InputSplit> splitList, 
+                               Collection<String> locations, 
+                               ArrayList<OneBlockInfo> validBlocks) {
+    // create an input split
+    Path[] fl = new Path[validBlocks.size()];
+    long[] offset = new long[validBlocks.size()];
+    long[] length = new long[validBlocks.size()];
+    for (int i = 0; i < validBlocks.size(); i++) {
+      fl[i] = validBlocks.get(i).onepath; 
+      offset[i] = validBlocks.get(i).offset;
+      length[i] = validBlocks.get(i).length;
+    }
+
+     // add this split to the list that is returned
+    CombineFileSplit thissplit = new CombineFileSplit(fl, offset, 
+                                   length, locations.toArray(new String[0]));
+    splitList.add(thissplit); 
+  }
+
+  /**
+   * This is not implemented yet. 
+   */
+  public abstract RecordReader<K, V> createRecordReader(InputSplit split,
+      TaskAttemptContext context) throws IOException;
+
+  /**
+   * information about one file from the File System
+   */
+  private static class OneFileInfo {
+    private long fileSize;               // size of the file
+    private OneBlockInfo[] blocks;       // all blocks in this file
+
+    OneFileInfo(Path path, Configuration conf,
+                boolean isSplitable,
+                HashMap<String, List<OneBlockInfo>> rackToBlocks,
+                HashMap<OneBlockInfo, String[]> blockToNodes,
+                HashMap<String, List<OneBlockInfo>> nodeToBlocks,
+                HashMap<String, Set<String>> rackToNodes,
+                long maxSize)
+                throws IOException {
+      this.fileSize = 0;
+
+      // get block locations from file system
+      FileSystem fs = path.getFileSystem(conf);
+      FileStatus stat = fs.getFileStatus(path);
+      BlockLocation[] locations = fs.getFileBlockLocations(stat, 0, 
+                                                           stat.getLen());
+      // create a list of all block and their locations
+      if (locations == null) {
+        blocks = new OneBlockInfo[0];
+      } else {
+        if (!isSplitable) {
+          // if the file is not splitable, just create the one block with
+          // full file length
+          blocks = new OneBlockInfo[1];
+          fileSize = stat.getLen();
+          blocks[0] = new OneBlockInfo(path, 0, fileSize, locations[0]
+              .getHosts(), locations[0].getTopologyPaths());
+        } else {
+          ArrayList<OneBlockInfo> blocksList = new ArrayList<OneBlockInfo>(
+              locations.length);
+          for (int i = 0; i < locations.length; i++) {
+            fileSize += locations[i].getLength();
+
+            // each split can be a maximum of maxSize
+            long left = locations[i].getLength();
+            long myOffset = locations[i].getOffset();
+            long myLength = 0;
+            while (left > 0) {
+              if (maxSize == 0) {
+                myLength = left;
+              } else {
+                if (left > maxSize && left < 2 * maxSize) {
+                  // if remainder is between max and 2*max - then
+                  // instead of creating splits of size max, left-max we
+                  // create splits of size left/2 and left/2. This is
+                  // a heuristic to avoid creating really really small
+                  // splits.
+                  myLength = left / 2;
+                } else {
+                  myLength = Math.min(maxSize, left);
+                }
+              }
+              OneBlockInfo oneblock = new OneBlockInfo(path, myOffset,
+                  myLength, locations[i].getHosts(), locations[i]
+                      .getTopologyPaths());
+              left -= myLength;
+              myOffset += myLength;
+
+              blocksList.add(oneblock);
+            }
+          }
+          blocks = blocksList.toArray(new OneBlockInfo[blocksList.size()]);
+        }
+
+        for (OneBlockInfo oneblock : blocks) {
+          // add this block to the block --> node locations map
+          blockToNodes.put(oneblock, oneblock.hosts);
+
+          // For blocks that do not have host/rack information,
+          // assign to default  rack.
+          String[] racks = null;
+          if (oneblock.hosts.length == 0) {
+            racks = new String[]{NetworkTopology.DEFAULT_RACK};
+          } else {
+            racks = oneblock.racks;
+          }
+
+          // add this block to the rack --> block map
+          for (int j = 0; j < racks.length; j++) {
+            String rack = racks[j];
+            List<OneBlockInfo> blklist = rackToBlocks.get(rack);
+            if (blklist == null) {
+              blklist = new ArrayList<OneBlockInfo>();
+              rackToBlocks.put(rack, blklist);
+            }
+            blklist.add(oneblock);
+            if (!racks[j].equals(NetworkTopology.DEFAULT_RACK)) {
+              // Add this host to rackToNodes map
+              addHostToRack(rackToNodes, racks[j], oneblock.hosts[j]);
+            }
+          }
+
+          // add this block to the node --> block map
+          for (int j = 0; j < oneblock.hosts.length; j++) {
+            String node = oneblock.hosts[j];
+            List<OneBlockInfo> blklist = nodeToBlocks.get(node);
+            if (blklist == null) {
+              blklist = new ArrayList<OneBlockInfo>();
+              nodeToBlocks.put(node, blklist);
+            }
+            blklist.add(oneblock);
+          }
+        }
+      }
+    }
+
+    long getLength() {
+      return fileSize;
+    }
+
+    OneBlockInfo[] getBlocks() {
+      return blocks;
+    }
+  }
+
+  /**
+   * information about one block from the File System
+   */
+  private static class OneBlockInfo {
+    Path onepath;                // name of this file
+    long offset;                 // offset in file
+    long length;                 // length of this block
+    String[] hosts;              // nodes on which this block resides
+    String[] racks;              // network topology of hosts
+
+    OneBlockInfo(Path path, long offset, long len, 
+                 String[] hosts, String[] topologyPaths) {
+      this.onepath = path;
+      this.offset = offset;
+      this.hosts = hosts;
+      this.length = len;
+      assert (hosts.length == topologyPaths.length ||
+              topologyPaths.length == 0);
+
+      // if the file system does not have any rack information, then
+      // use dummy rack location.
+      if (topologyPaths.length == 0) {
+        topologyPaths = new String[hosts.length];
+        for (int i = 0; i < topologyPaths.length; i++) {
+          topologyPaths[i] = (new NodeBase(hosts[i], 
+                              NetworkTopology.DEFAULT_RACK)).toString();
+        }
+      }
+
+      // The topology paths have the host name included as the last 
+      // component. Strip it.
+      this.racks = new String[topologyPaths.length];
+      for (int i = 0; i < topologyPaths.length; i++) {
+        this.racks[i] = (new NodeBase(topologyPaths[i])).getNetworkLocation();
+      }
+    }
+  }
+
+  protected BlockLocation[] getFileBlockLocations(
+    FileSystem fs, FileStatus stat) throws IOException {
+    return fs.getFileBlockLocations(stat, 0, stat.getLen());
+  }
+
+  private static void addHostToRack(HashMap<String, Set<String>> rackToNodes,
+                                    String rack, String host) {
+    Set<String> hosts = rackToNodes.get(rack);
+    if (hosts == null) {
+      hosts = new HashSet<String>();
+      rackToNodes.put(rack, hosts);
+    }
+    hosts.add(host);
+  }
+  
+  private Set<String> getHosts(Set<String> racks) {
+    Set<String> hosts = new HashSet<String>();
+    for (String rack : racks) {
+      if (rackToNodes.containsKey(rack)) {
+        hosts.addAll(rackToNodes.get(rack));
+      }
+    }
+    return hosts;
+  }
+  
+  /**
+   * Accept a path only if any one of filters given in the
+   * constructor do. 
+   */
+  private static class MultiPathFilter implements PathFilter {
+    private List<PathFilter> filters;
+
+    public MultiPathFilter() {
+      this.filters = new ArrayList<PathFilter>();
+    }
+
+    public MultiPathFilter(List<PathFilter> filters) {
+      this.filters = filters;
+    }
+
+    public void add(PathFilter one) {
+      filters.add(one);
+    }
+
+    public boolean accept(Path path) {
+      for (PathFilter filter : filters) {
+        if (filter.accept(path)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public String toString() {
+      StringBuffer buf = new StringBuffer();
+      buf.append("[");
+      for (PathFilter f: filters) {
+        buf.append(f);
+        buf.append(",");
+      }
+      buf.append("]");
+      return buf.toString();
+    }
+  }
+}

+ 169 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReader.java

@@ -0,0 +1,169 @@
+/**
+ * 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.*;
+import java.lang.reflect.*;
+
+import org.apache.hadoop.fs.FileSystem;
+
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A generic RecordReader that can hand out different recordReaders
+ * for each chunk in a {@link CombineFileSplit}.
+ * A CombineFileSplit can combine data chunks from multiple files. 
+ * This class allows using different RecordReaders for processing
+ * these data chunks from different files.
+ * @see CombineFileSplit
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class CombineFileRecordReader<K, V> extends RecordReader<K, V> {
+
+  static final Class [] constructorSignature = new Class [] 
+                                         {CombineFileSplit.class,
+                                          TaskAttemptContext.class,
+                                          Integer.class};
+
+  protected CombineFileSplit split;
+  protected Class<? extends RecordReader<K,V>> rrClass;
+  protected Constructor<? extends RecordReader<K,V>> rrConstructor;
+  protected FileSystem fs;
+  protected TaskAttemptContext context;
+  
+  protected int idx;
+  protected long progress;
+  protected RecordReader<K, V> curReader;
+  
+  public void initialize(InputSplit split,
+      TaskAttemptContext context) throws IOException, InterruptedException {
+    this.split = (CombineFileSplit)split;
+    this.context = context;
+    if (null != this.curReader) {
+      this.curReader.initialize(split, context);
+    }
+  }
+  
+  public boolean nextKeyValue() throws IOException, InterruptedException {
+
+    while ((curReader == null) || !curReader.nextKeyValue()) {
+      if (!initNextRecordReader()) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public K getCurrentKey() throws IOException, InterruptedException {
+    return curReader.getCurrentKey();
+  }
+  
+  public V getCurrentValue() throws IOException, InterruptedException {
+    return curReader.getCurrentValue();
+  }
+  
+  public void close() throws IOException {
+    if (curReader != null) {
+      curReader.close();
+      curReader = null;
+    }
+  }
+  
+  /**
+   * return progress based on the amount of data processed so far.
+   */
+  public float getProgress() throws IOException, InterruptedException {
+    long subprogress = 0;    // bytes processed in current split
+    if (null != curReader) {
+      // idx is always one past the current subsplit's true index.
+      subprogress = (long)(curReader.getProgress() * split.getLength(idx - 1));
+    }
+    return Math.min(1.0f,  (progress + subprogress)/(float)(split.getLength()));
+  }
+  
+  /**
+   * A generic RecordReader that can hand out different recordReaders
+   * for each chunk in the CombineFileSplit.
+   */
+  public CombineFileRecordReader(CombineFileSplit split,
+                                 TaskAttemptContext context,
+                                 Class<? extends RecordReader<K,V>> rrClass)
+    throws IOException {
+    this.split = split;
+    this.context = context;
+    this.rrClass = rrClass;
+    this.idx = 0;
+    this.curReader = null;
+    this.progress = 0;
+
+    try {
+      rrConstructor = rrClass.getDeclaredConstructor(constructorSignature);
+      rrConstructor.setAccessible(true);
+    } catch (Exception e) {
+      throw new RuntimeException(rrClass.getName() + 
+                                 " does not have valid constructor", e);
+    }
+    initNextRecordReader();
+  }
+  
+  /**
+   * Get the record reader for the next chunk in this CombineFileSplit.
+   */
+  protected boolean initNextRecordReader() throws IOException {
+
+    if (curReader != null) {
+      curReader.close();
+      curReader = null;
+      if (idx > 0) {
+        progress += split.getLength(idx-1);    // done processing so far
+      }
+    }
+
+    // if all chunks have been processed, nothing more to do.
+    if (idx == split.getNumPaths()) {
+      return false;
+    }
+
+    // get a record reader for the idx-th chunk
+    try {
+      Configuration conf = context.getConfiguration();
+      // setup some helper config variables.
+      conf.set("map.input.file", split.getPath(idx).toString());
+      conf.setLong("map.input.start", split.getOffset(idx));
+      conf.setLong("map.input.length", split.getLength(idx));
+
+      curReader =  rrConstructor.newInstance(new Object [] 
+                            {split, context, Integer.valueOf(idx)});
+
+      if (idx > 0) {
+        // initialize() for the first RecordReader will be called by MapTask;
+        // we're responsible for initializing subsequent RecordReaders.
+        curReader.initialize(split, context);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException (e);
+    }
+    idx++;
+    return true;
+  }
+}

+ 200 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/CombineFileSplit.java

@@ -0,0 +1,200 @@
+/**
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+
+/**
+ * A sub-collection of input files. 
+ * 
+ * Unlike {@link FileSplit}, CombineFileSplit class does not represent 
+ * a split of a file, but a split of input files into smaller sets. 
+ * A split may contain blocks from different file but all 
+ * the blocks in the same split are probably local to some rack <br> 
+ * CombineFileSplit can be used to implement {@link RecordReader}'s, 
+ * with reading one record per file.
+ * 
+ * @see FileSplit
+ * @see CombineFileInputFormat 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class CombineFileSplit extends InputSplit implements Writable {
+
+  private Path[] paths;
+  private long[] startoffset;
+  private long[] lengths;
+  private String[] locations;
+  private long totLength;
+
+  /**
+   * default constructor
+   */
+  public CombineFileSplit() {}
+  public CombineFileSplit(Path[] files, long[] start, 
+                          long[] lengths, String[] locations) {
+    initSplit(files, start, lengths, locations);
+  }
+
+  public CombineFileSplit(Path[] files, long[] lengths) {
+    long[] startoffset = new long[files.length];
+    for (int i = 0; i < startoffset.length; i++) {
+      startoffset[i] = 0;
+    }
+    String[] locations = new String[files.length];
+    for (int i = 0; i < locations.length; i++) {
+      locations[i] = "";
+    }
+    initSplit(files, startoffset, lengths, locations);
+  }
+  
+  private void initSplit(Path[] files, long[] start, 
+                         long[] lengths, String[] locations) {
+    this.startoffset = start;
+    this.lengths = lengths;
+    this.paths = files;
+    this.totLength = 0;
+    this.locations = locations;
+    for(long length : lengths) {
+      totLength += length;
+    }
+  }
+
+  /**
+   * Copy constructor
+   */
+  public CombineFileSplit(CombineFileSplit old) throws IOException {
+    this(old.getPaths(), old.getStartOffsets(),
+         old.getLengths(), old.getLocations());
+  }
+
+  public long getLength() {
+    return totLength;
+  }
+
+  /** Returns an array containing the start offsets of the files in the split*/ 
+  public long[] getStartOffsets() {
+    return startoffset;
+  }
+  
+  /** Returns an array containing the lengths of the files in the split*/ 
+  public long[] getLengths() {
+    return lengths;
+  }
+
+  /** Returns the start offset of the i<sup>th</sup> Path */
+  public long getOffset(int i) {
+    return startoffset[i];
+  }
+  
+  /** Returns the length of the i<sup>th</sup> Path */
+  public long getLength(int i) {
+    return lengths[i];
+  }
+  
+  /** Returns the number of Paths in the split */
+  public int getNumPaths() {
+    return paths.length;
+  }
+
+  /** Returns the i<sup>th</sup> Path */
+  public Path getPath(int i) {
+    return paths[i];
+  }
+  
+  /** Returns all the Paths in the split */
+  public Path[] getPaths() {
+    return paths;
+  }
+
+  /** Returns all the Paths where this input-split resides */
+  public String[] getLocations() throws IOException {
+    return locations;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    totLength = in.readLong();
+    int arrLength = in.readInt();
+    lengths = new long[arrLength];
+    for(int i=0; i<arrLength;i++) {
+      lengths[i] = in.readLong();
+    }
+    int filesLength = in.readInt();
+    paths = new Path[filesLength];
+    for(int i=0; i<filesLength;i++) {
+      paths[i] = new Path(Text.readString(in));
+    }
+    arrLength = in.readInt();
+    startoffset = new long[arrLength];
+    for(int i=0; i<arrLength;i++) {
+      startoffset[i] = in.readLong();
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(totLength);
+    out.writeInt(lengths.length);
+    for(long length : lengths) {
+      out.writeLong(length);
+    }
+    out.writeInt(paths.length);
+    for(Path p : paths) {
+      Text.writeString(out, p.toString());
+    }
+    out.writeInt(startoffset.length);
+    for(long length : startoffset) {
+      out.writeLong(length);
+    }
+  }
+  
+  @Override
+ public String toString() {
+    StringBuffer sb = new StringBuffer();
+    for (int i = 0; i < paths.length; i++) {
+      if (i == 0 ) {
+        sb.append("Paths:");
+      }
+      sb.append(paths[i].toUri().getPath() + ":" + startoffset[i] +
+                "+" + lengths[i]);
+      if (i < paths.length -1) {
+        sb.append(",");
+      }
+    }
+    if (locations != null) {
+      String locs = "";
+      StringBuffer locsb = new StringBuffer();
+      for (int i = 0; i < locations.length; i++) {
+        locsb.append(locations[i] + ":");
+      }
+      locs = locsb.toString();
+      sb.append(" Locations:" + locs + "; ");
+    }
+    return sb.toString();
+  }
+}

+ 131 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/DelegatingInputFormat.java

@@ -0,0 +1,131 @@
+/**
+ * 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.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * An {@link InputFormat} that delegates behavior of paths to multiple other
+ * InputFormats.
+ * 
+ * @see MultipleInputs#addInputPath(Job, Path, Class, Class)
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DelegatingInputFormat<K, V> extends InputFormat<K, V> {
+
+  @SuppressWarnings("unchecked")
+  public List<InputSplit> getSplits(JobContext job) 
+      throws IOException, InterruptedException {
+    Configuration conf = job.getConfiguration();
+    Job jobCopy =new Job(conf);
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+    Map<Path, InputFormat> formatMap = 
+      MultipleInputs.getInputFormatMap(job);
+    Map<Path, Class<? extends Mapper>> mapperMap = MultipleInputs
+       .getMapperTypeMap(job);
+    Map<Class<? extends InputFormat>, List<Path>> formatPaths
+        = new HashMap<Class<? extends InputFormat>, List<Path>>();
+
+    // First, build a map of InputFormats to Paths
+    for (Entry<Path, InputFormat> entry : formatMap.entrySet()) {
+      if (!formatPaths.containsKey(entry.getValue().getClass())) {
+       formatPaths.put(entry.getValue().getClass(), new LinkedList<Path>());
+      }
+
+      formatPaths.get(entry.getValue().getClass()).add(entry.getKey());
+    }
+
+    for (Entry<Class<? extends InputFormat>, List<Path>> formatEntry : 
+        formatPaths.entrySet()) {
+      Class<? extends InputFormat> formatClass = formatEntry.getKey();
+      InputFormat format = (InputFormat) ReflectionUtils.newInstance(
+         formatClass, conf);
+      List<Path> paths = formatEntry.getValue();
+
+      Map<Class<? extends Mapper>, List<Path>> mapperPaths
+          = new HashMap<Class<? extends Mapper>, List<Path>>();
+
+      // Now, for each set of paths that have a common InputFormat, build
+      // a map of Mappers to the paths they're used for
+      for (Path path : paths) {
+       Class<? extends Mapper> mapperClass = mapperMap.get(path);
+       if (!mapperPaths.containsKey(mapperClass)) {
+         mapperPaths.put(mapperClass, new LinkedList<Path>());
+       }
+
+       mapperPaths.get(mapperClass).add(path);
+      }
+
+      // Now each set of paths that has a common InputFormat and Mapper can
+      // be added to the same job, and split together.
+      for (Entry<Class<? extends Mapper>, List<Path>> mapEntry :
+          mapperPaths.entrySet()) {
+       paths = mapEntry.getValue();
+       Class<? extends Mapper> mapperClass = mapEntry.getKey();
+
+       if (mapperClass == null) {
+         try {
+           mapperClass = job.getMapperClass();
+         } catch (ClassNotFoundException e) {
+           throw new IOException("Mapper class is not found", e);
+         }
+       }
+
+       FileInputFormat.setInputPaths(jobCopy, paths.toArray(new Path[paths
+           .size()]));
+
+       // Get splits for each input path and tag with InputFormat
+       // and Mapper types by wrapping in a TaggedInputSplit.
+       List<InputSplit> pathSplits = format.getSplits(jobCopy);
+       for (InputSplit pathSplit : pathSplits) {
+         splits.add(new TaggedInputSplit(pathSplit, conf, format.getClass(),
+             mapperClass));
+       }
+      }
+    }
+
+    return splits;
+  }
+
+  @Override
+  public RecordReader<K, V> createRecordReader(InputSplit split,
+      TaskAttemptContext context) throws IOException, InterruptedException {
+    return new DelegatingRecordReader<K, V>(split, context);
+  }
+}

+ 58 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/DelegatingMapper.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.lib.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * An {@link Mapper} that delegates behavior of paths to multiple other
+ * mappers.
+ * 
+ * @see MultipleInputs#addInputPath(Job, Path, Class, Class)
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DelegatingMapper<K1, V1, K2, V2> extends Mapper<K1, V1, K2, V2> {
+
+  private Mapper<K1, V1, K2, V2> mapper;
+
+  @SuppressWarnings("unchecked")
+  protected void setup(Context context)
+      throws IOException, InterruptedException {
+    // Find the Mapper from the TaggedInputSplit.
+    TaggedInputSplit inputSplit = (TaggedInputSplit) context.getInputSplit();
+    mapper = (Mapper<K1, V1, K2, V2>) ReflectionUtils.newInstance(inputSplit
+       .getMapperClass(), context.getConfiguration());
+    
+  }
+
+  @SuppressWarnings("unchecked")
+  public void run(Context context) 
+      throws IOException, InterruptedException {
+    setup(context);
+    mapper.run(context);
+    cleanup(context);
+  }
+}

+ 92 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/DelegatingRecordReader.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.lib.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * This is a delegating RecordReader, which delegates the functionality to the
+ * underlying record reader in {@link TaggedInputSplit}  
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DelegatingRecordReader<K, V> extends RecordReader<K, V> {
+  RecordReader<K, V> originalRR;
+
+  /**
+   * Constructs the DelegatingRecordReader.
+   * 
+   * @param split TaggegInputSplit object
+   * @param context TaskAttemptContext object
+   *  
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @SuppressWarnings("unchecked")
+  public DelegatingRecordReader(InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    // Find the InputFormat and then the RecordReader from the
+    // TaggedInputSplit.
+    TaggedInputSplit taggedInputSplit = (TaggedInputSplit) split;
+    InputFormat<K, V> inputFormat = (InputFormat<K, V>) ReflectionUtils
+        .newInstance(taggedInputSplit.getInputFormatClass(), context
+            .getConfiguration());
+    originalRR = inputFormat.createRecordReader(taggedInputSplit
+        .getInputSplit(), context);
+  }
+
+  @Override
+  public void close() throws IOException {
+    originalRR.close();
+  }
+
+  @Override
+  public K getCurrentKey() throws IOException, InterruptedException {
+    return originalRR.getCurrentKey();
+  }
+
+  @Override
+  public V getCurrentValue() throws IOException, InterruptedException {
+    return originalRR.getCurrentValue();
+  }
+
+  @Override
+  public float getProgress() throws IOException, InterruptedException {
+    return originalRR.getProgress();
+  }
+
+  @Override
+  public void initialize(InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    originalRR.initialize(((TaggedInputSplit) split).getInputSplit(), context);
+  }
+
+  @Override
+  public boolean nextKeyValue() throws IOException, InterruptedException {
+    return originalRR.nextKeyValue();
+  }
+
+}

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

@@ -422,6 +422,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
    */
   public static Path[] getInputPaths(JobContext context) {
     String dirs = context.getConfiguration().get("mapred.input.dir", "");
+    System.out.println("****" + dirs);
     String [] list = StringUtils.split(dirs);
     Path[] result = new Path[list.length];
     for (int i = 0; i < list.length; i++) {

+ 128 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/KeyValueLineRecordReader.java

@@ -0,0 +1,128 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * This class treats a line in the input as a key/value pair separated by a 
+ * separator character. The separator can be specified in config file 
+ * under the attribute name mapreduce.input.keyvaluelinerecordreader.key.value.separator. The default
+ * separator is the tab character ('\t').
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class KeyValueLineRecordReader extends RecordReader<Text, Text> {
+  public static final String KEY_VALUE_SEPERATOR = 
+    "mapreduce.input.keyvaluelinerecordreader.key.value.separator";
+  
+  private final LineRecordReader lineRecordReader;
+
+  private byte separator = (byte) '\t';
+
+  private Text innerValue;
+
+  private Text key;
+  
+  private Text value;
+  
+  public Class getKeyClass() { return Text.class; }
+  
+  public KeyValueLineRecordReader(Configuration conf)
+    throws IOException {
+    
+    lineRecordReader = new LineRecordReader();
+    String sepStr = conf.get(KEY_VALUE_SEPERATOR, "\t");
+    this.separator = (byte) sepStr.charAt(0);
+  }
+
+  public void initialize(InputSplit genericSplit,
+      TaskAttemptContext context) throws IOException {
+    lineRecordReader.initialize(genericSplit, context);
+  }
+  
+  public static int findSeparator(byte[] utf, int start, int length, 
+      byte sep) {
+    for (int i = start; i < (start + length); i++) {
+      if (utf[i] == sep) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  public static void setKeyValue(Text key, Text value, byte[] line,
+      int lineLen, int pos) {
+    if (pos == -1) {
+      key.set(line, 0, lineLen);
+      value.set("");
+    } else {
+      key.set(line, 0, pos);
+      value.set(line, pos + 1, lineLen - pos - 1);
+    }
+  }
+  /** Read key/value pair in a line. */
+  public synchronized boolean nextKeyValue()
+    throws IOException {
+    byte[] line = null;
+    int lineLen = -1;
+    if (lineRecordReader.nextKeyValue()) {
+      innerValue = lineRecordReader.getCurrentValue();
+      line = innerValue.getBytes();
+      lineLen = innerValue.getLength();
+    } else {
+      return false;
+    }
+    if (line == null)
+      return false;
+    if (key == null) {
+      key = new Text();
+    }
+    if (value == null) {
+      value = new Text();
+    }
+    int pos = findSeparator(line, 0, lineLen, this.separator);
+    setKeyValue(key, value, line, lineLen, pos);
+    return true;
+  }
+  
+  public Text getCurrentKey() {
+    return key;
+  }
+
+  public Text getCurrentValue() {
+    return value;
+  }
+
+  public float getProgress() throws IOException {
+    return lineRecordReader.getProgress();
+  }
+  
+  public synchronized void close() throws IOException { 
+    lineRecordReader.close();
+  }
+}

+ 59 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/KeyValueTextInputFormat.java

@@ -0,0 +1,59 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+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 line feed or carriage-return are used to signal end of line. 
+ * Each line is divided into key and value parts by a separator byte. If no
+ * such a byte exists, the key will be the entire line and value will be empty.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class KeyValueTextInputFormat extends FileInputFormat<Text, Text> {
+
+  @Override
+  protected boolean isSplitable(JobContext context, Path file) {
+    final CompressionCodec codec =
+      new CompressionCodecFactory(context.getConfiguration()).getCodec(file);
+    return codec == null;
+  }
+
+  public RecordReader<Text, Text> createRecordReader(InputSplit genericSplit,
+      TaskAttemptContext context) throws IOException {
+    
+    context.setStatus(genericSplit.toString());
+    return new KeyValueLineRecordReader(context.getConfiguration());
+  }
+
+}

+ 150 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/MultipleInputs.java

@@ -0,0 +1,150 @@
+/**
+ * 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.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * This class supports MapReduce jobs that have multiple input paths with
+ * a different {@link InputFormat} and {@link Mapper} for each path 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class MultipleInputs {
+  public static final String DIR_FORMATS = 
+    "mapreduce.input.multipleinputs.dir.formats";
+  public static final String DIR_MAPPERS = 
+    "mapreduce.input.multipleinputs.dir.mappers";
+  
+  /**
+   * Add a {@link Path} with a custom {@link InputFormat} to the list of
+   * inputs for the map-reduce job.
+   * 
+   * @param job The {@link Job}
+   * @param path {@link Path} to be added to the list of inputs for the job
+   * @param inputFormatClass {@link InputFormat} class to use for this path
+   */
+  @SuppressWarnings("unchecked")
+  public static void addInputPath(Job job, Path path,
+      Class<? extends InputFormat> inputFormatClass) {
+    String inputFormatMapping = path.toString() + ";"
+       + inputFormatClass.getName();
+    Configuration conf = job.getConfiguration();
+    String inputFormats = conf.get(DIR_FORMATS);
+    conf.set(DIR_FORMATS,
+       inputFormats == null ? inputFormatMapping : inputFormats + ","
+           + inputFormatMapping);
+
+    job.setInputFormatClass(DelegatingInputFormat.class);
+  }
+
+  /**
+   * Add a {@link Path} with a custom {@link InputFormat} and
+   * {@link Mapper} to the list of inputs for the map-reduce job.
+   * 
+   * @param job The {@link Job}
+   * @param path {@link Path} to be added to the list of inputs for the job
+   * @param inputFormatClass {@link InputFormat} class to use for this path
+   * @param mapperClass {@link Mapper} class to use for this path
+   */
+  @SuppressWarnings("unchecked")
+  public static void addInputPath(Job job, Path path,
+      Class<? extends InputFormat> inputFormatClass,
+      Class<? extends Mapper> mapperClass) {
+
+    addInputPath(job, path, inputFormatClass);
+    Configuration conf = job.getConfiguration();
+    String mapperMapping = path.toString() + ";" + mapperClass.getName();
+    String mappers = conf.get(DIR_MAPPERS);
+    conf.set(DIR_MAPPERS, mappers == null ? mapperMapping
+       : mappers + "," + mapperMapping);
+
+    job.setMapperClass(DelegatingMapper.class);
+  }
+
+  /**
+   * Retrieves a map of {@link Path}s to the {@link InputFormat} class
+   * that should be used for them.
+   * 
+   * @param job The {@link JobContext}
+   * @see #addInputPath(JobConf, Path, Class)
+   * @return A map of paths to inputformats for the job
+   */
+  @SuppressWarnings("unchecked")
+  static Map<Path, InputFormat> getInputFormatMap(JobContext job) {
+    Map<Path, InputFormat> m = new HashMap<Path, InputFormat>();
+    Configuration conf = job.getConfiguration();
+    String[] pathMappings = conf.get(DIR_FORMATS).split(",");
+    for (String pathMapping : pathMappings) {
+      String[] split = pathMapping.split(";");
+      InputFormat inputFormat;
+      try {
+       inputFormat = (InputFormat) ReflectionUtils.newInstance(conf
+           .getClassByName(split[1]), conf);
+      } catch (ClassNotFoundException e) {
+       throw new RuntimeException(e);
+      }
+      m.put(new Path(split[0]), inputFormat);
+    }
+    return m;
+  }
+
+  /**
+   * Retrieves a map of {@link Path}s to the {@link Mapper} class that
+   * should be used for them.
+   * 
+   * @param job The {@link JobContext}
+   * @see #addInputPath(JobConf, Path, Class, Class)
+   * @return A map of paths to mappers for the job
+   */
+  @SuppressWarnings("unchecked")
+  static Map<Path, Class<? extends Mapper>> 
+      getMapperTypeMap(JobContext job) {
+    Configuration conf = job.getConfiguration();
+    if (conf.get(DIR_MAPPERS) == null) {
+      return Collections.emptyMap();
+    }
+    Map<Path, Class<? extends Mapper>> m = 
+      new HashMap<Path, Class<? extends Mapper>>();
+    String[] pathMappings = conf.get(DIR_MAPPERS).split(",");
+    for (String pathMapping : pathMappings) {
+      String[] split = pathMapping.split(";");
+      Class<? extends Mapper> mapClass;
+      try {
+       mapClass = 
+         (Class<? extends Mapper>) conf.getClassByName(split[1]);
+      } catch (ClassNotFoundException e) {
+       throw new RuntimeException(e);
+      }
+      m.put(new Path(split[0]), mapClass);
+    }
+    return m;
+  }
+}

+ 155 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java

@@ -0,0 +1,155 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+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.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.LineReader;
+
+/**
+ * NLineInputFormat which splits N lines of input as one split.
+ *
+ * In many "pleasantly" parallel applications, each process/mapper 
+ * processes the same input file (s), but with computations are 
+ * controlled by different parameters.(Referred to as "parameter sweeps").
+ * One way to achieve this, is to specify a set of parameters 
+ * (one set per line) as input in a control file 
+ * (which is the input path to the map-reduce application,
+ * where as the input dataset is specified 
+ * via a config variable in JobConf.).
+ * 
+ * The NLineInputFormat can be used in such applications, that splits 
+ * the input file such that by default, one line is fed as
+ * a value to one map task, and key is the offset.
+ * i.e. (k,v) is (LongWritable, Text).
+ * The location hints will span the whole mapred cluster.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class NLineInputFormat extends FileInputFormat<LongWritable, Text> { 
+  public static final String LINES_PER_MAP = 
+    "mapreduce.input.lineinputformat.linespermap";
+
+  public RecordReader<LongWritable, Text> createRecordReader(
+      InputSplit genericSplit, TaskAttemptContext context) 
+      throws IOException {
+    context.setStatus(genericSplit.toString());
+    return new LineRecordReader();
+  }
+
+  /** 
+   * Logically splits the set of input files for the job, splits N lines
+   * of the input as one split.
+   * 
+   * @see FileInputFormat#getSplits(JobContext)
+   */
+  public List<InputSplit> getSplits(JobContext job)
+  throws IOException {
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+    int numLinesPerSplit = getNumLinesPerSplit(job);
+    for (FileStatus status : listStatus(job)) {
+      splits.addAll(getSplitsForFile(status,
+        job.getConfiguration(), numLinesPerSplit));
+    }
+    return splits;
+  }
+  
+  public static List<FileSplit> getSplitsForFile(FileStatus status,
+      Configuration conf, int numLinesPerSplit) throws IOException {
+    List<FileSplit> splits = new ArrayList<FileSplit> ();
+    Path fileName = status.getPath();
+    if (status.isDir()) {
+      throw new IOException("Not a file: " + fileName);
+    }
+    FileSystem  fs = fileName.getFileSystem(conf);
+    LineReader lr = null;
+    try {
+      FSDataInputStream in  = fs.open(fileName);
+      lr = new LineReader(in, conf);
+      Text line = new Text();
+      int numLines = 0;
+      long begin = 0;
+      long length = 0;
+      int num = -1;
+      while ((num = lr.readLine(line)) > 0) {
+        numLines++;
+        length += num;
+        if (numLines == numLinesPerSplit) {
+          // NLineInputFormat uses LineRecordReader, which always reads
+          // (and consumes) at least one character out of its upper split
+          // boundary. So to make sure that each mapper gets N lines, we
+          // move back the upper split limits of each split 
+          // by one character here.
+          if (begin == 0) {
+            splits.add(new FileSplit(fileName, begin, length - 1,
+              new String[] {}));
+          } else {
+            splits.add(new FileSplit(fileName, begin - 1, length,
+              new String[] {}));
+          }
+          begin += length;
+          length = 0;
+          numLines = 0;
+        }
+      }
+      if (numLines != 0) {
+        splits.add(new FileSplit(fileName, begin, length, new String[]{}));
+      }
+    } finally {
+      if (lr != null) {
+        lr.close();
+      }
+    }
+    return splits; 
+  }
+  
+  /**
+   * Set the number of lines per split
+   * @param job the job to modify
+   * @param numLines the number of lines per split
+   */
+  public static void setNumLinesPerSplit(Job job, int numLines) {
+    job.getConfiguration().setInt(LINES_PER_MAP, numLines);
+  }
+
+  /**
+   * Get the number of lines per split
+   * @param job the job
+   * @return the number of lines per split
+   */
+  public static int getNumLinesPerSplit(JobContext job) {
+    return job.getConfiguration().getInt(LINES_PER_MAP, 1);
+  }
+}

+ 154 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileAsBinaryInputFormat.java

@@ -0,0 +1,154 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * InputFormat reading keys, values from SequenceFiles in binary (raw)
+ * format.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class SequenceFileAsBinaryInputFormat
+    extends SequenceFileInputFormat<BytesWritable,BytesWritable> {
+
+  public SequenceFileAsBinaryInputFormat() {
+    super();
+  }
+
+  public RecordReader<BytesWritable,BytesWritable> createRecordReader(
+      InputSplit split, TaskAttemptContext context)
+      throws IOException {
+    return new SequenceFileAsBinaryRecordReader();
+  }
+
+  /**
+   * Read records from a SequenceFile as binary (raw) bytes.
+   */
+  public static class SequenceFileAsBinaryRecordReader
+      extends RecordReader<BytesWritable,BytesWritable> {
+    private SequenceFile.Reader in;
+    private long start;
+    private long end;
+    private boolean done = false;
+    private DataOutputBuffer buffer = new DataOutputBuffer();
+    private SequenceFile.ValueBytes vbytes;
+    private BytesWritable key = null;
+    private BytesWritable value = null;
+
+    public void initialize(InputSplit split, TaskAttemptContext context) 
+        throws IOException, InterruptedException {
+      Path path = ((FileSplit)split).getPath();
+      Configuration conf = context.getConfiguration();
+      FileSystem fs = path.getFileSystem(conf);
+      this.in = new SequenceFile.Reader(fs, path, conf);
+      this.end = ((FileSplit)split).getStart() + split.getLength();
+      if (((FileSplit)split).getStart() > in.getPosition()) {
+        in.sync(((FileSplit)split).getStart());    // sync to start
+      }
+      this.start = in.getPosition();
+      vbytes = in.createValueBytes();
+      done = start >= end;
+    }
+    
+    @Override
+    public BytesWritable getCurrentKey() 
+        throws IOException, InterruptedException {
+      return key;
+    }
+    
+    @Override
+    public BytesWritable getCurrentValue() 
+        throws IOException, InterruptedException {
+      return value;
+    }
+
+    /**
+     * Retrieve the name of the key class for this SequenceFile.
+     * @see org.apache.hadoop.io.SequenceFile.Reader#getKeyClassName
+     */
+    public String getKeyClassName() {
+      return in.getKeyClassName();
+    }
+
+    /**
+     * Retrieve the name of the value class for this SequenceFile.
+     * @see org.apache.hadoop.io.SequenceFile.Reader#getValueClassName
+     */
+    public String getValueClassName() {
+      return in.getValueClassName();
+    }
+
+    /**
+     * Read raw bytes from a SequenceFile.
+     */
+    public synchronized boolean nextKeyValue()
+        throws IOException, InterruptedException {
+      if (done) {
+        return false;
+      }
+      long pos = in.getPosition();
+      boolean eof = -1 == in.nextRawKey(buffer);
+      if (!eof) {
+        if (key == null) {
+          key = new BytesWritable();
+        }
+        if (value == null) {
+          value = new BytesWritable();
+        }
+        key.set(buffer.getData(), 0, buffer.getLength());
+        buffer.reset();
+        in.nextRawValue(vbytes);
+        vbytes.writeUncompressedBytes(buffer);
+        value.set(buffer.getData(), 0, buffer.getLength());
+        buffer.reset();
+      }
+      return !(done = (eof || (pos >= end && in.syncSeen())));
+    }
+
+    public void close() throws IOException {
+      in.close();
+    }
+
+    /**
+     * Return the progress within the input split
+     * @return 0.0 to 1.0 of the input byte range
+     */
+    public float getProgress() throws IOException, InterruptedException {
+      if (end == start) {
+        return 0.0f;
+      } else {
+        return Math.min(1.0f, (float)((in.getPosition() - start) /
+                                      (double)(end - start)));
+      }
+    }
+  }
+}

+ 49 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileAsTextInputFormat.java

@@ -0,0 +1,49 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * This class is similar to SequenceFileInputFormat, except it generates
+ * SequenceFileAsTextRecordReader which converts the input keys and values
+ * to their String forms by calling toString() method. 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class SequenceFileAsTextInputFormat
+  extends SequenceFileInputFormat<Text, Text> {
+
+  public SequenceFileAsTextInputFormat() {
+    super();
+  }
+
+  public RecordReader<Text, Text> createRecordReader(InputSplit split,
+      TaskAttemptContext context) throws IOException {
+    context.setStatus(split.toString());
+    return new SequenceFileAsTextRecordReader();
+  }
+}

+ 95 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileAsTextRecordReader.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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * This class converts the input keys and values to their String forms by
+ * calling toString() method. This class to SequenceFileAsTextInputFormat
+ * class is as LineRecordReader class to TextInputFormat class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class SequenceFileAsTextRecordReader
+  extends RecordReader<Text, Text> {
+  
+  private final SequenceFileRecordReader<WritableComparable<?>, Writable>
+    sequenceFileRecordReader;
+
+  private Text key;
+  private Text value;
+
+  public SequenceFileAsTextRecordReader()
+    throws IOException {
+    sequenceFileRecordReader =
+      new SequenceFileRecordReader<WritableComparable<?>, Writable>();
+  }
+
+  public void initialize(InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    sequenceFileRecordReader.initialize(split, context);
+  }
+
+  @Override
+  public Text getCurrentKey() 
+      throws IOException, InterruptedException {
+    return key;
+  }
+  
+  @Override
+  public Text getCurrentValue() 
+      throws IOException, InterruptedException {
+    return value;
+  }
+  
+  /** Read key/value pair in a line. */
+  public synchronized boolean nextKeyValue() 
+      throws IOException, InterruptedException {
+    if (!sequenceFileRecordReader.nextKeyValue()) {
+      return false;
+    }
+    if (key == null) {
+      key = new Text(); 
+    }
+    if (value == null) {
+      value = new Text(); 
+    }
+    key.set(sequenceFileRecordReader.getCurrentKey().toString());
+    value.set(sequenceFileRecordReader.getCurrentValue().toString());
+    return true;
+  }
+  
+  public float getProgress() throws IOException,  InterruptedException {
+    return sequenceFileRecordReader.getProgress();
+  }
+  
+  public synchronized void close() throws IOException {
+    sequenceFileRecordReader.close();
+  }
+}

+ 325 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/SequenceFileInputFilter.java

@@ -0,0 +1,325 @@
+/**
+ * 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.nio.ByteBuffer;
+import java.security.DigestException;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * A class that allows a map/red job to work on a sample of sequence files.
+ * The sample is decided by the filter class set by the job.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class SequenceFileInputFilter<K, V>
+    extends SequenceFileInputFormat<K, V> {
+  public static final Log LOG = LogFactory.getLog(FileInputFormat.class);
+  
+  final public static String FILTER_CLASS = 
+    "mapreduce.input.sequencefileinputfilter.class";
+  final public static String FILTER_FREQUENCY = 
+    "mapreduce.input.sequencefileinputfilter.frequency";
+  final public static String FILTER_REGEX = 
+    "mapreduce.input.sequencefileinputfilter.regex";
+    
+  public SequenceFileInputFilter() {
+  }
+    
+  /** Create a record reader for the given split
+   * @param split file split
+   * @param context the task-attempt context
+   * @return RecordReader
+   */
+  public RecordReader<K, V> createRecordReader(InputSplit split,
+      TaskAttemptContext context) throws IOException {
+    context.setStatus(split.toString());
+    return new FilterRecordReader<K, V>(context.getConfiguration());
+  }
+
+
+  /** set the filter class
+   * 
+   * @param job The job
+   * @param filterClass filter class
+   */
+  public static void setFilterClass(Job job, Class<?> filterClass) {
+    job.getConfiguration().set(FILTER_CLASS, filterClass.getName());
+  }
+
+         
+  /**
+   * filter interface
+   */
+  public interface Filter extends Configurable {
+    /** filter function
+     * Decide if a record should be filtered or not
+     * @param key record key
+     * @return true if a record is accepted; return false otherwise
+     */
+    public abstract boolean accept(Object key);
+  }
+    
+  /**
+   * base class for Filters
+   */
+  public static abstract class FilterBase implements Filter {
+    Configuration conf;
+        
+    public Configuration getConf() {
+      return conf;
+    }
+  }
+    
+  /** Records filter by matching key to regex
+   */
+  public static class RegexFilter extends FilterBase {
+    private Pattern p;
+    /** Define the filtering regex and stores it in conf
+     * @param conf where the regex is set
+     * @param regex regex used as a filter
+     */
+    public static void setPattern(Configuration conf, String regex)
+        throws PatternSyntaxException {
+      try {
+        Pattern.compile(regex);
+      } catch (PatternSyntaxException e) {
+        throw new IllegalArgumentException("Invalid pattern: "+regex);
+      }
+      conf.set(FILTER_REGEX, regex);
+    }
+        
+    public RegexFilter() { }
+        
+    /** configure the Filter by checking the configuration
+     */
+    public void setConf(Configuration conf) {
+      String regex = conf.get(FILTER_REGEX);
+      if (regex == null)
+        throw new RuntimeException(FILTER_REGEX + "not set");
+      this.p = Pattern.compile(regex);
+      this.conf = conf;
+    }
+
+
+    /** Filtering method
+     * If key matches the regex, return true; otherwise return false
+     * @see Filter#accept(Object)
+     */
+    public boolean accept(Object key) {
+      return p.matcher(key.toString()).matches();
+    }
+  }
+
+  /** This class returns a percentage of records
+   * The percentage is determined by a filtering frequency <i>f</i> using
+   * the criteria record# % f == 0.
+   * For example, if the frequency is 10, one out of 10 records is returned.
+   */
+  public static class PercentFilter extends FilterBase {
+    private int frequency;
+    private int count;
+
+    /** set the frequency and stores it in conf
+     * @param conf configuration
+     * @param frequency filtering frequencey
+     */
+    public static void setFrequency(Configuration conf, int frequency) {
+      if (frequency <= 0)
+        throw new IllegalArgumentException(
+          "Negative " + FILTER_FREQUENCY + ": " + frequency);
+      conf.setInt(FILTER_FREQUENCY, frequency);
+    }
+        
+    public PercentFilter() { }
+        
+    /** configure the filter by checking the configuration
+     * 
+     * @param conf configuration
+     */
+    public void setConf(Configuration conf) {
+      this.frequency = conf.getInt(FILTER_FREQUENCY, 10);
+      if (this.frequency <= 0) {
+        throw new RuntimeException(
+          "Negative "+FILTER_FREQUENCY + ": " + this.frequency);
+      }
+      this.conf = conf;
+    }
+
+    /** Filtering method
+     * If record# % frequency==0, return true; otherwise return false
+     * @see Filter#accept(Object)
+     */
+    public boolean accept(Object key) {
+      boolean accepted = false;
+      if (count == 0)
+        accepted = true;
+      if (++count == frequency) {
+        count = 0;
+      }
+      return accepted;
+    }
+  }
+
+  /** This class returns a set of records by examing the MD5 digest of its
+   * key against a filtering frequency <i>f</i>. The filtering criteria is
+   * MD5(key) % f == 0.
+   */
+  public static class MD5Filter extends FilterBase {
+    private int frequency;
+    private static final MessageDigest DIGESTER;
+    public static final int MD5_LEN = 16;
+    private byte [] digest = new byte[MD5_LEN];
+        
+    static {
+      try {
+        DIGESTER = MessageDigest.getInstance("MD5");
+      } catch (NoSuchAlgorithmException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+
+    /** set the filtering frequency in configuration
+     * 
+     * @param conf configuration
+     * @param frequency filtering frequency
+     */
+    public static void setFrequency(Configuration conf, int frequency) {
+      if (frequency <= 0)
+        throw new IllegalArgumentException(
+          "Negative " + FILTER_FREQUENCY + ": " + frequency);
+      conf.setInt(FILTER_FREQUENCY, frequency);
+    }
+        
+    public MD5Filter() { }
+        
+    /** configure the filter according to configuration
+     * 
+     * @param conf configuration
+     */
+    public void setConf(Configuration conf) {
+      this.frequency = conf.getInt(FILTER_FREQUENCY, 10);
+      if (this.frequency <= 0) {
+        throw new RuntimeException(
+          "Negative " + FILTER_FREQUENCY + ": " + this.frequency);
+      }
+      this.conf = conf;
+    }
+
+    /** Filtering method
+     * If MD5(key) % frequency==0, return true; otherwise return false
+     * @see Filter#accept(Object)
+     */
+    public boolean accept(Object key) {
+      try {
+        long hashcode;
+        if (key instanceof Text) {
+          hashcode = MD5Hashcode((Text)key);
+        } else if (key instanceof BytesWritable) {
+          hashcode = MD5Hashcode((BytesWritable)key);
+        } else {
+          ByteBuffer bb;
+          bb = Text.encode(key.toString());
+          hashcode = MD5Hashcode(bb.array(), 0, bb.limit());
+        }
+        if (hashcode / frequency * frequency == hashcode)
+          return true;
+      } catch(Exception e) {
+        LOG.warn(e);
+        throw new RuntimeException(e);
+      }
+      return false;
+    }
+        
+    private long MD5Hashcode(Text key) throws DigestException {
+      return MD5Hashcode(key.getBytes(), 0, key.getLength());
+    }
+        
+    private long MD5Hashcode(BytesWritable key) throws DigestException {
+      return MD5Hashcode(key.getBytes(), 0, key.getLength());
+    }
+    
+    synchronized private long MD5Hashcode(byte[] bytes, 
+        int start, int length) throws DigestException {
+      DIGESTER.update(bytes, 0, length);
+      DIGESTER.digest(digest, 0, MD5_LEN);
+      long hashcode=0;
+      for (int i = 0; i < 8; i++)
+        hashcode |= ((digest[i] & 0xffL) << (8 * (7 - i)));
+      return hashcode;
+    }
+  }
+    
+  private static class FilterRecordReader<K, V>
+      extends SequenceFileRecordReader<K, V> {
+    
+    private Filter filter;
+    private K key;
+    private V value;
+        
+    public FilterRecordReader(Configuration conf)
+        throws IOException {
+      super();
+      // instantiate filter
+      filter = (Filter)ReflectionUtils.newInstance(
+        conf.getClass(FILTER_CLASS, PercentFilter.class), conf);
+    }
+    
+    public synchronized boolean nextKeyValue() 
+        throws IOException, InterruptedException {
+      while (super.nextKeyValue()) {
+        key = super.getCurrentKey();
+        if (filter.accept(key)) {
+          value = super.getCurrentValue();
+          return true;
+        }
+      }
+      return false;
+    }
+    
+    @Override
+    public K getCurrentKey() {
+      return key;
+    }
+    
+    @Override
+    public V getCurrentValue() {
+      return value;
+    }
+  }
+}

+ 159 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/input/TaggedInputSplit.java

@@ -0,0 +1,159 @@
+/**
+ * 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.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * An {@link InputSplit} that tags another InputSplit with extra data for use
+ * by {@link DelegatingInputFormat}s and {@link DelegatingMapper}s.
+ */
+class TaggedInputSplit extends InputSplit implements Configurable, Writable {
+
+  private Class<? extends InputSplit> inputSplitClass;
+
+  private InputSplit inputSplit;
+
+  @SuppressWarnings("unchecked")
+  private Class<? extends InputFormat> inputFormatClass;
+
+  @SuppressWarnings("unchecked")
+  private Class<? extends Mapper> mapperClass;
+
+  private Configuration conf;
+
+  public TaggedInputSplit() {
+    // Default constructor.
+  }
+
+  /**
+   * Creates a new TaggedInputSplit.
+   * 
+   * @param inputSplit The InputSplit to be tagged
+   * @param conf The configuration to use
+   * @param inputFormatClass The InputFormat class to use for this job
+   * @param mapperClass The Mapper class to use for this job
+   */
+  @SuppressWarnings("unchecked")
+  public TaggedInputSplit(InputSplit inputSplit, Configuration conf,
+      Class<? extends InputFormat> inputFormatClass,
+      Class<? extends Mapper> mapperClass) {
+    this.inputSplitClass = inputSplit.getClass();
+    this.inputSplit = inputSplit;
+    this.conf = conf;
+    this.inputFormatClass = inputFormatClass;
+    this.mapperClass = mapperClass;
+  }
+
+  /**
+   * Retrieves the original InputSplit.
+   * 
+   * @return The InputSplit that was tagged
+   */
+  public InputSplit getInputSplit() {
+    return inputSplit;
+  }
+
+  /**
+   * Retrieves the InputFormat class to use for this split.
+   * 
+   * @return The InputFormat class to use
+   */
+  @SuppressWarnings("unchecked")
+  public Class<? extends InputFormat> getInputFormatClass() {
+    return inputFormatClass;
+  }
+
+  /**
+   * Retrieves the Mapper class to use for this split.
+   * 
+   * @return The Mapper class to use
+   */
+  @SuppressWarnings("unchecked")
+  public Class<? extends Mapper> getMapperClass() {
+    return mapperClass;
+  }
+
+  public long getLength() throws IOException, InterruptedException {
+    return inputSplit.getLength();
+  }
+
+  public String[] getLocations() throws IOException, InterruptedException {
+    return inputSplit.getLocations();
+  }
+
+  @SuppressWarnings("unchecked")
+  public void readFields(DataInput in) throws IOException {
+    inputSplitClass = (Class<? extends InputSplit>) readClass(in);
+    inputFormatClass = (Class<? extends InputFormat<?, ?>>) readClass(in);
+    mapperClass = (Class<? extends Mapper<?, ?, ?, ?>>) readClass(in);
+    inputSplit = (InputSplit) ReflectionUtils
+       .newInstance(inputSplitClass, conf);
+    SerializationFactory factory = new SerializationFactory(conf);
+    Deserializer deserializer = factory.getDeserializer(inputSplitClass);
+    deserializer.open((DataInputStream)in);
+    inputSplit = (InputSplit)deserializer.deserialize(inputSplit);
+  }
+
+  private Class<?> readClass(DataInput in) throws IOException {
+    String className = Text.readString(in);
+    try {
+      return conf.getClassByName(className);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException("readObject can't find class", e);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, inputSplitClass.getName());
+    Text.writeString(out, inputFormatClass.getName());
+    Text.writeString(out, mapperClass.getName());
+    SerializationFactory factory = new SerializationFactory(conf);
+    Serializer serializer = 
+          factory.getSerializer(inputSplitClass);
+    serializer.open((DataOutputStream)out);
+    serializer.serialize(inputSplit);
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+}

+ 345 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/jobcontrol/ControlledJob.java

@@ -0,0 +1,345 @@
+/**
+ * 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.jobcontrol;
+
+
+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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.util.StringUtils;
+
+/** 
+ *  This class encapsulates a MapReduce job and its dependency. It monitors 
+ *  the states of the depending jobs and updates the state of this job.
+ *  A job starts in the WAITING state. If it does not have any depending jobs,
+ *  or all of the depending jobs are in SUCCESS state, then the job state 
+ *  will become READY. If any depending jobs fail, the job will fail too. 
+ *  When in READY state, the job can be submitted to Hadoop for execution, with
+ *  the state changing into RUNNING state. From RUNNING state, the job 
+ *  can get into SUCCESS or FAILED state, depending 
+ *  the status of the job execution.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ControlledJob {
+  private static final Log LOG = LogFactory.getLog(ControlledJob.class);
+
+  // A job will be in one of the following states
+  public static enum State {SUCCESS, WAITING, RUNNING, READY, FAILED,
+                            DEPENDENT_FAILED}; 
+  public static final String CREATE_DIR = "mapreduce.jobcontrol.createdir.ifnotexist";
+  private State state;
+  private String controlID;     // assigned and used by JobControl class
+  private Job job;               // mapreduce job to be executed.
+  // some info for human consumption, e.g. the reason why the job failed
+  private String message;
+  // the jobs the current job depends on
+  private List<ControlledJob> dependingJobs;
+	
+  /** 
+   * Construct a job.
+   * @param job a mapreduce job to be executed.
+   * @param dependingJobs an array of jobs the current job depends on
+   */
+  public ControlledJob(Job job, List<ControlledJob> dependingJobs) 
+      throws IOException {
+    this.job = job;
+    this.dependingJobs = dependingJobs;
+    this.state = State.WAITING;
+    this.controlID = "unassigned";
+    this.message = "just initialized";
+  }
+  
+  /**
+   * Construct a job.
+   * 
+   * @param conf mapred job configuration representing a job to be executed.
+   * @throws IOException
+   */
+  public ControlledJob(Configuration conf) throws IOException {
+    this(new Job(conf), null);
+  }
+	
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("job name:\t").append(this.job.getJobName()).append("\n");
+    sb.append("job id:\t").append(this.controlID).append("\n");
+    sb.append("job state:\t").append(this.state).append("\n");
+    sb.append("job mapred id:\t").append(this.job.getJobID()).append("\n");
+    sb.append("job message:\t").append(this.message).append("\n");
+		
+    if (this.dependingJobs == null || this.dependingJobs.size() == 0) {
+      sb.append("job has no depending job:\t").append("\n");
+    } else {
+      sb.append("job has ").append(this.dependingJobs.size()).
+         append(" dependeng jobs:\n");
+      for (int i = 0; i < this.dependingJobs.size(); i++) {
+        sb.append("\t depending job ").append(i).append(":\t");
+        sb.append((this.dependingJobs.get(i)).getJobName()).append("\n");
+      }
+    }
+    return sb.toString();
+  }
+	
+  /**
+   * @return the job name of this job
+   */
+  public String getJobName() {
+    return job.getJobName();
+  }
+	
+  /**
+   * Set the job name for  this job.
+   * @param jobName the job name
+   */
+  public void setJobName(String jobName) {
+    job.setJobName(jobName);
+  }
+	
+  /**
+   * @return the job ID of this job assigned by JobControl
+   */
+  public String getJobID() {
+    return this.controlID;
+  }
+	
+  /**
+   * Set the job ID for  this job.
+   * @param id the job ID
+   */
+  public void setJobID(String id) {
+    this.controlID = id;
+  }
+	
+  /**
+   * @return the mapred ID of this job as assigned by the 
+   * mapred framework.
+   */
+  public JobID getMapredJobID() {
+    return this.job.getJobID();
+  }
+  
+  /**
+   * @return the mapreduce job 
+   */
+  public synchronized Job getJob() {
+    return this.job;
+  }
+
+  /**
+   * Set the mapreduce job
+   * @param job the mapreduce job for this job.
+   */
+  public synchronized void setJob(Job job) {
+    this.job = job;
+  }
+
+  /**
+   * @return the state of this job
+   */
+  public synchronized State getJobState() {
+    return this.state;
+  }
+	
+  /**
+   * Set the state for this job.
+   * @param state the new state for this job.
+   */
+  protected synchronized void setJobState(State state) {
+    this.state = state;
+  }
+	
+  /**
+   * @return the message of this job
+   */
+  public synchronized String getMessage() {
+    return this.message;
+  }
+
+  /**
+   * Set the message for this job.
+   * @param message the message for this job.
+   */
+  public synchronized void setMessage(String message) {
+    this.message = message;
+  }
+
+  /**
+   * @return the depending jobs of this job
+   */
+  public List<ControlledJob> getDependentJobs() {
+    return this.dependingJobs;
+  }
+  
+  /**
+   * Add a job to this jobs' dependency list. 
+   * Dependent jobs can only be added while a Job 
+   * is waiting to run, not during or afterwards.
+   * 
+   * @param dependingJob Job that this Job depends on.
+   * @return <tt>true</tt> if the Job was added.
+   */
+  public synchronized boolean addDependingJob(ControlledJob dependingJob) {
+    if (this.state == State.WAITING) { //only allowed to add jobs when waiting
+      if (this.dependingJobs == null) {
+        this.dependingJobs = new ArrayList<ControlledJob>();
+      }
+      return this.dependingJobs.add(dependingJob);
+    } else {
+      return false;
+    }
+  }
+	
+  /**
+   * @return true if this job is in a complete state
+   */
+  public synchronized boolean isCompleted() {
+    return this.state == State.FAILED || 
+      this.state == State.DEPENDENT_FAILED ||
+      this.state == State.SUCCESS;
+  }
+	
+  /**
+   * @return true if this job is in READY state
+   */
+  public synchronized boolean isReady() {
+    return this.state == State.READY;
+  }
+
+  public void killJob() throws IOException, InterruptedException {
+    job.killJob();
+  }
+  
+  public synchronized void failJob(String message) throws IOException, InterruptedException {
+    try {
+      if(job != null && this.state == State.RUNNING) {
+        job.killJob();
+      }
+    } finally {
+      this.state = State.FAILED;
+      this.message = message;
+    }
+  }
+  
+  /**
+   * Check the state of this running job. The state may 
+   * remain the same, become SUCCESS or FAILED.
+   */
+  private void checkRunningState() throws IOException, InterruptedException {
+    try {
+      if (job.isComplete()) {
+        if (job.isSuccessful()) {
+          this.state = State.SUCCESS;
+        } else {
+          this.state = State.FAILED;
+          this.message = "Job failed!";
+        }
+      }
+    } catch (IOException ioe) {
+      this.state = State.FAILED;
+      this.message = StringUtils.stringifyException(ioe);
+      try {
+        if (job != null) {
+          job.killJob();
+        }
+      } catch (IOException e) {}
+    }
+  }
+	
+  /**
+   * Check and update the state of this job. The state changes  
+   * depending on its current state and the states of the depending jobs.
+   */
+   synchronized State checkState() throws IOException, InterruptedException {
+    if (this.state == State.RUNNING) {
+      checkRunningState();
+    }
+    if (this.state != State.WAITING) {
+      return this.state;
+    }
+    if (this.dependingJobs == null || this.dependingJobs.size() == 0) {
+      this.state = State.READY;
+      return this.state;
+    }
+    ControlledJob pred = null;
+    int n = this.dependingJobs.size();
+    for (int i = 0; i < n; i++) {
+      pred = this.dependingJobs.get(i);
+      State s = pred.checkState();
+      if (s == State.WAITING || s == State.READY || s == State.RUNNING) {
+        break; // a pred is still not completed, continue in WAITING
+        // state
+      }
+      if (s == State.FAILED || s == State.DEPENDENT_FAILED) {
+        this.state = State.DEPENDENT_FAILED;
+        this.message = "depending job " + i + " with jobID "
+          + pred.getJobID() + " failed. " + pred.getMessage();
+        break;
+      }
+      // pred must be in success state
+      if (i == n - 1) {
+        this.state = State.READY;
+      }
+    }
+
+    return this.state;
+  }
+	
+  /**
+   * Submit this job to mapred. The state becomes RUNNING if submission 
+   * is successful, FAILED otherwise.  
+   */
+  protected synchronized void submit() {
+    try {
+      Configuration conf = job.getConfiguration();
+      if (conf.getBoolean(CREATE_DIR, false)) {
+        FileSystem fs = FileSystem.get(conf);
+        Path inputPaths[] = FileInputFormat.getInputPaths(job);
+        for (int i = 0; i < inputPaths.length; i++) {
+          if (!fs.exists(inputPaths[i])) {
+            try {
+              fs.mkdirs(inputPaths[i]);
+            } catch (IOException e) {
+
+            }
+          }
+        }
+      }
+      job.submit();
+      this.state = State.RUNNING;
+    } catch (Exception ioe) {
+      LOG.info(getJobName()+" got an error while submitting ",ioe);
+      this.state = State.FAILED;
+      this.message = StringUtils.stringifyException(ioe);
+    }
+  }
+	
+}

+ 284 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/jobcontrol/JobControl.java

@@ -0,0 +1,284 @@
+/**
+ * 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.jobcontrol;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob.State;
+import org.apache.hadoop.util.StringUtils;
+
+/** 
+ *  This class encapsulates a set of MapReduce jobs and its dependency.
+ *   
+ *  It tracks the states of the jobs by placing them into different tables
+ *  according to their states. 
+ *  
+ *  This class provides APIs for the client app to add a job to the group 
+ *  and to get the jobs in the group in different states. When a job is 
+ *  added, an ID unique to the group is assigned to the job. 
+ *  
+ *  This class has a thread that submits jobs when they become ready, 
+ *  monitors the states of the running jobs, and updates the states of jobs
+ *  based on the state changes of their depending jobs states. The class 
+ *  provides APIs for suspending/resuming the thread, and 
+ *  for stopping the thread.
+ *  
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class JobControl implements Runnable {
+  private static final Log LOG = LogFactory.getLog(JobControl.class);
+
+  // The thread can be in one of the following state
+  public static enum ThreadState {RUNNING, SUSPENDED,STOPPED, STOPPING, READY};
+	
+  private ThreadState runnerState;			// the thread state
+	
+  private LinkedList<ControlledJob> jobsInProgress = new LinkedList<ControlledJob>();
+  private LinkedList<ControlledJob> successfulJobs = new LinkedList<ControlledJob>();
+  private LinkedList<ControlledJob> failedJobs = new LinkedList<ControlledJob>();
+	
+  private long nextJobID;
+  private String groupName;
+	
+  /** 
+   * Construct a job control for a group of jobs.
+   * @param groupName a name identifying this group
+   */
+  public JobControl(String groupName) {
+    this.nextJobID = -1;
+    this.groupName = groupName;
+    this.runnerState = ThreadState.READY;
+  }
+	
+  private static List<ControlledJob> toList(
+                   LinkedList<ControlledJob> jobs) {
+    ArrayList<ControlledJob> retv = new ArrayList<ControlledJob>();
+    synchronized (jobs) {
+      for (ControlledJob job : jobs) {
+        retv.add(job);
+      }
+    }
+    return retv;
+  }
+	
+  synchronized private List<ControlledJob> getJobsIn(State state) {
+    LinkedList<ControlledJob> l = new LinkedList<ControlledJob>();
+    for(ControlledJob j: jobsInProgress) {
+      if(j.getJobState() == state) {
+        l.add(j);
+      }
+    }
+    return l;
+  }
+  
+  /**
+   * @return the jobs in the waiting state
+   */
+  public List<ControlledJob> getWaitingJobList() {
+    return getJobsIn(State.WAITING);
+  }
+	
+  /**
+   * @return the jobs in the running state
+   */
+  public List<ControlledJob> getRunningJobList() {
+    return getJobsIn(State.RUNNING);
+  }
+	
+  /**
+   * @return the jobs in the ready state
+   */
+  public List<ControlledJob> getReadyJobsList() {
+    return getJobsIn(State.READY);
+  }
+	
+  /**
+   * @return the jobs in the success state
+   */
+  public List<ControlledJob> getSuccessfulJobList() {
+    return toList(this.successfulJobs);
+  }
+	
+  public List<ControlledJob> getFailedJobList() {
+    return toList(this.failedJobs);
+  }
+	
+  private String getNextJobID() {
+    nextJobID += 1;
+    return this.groupName + this.nextJobID;
+  }
+
+  /**
+   * Add a new job.
+   * @param aJob the new job
+   */
+  synchronized public String addJob(ControlledJob aJob) {
+    String id = this.getNextJobID();
+    aJob.setJobID(id);
+    aJob.setJobState(State.WAITING);
+    jobsInProgress.add(aJob);
+    return id;	
+  }
+	
+  /**
+   * Add a collection of jobs
+   * 
+   * @param jobs
+   */
+  public void addJobCollection(Collection<ControlledJob> jobs) {
+    for (ControlledJob job : jobs) {
+      addJob(job);
+    }
+  }
+	
+  /**
+   * @return the thread state
+   */
+  public ThreadState getThreadState() {
+    return this.runnerState;
+  }
+	
+  /**
+   * set the thread state to STOPPING so that the 
+   * thread will stop when it wakes up.
+   */
+  public void stop() {
+    this.runnerState = ThreadState.STOPPING;
+  }
+	
+  /**
+   * suspend the running thread
+   */
+  public void suspend () {
+    if (this.runnerState == ThreadState.RUNNING) {
+      this.runnerState = ThreadState.SUSPENDED;
+    }
+  }
+	
+  /**
+   * resume the suspended thread
+   */
+  public void resume () {
+    if (this.runnerState == ThreadState.SUSPENDED) {
+      this.runnerState = ThreadState.RUNNING;
+    }
+  }
+	
+  synchronized public boolean allFinished() {
+    return jobsInProgress.isEmpty();
+  }
+	
+  /**
+   *  The main loop for the thread.
+   *  The loop does the following:
+   *  	Check the states of the running jobs
+   *  	Update the states of waiting jobs
+   *  	Submit the jobs in ready state
+   */
+  public void run() {
+    try {
+      this.runnerState = ThreadState.RUNNING;
+      while (true) {
+        while (this.runnerState == ThreadState.SUSPENDED) {
+          try {
+            Thread.sleep(5000);
+          }
+          catch (Exception e) {
+            //TODO the thread was interrupted, do something!!!
+          }
+        }
+        
+        synchronized(this) {
+          Iterator<ControlledJob> it = jobsInProgress.iterator();
+          while(it.hasNext()) {
+            ControlledJob j = it.next();
+            LOG.debug("Checking state of job "+j);
+            switch(j.checkState()) {
+            case SUCCESS:
+              successfulJobs.add(j);
+              it.remove();
+              break;
+            case FAILED:
+            case DEPENDENT_FAILED:
+              failedJobs.add(j);
+              it.remove();
+              break;
+            case READY:
+              j.submit();
+              break;
+            case RUNNING:
+            case WAITING:
+              //Do Nothing
+              break;
+            }
+          }
+        }
+        
+        if (this.runnerState != ThreadState.RUNNING && 
+            this.runnerState != ThreadState.SUSPENDED) {
+          break;
+        }
+        try {
+          Thread.sleep(5000);
+        }
+        catch (Exception e) {
+          //TODO the thread was interrupted, do something!!!
+        }
+        if (this.runnerState != ThreadState.RUNNING && 
+            this.runnerState != ThreadState.SUSPENDED) {
+          break;
+        }
+      }
+    }catch(Throwable t) {
+      LOG.error("Error while trying to run jobs.",t);
+      //Mark all jobs as failed because we got something bad.
+      failAllJobs(t);
+    }
+    this.runnerState = ThreadState.STOPPED;
+  }
+
+  synchronized private void failAllJobs(Throwable t) {
+    String message = "Unexpected System Error Occured: "+
+    StringUtils.stringifyException(t);
+    Iterator<ControlledJob> it = jobsInProgress.iterator();
+    while(it.hasNext()) {
+      ControlledJob j = it.next();
+      try {
+        j.failJob(message);
+      } catch (IOException e) {
+        LOG.error("Error while tyring to clean up "+j.getJobName(), e);
+      } catch (InterruptedException e) {
+        LOG.error("Error while tyring to clean up "+j.getJobName(), e);
+      } finally {
+        failedJobs.add(j);
+        it.remove();
+      }
+    }
+  }
+}

+ 19 - 2
src/mapred/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java

@@ -39,6 +39,9 @@ import org.apache.hadoop.mapreduce.security.TokenCache;
 
 /** A base class for {@link OutputFormat}s that read from {@link FileSystem}s.*/
 public abstract class FileOutputFormat<K, V> extends OutputFormat<K, V> {
+  
+  protected static final String BASE_OUTPUT_NAME = "mapreduce.output.basename";
+  protected static final String PART = "part";
 
   public static enum Counter { 
     BYTES_WRITTEN
@@ -263,8 +266,22 @@ public abstract class FileOutputFormat<K, V> extends OutputFormat<K, V> {
                                  String extension) throws IOException{
     FileOutputCommitter committer = 
       (FileOutputCommitter) getOutputCommitter(context);
-    return new Path(committer.getWorkPath(), getUniqueFile(context, "part", 
-                                                           extension));
+    return new Path(committer.getWorkPath(), getUniqueFile(context, 
+        getOutputName(context), extension));
+  }
+  
+  /**
+   * Get the base output name for the output file.
+   */
+  protected static String getOutputName(JobContext job) {
+    return job.getConfiguration().get(BASE_OUTPUT_NAME, PART);
+  }
+
+  /**
+   * Set the base output name for output file to be created.
+   */
+  protected static void setOutputName(JobContext job, String name) {
+    job.getConfiguration().set(BASE_OUTPUT_NAME, name);
   }
 
   public synchronized 

+ 111 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/output/FilterOutputFormat.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.lib.output;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * FilterOutputFormat is a convenience class that wraps OutputFormat. 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class FilterOutputFormat <K,V> extends OutputFormat<K, V> {
+
+  protected OutputFormat<K,V> baseOut;
+
+  public FilterOutputFormat() {
+    this.baseOut = null;
+  }
+  
+  /**
+   * Create a FilterOutputFormat based on the underlying output format.
+   * @param baseOut the underlying OutputFormat
+   */
+  public FilterOutputFormat(OutputFormat<K,V> baseOut) {
+    this.baseOut = baseOut;
+  }
+
+  @Override
+  public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context) 
+  throws IOException, InterruptedException {
+    return getBaseOut().getRecordWriter(context);
+  }
+
+  @Override
+  public void checkOutputSpecs(JobContext context) 
+  throws IOException, InterruptedException {
+    getBaseOut().checkOutputSpecs(context);
+  }
+
+  @Override
+  public OutputCommitter getOutputCommitter(TaskAttemptContext context) 
+  throws IOException, InterruptedException {
+    return getBaseOut().getOutputCommitter(context);
+  }
+
+  private OutputFormat<K,V> getBaseOut() throws IOException {
+    if (baseOut == null) {
+      throw new IOException("OutputFormat not set for FilterOutputFormat");
+    }
+    return baseOut;
+  }
+  /**
+   * <code>FilterRecordWriter</code> is a convenience wrapper
+   * class that extends the {@link RecordWriter}.
+   */
+
+  public static class FilterRecordWriter<K,V> extends RecordWriter<K,V> {
+
+    protected RecordWriter<K,V> rawWriter = null;
+
+    public FilterRecordWriter() {
+      rawWriter = null;
+    }
+    
+    public FilterRecordWriter(RecordWriter<K,V> rwriter) {
+      this.rawWriter = rwriter;
+    }
+    
+    @Override
+    public void write(K key, V value) throws IOException, InterruptedException {
+      getRawWriter().write(key, value);
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) 
+    throws IOException, InterruptedException {
+      getRawWriter().close(context);
+    }
+    
+    private RecordWriter<K,V> getRawWriter() throws IOException {
+      if (rawWriter == null) {
+        throw new IOException("Record Writer not set for FilterRecordWriter");
+      }
+      return rawWriter;
+    }
+  }
+}

+ 124 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/output/LazyOutputFormat.java

@@ -0,0 +1,124 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * A Convenience class that creates output lazily.  
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class LazyOutputFormat <K,V> extends FilterOutputFormat<K, V> {
+  public static String OUTPUT_FORMAT = 
+    "mapreduce.output.lazyoutputformat.outputformat";
+  /**
+   * Set the underlying output format for LazyOutputFormat.
+   * @param job the {@link Job} to modify
+   * @param theClass the underlying class
+   */
+  @SuppressWarnings("unchecked")
+  public static void  setOutputFormatClass(Job job, 
+                                     Class<? extends OutputFormat> theClass) {
+      job.setOutputFormatClass(LazyOutputFormat.class);
+      job.getConfiguration().setClass(OUTPUT_FORMAT, 
+          theClass, OutputFormat.class);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void getBaseOutputFormat(Configuration conf) 
+  throws IOException {
+    baseOut =  ((OutputFormat<K, V>) ReflectionUtils.newInstance(
+      conf.getClass(OUTPUT_FORMAT, null), conf));
+    if (baseOut == null) {
+      throw new IOException("Output Format not set for LazyOutputFormat");
+    }
+  }
+
+  @Override
+  public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
+  throws IOException, InterruptedException {
+    if (baseOut == null) {
+      getBaseOutputFormat(context.getConfiguration());
+    }
+    return new LazyRecordWriter<K, V>(baseOut, context);
+  }
+  
+  @Override
+  public void checkOutputSpecs(JobContext context) 
+  throws IOException, InterruptedException {
+    if (baseOut == null) {
+      getBaseOutputFormat(context.getConfiguration());
+    }
+   super.checkOutputSpecs(context);
+  }
+  
+  @Override
+  public OutputCommitter getOutputCommitter(TaskAttemptContext context) 
+  throws IOException, InterruptedException {
+    if (baseOut == null) {
+      getBaseOutputFormat(context.getConfiguration());
+    }
+    return super.getOutputCommitter(context);
+  }
+  
+  /**
+   * A convenience class to be used with LazyOutputFormat
+   */
+  private static class LazyRecordWriter<K,V> extends FilterRecordWriter<K,V> {
+
+    final OutputFormat<K,V> outputFormat;
+    final TaskAttemptContext taskContext;
+
+    public LazyRecordWriter(OutputFormat<K,V> out, 
+                            TaskAttemptContext taskContext)
+    throws IOException, InterruptedException {
+      this.outputFormat = out;
+      this.taskContext = taskContext;
+    }
+
+    @Override
+    public void write(K key, V value) throws IOException, InterruptedException {
+      if (rawWriter == null) {
+        rawWriter = outputFormat.getRecordWriter(taskContext);
+      }
+      rawWriter.write(key, value);
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) 
+    throws IOException, InterruptedException {
+      if (rawWriter != null) {
+        rawWriter.close(context);
+      }
+    }
+
+  }
+}

+ 468 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java

@@ -0,0 +1,468 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * The MultipleOutputs class simplifies writing output data 
+ * to multiple outputs
+ * 
+ * <p> 
+ * Case one: writing to additional outputs other than the job default output.
+ *
+ * Each additional output, or named output, may be configured with its own
+ * <code>OutputFormat</code>, with its own key class and with its own value
+ * class.
+ * 
+ * <p>
+ * Case two: to write data to different files provided by user
+ * </p>
+ * 
+ * <p>
+ * MultipleOutputs supports counters, by default they are disabled. The 
+ * counters group is the {@link MultipleOutputs} class name. The names of the 
+ * counters are the same as the output name. These count the number records 
+ * written to each output name.
+ * </p>
+ * 
+ * Usage pattern for job submission:
+ * <pre>
+ *
+ * Job job = new Job();
+ *
+ * FileInputFormat.setInputPath(job, inDir);
+ * FileOutputFormat.setOutputPath(job, outDir);
+ *
+ * job.setMapperClass(MOMap.class);
+ * job.setReducerClass(MOReduce.class);
+ * ...
+ *
+ * // Defines additional single text based output 'text' for the job
+ * MultipleOutputs.addNamedOutput(job, "text", TextOutputFormat.class,
+ * LongWritable.class, Text.class);
+ *
+ * // Defines additional sequence-file based output 'sequence' for the job
+ * MultipleOutputs.addNamedOutput(job, "seq",
+ *   SequenceFileOutputFormat.class,
+ *   LongWritable.class, Text.class);
+ * ...
+ *
+ * job.waitForCompletion(true);
+ * ...
+ * </pre>
+ * <p>
+ * Usage in Reducer:
+ * <pre>
+ * <K, V> String generateFileName(K k, V v) {
+ *   return k.toString() + "_" + v.toString();
+ * }
+ * 
+ * public class MOReduce extends
+ *   Reducer&lt;WritableComparable, Writable,WritableComparable, Writable&gt; {
+ * private MultipleOutputs mos;
+ * public void setup(Context context) {
+ * ...
+ * mos = new MultipleOutputs(context);
+ * }
+ *
+ * public void reduce(WritableComparable key, Iterator&lt;Writable&gt; values,
+ * Context context)
+ * throws IOException {
+ * ...
+ * mos.write("text", , key, new Text("Hello"));
+ * mos.write("seq", LongWritable(1), new Text("Bye"), "seq_a");
+ * mos.write("seq", LongWritable(2), key, new Text("Chau"), "seq_b");
+ * mos.write(key, new Text("value"), generateFileName(key, new Text("value")));
+ * ...
+ * }
+ *
+ * public void cleanup(Context) throws IOException {
+ * mos.close();
+ * ...
+ * }
+ *
+ * }
+ * </pre>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class MultipleOutputs<KEYOUT, VALUEOUT> {
+
+  private static final String MULTIPLE_OUTPUTS = "mapreduce.multipleoutputs";
+
+  private static final String MO_PREFIX = 
+    "mapreduce.multipleoutputs.namedOutput.";
+
+  private static final String FORMAT = ".format";
+  private static final String KEY = ".key";
+  private static final String VALUE = ".value";
+  private static final String COUNTERS_ENABLED = 
+    "mapreduce.multipleoutputs.counters";
+
+  /**
+   * Counters group used by the counters of MultipleOutputs.
+   */
+  private static final String COUNTERS_GROUP = MultipleOutputs.class.getName();
+
+  /**
+   * Cache for the taskContexts
+   */
+  private Map<String, TaskAttemptContext> taskContexts = new HashMap<String, TaskAttemptContext>();
+  /**
+   * Cached TaskAttemptContext which uses the job's configured settings
+   */
+  private TaskAttemptContext jobOutputFormatContext;
+
+  /**
+   * Checks if a named output name is valid token.
+   *
+   * @param namedOutput named output Name
+   * @throws IllegalArgumentException if the output name is not valid.
+   */
+  private static void checkTokenName(String namedOutput) {
+    if (namedOutput == null || namedOutput.length() == 0) {
+      throw new IllegalArgumentException(
+        "Name cannot be NULL or emtpy");
+    }
+    for (char ch : namedOutput.toCharArray()) {
+      if ((ch >= 'A') && (ch <= 'Z')) {
+        continue;
+      }
+      if ((ch >= 'a') && (ch <= 'z')) {
+        continue;
+      }
+      if ((ch >= '0') && (ch <= '9')) {
+        continue;
+      }
+      throw new IllegalArgumentException(
+        "Name cannot be have a '" + ch + "' char");
+    }
+  }
+
+  /**
+   * Checks if output name is valid.
+   *
+   * name cannot be the name used for the default output
+   * @param outputPath base output Name
+   * @throws IllegalArgumentException if the output name is not valid.
+   */
+  private static void checkBaseOutputPath(String outputPath) {
+    if (outputPath.equals(FileOutputFormat.PART)) {
+      throw new IllegalArgumentException("output name cannot be 'part'");
+    }
+  }
+  
+  /**
+   * Checks if a named output name is valid.
+   *
+   * @param namedOutput named output Name
+   * @throws IllegalArgumentException if the output name is not valid.
+   */
+  private static void checkNamedOutputName(JobContext job,
+      String namedOutput, boolean alreadyDefined) {
+    checkTokenName(namedOutput);
+    checkBaseOutputPath(namedOutput);
+    List<String> definedChannels = getNamedOutputsList(job);
+    if (alreadyDefined && definedChannels.contains(namedOutput)) {
+      throw new IllegalArgumentException("Named output '" + namedOutput +
+        "' already alreadyDefined");
+    } else if (!alreadyDefined && !definedChannels.contains(namedOutput)) {
+      throw new IllegalArgumentException("Named output '" + namedOutput +
+        "' not defined");
+    }
+  }
+
+  // Returns list of channel names.
+  private static List<String> getNamedOutputsList(JobContext job) {
+    List<String> names = new ArrayList<String>();
+    StringTokenizer st = new StringTokenizer(
+      job.getConfiguration().get(MULTIPLE_OUTPUTS, ""), " ");
+    while (st.hasMoreTokens()) {
+      names.add(st.nextToken());
+    }
+    return names;
+  }
+
+  // Returns the named output OutputFormat.
+  @SuppressWarnings("unchecked")
+  private static Class<? extends OutputFormat<?, ?>> getNamedOutputFormatClass(
+    JobContext job, String namedOutput) {
+    return (Class<? extends OutputFormat<?, ?>>)
+      job.getConfiguration().getClass(MO_PREFIX + namedOutput + FORMAT, null,
+      OutputFormat.class);
+  }
+
+  // Returns the key class for a named output.
+  private static Class<?> getNamedOutputKeyClass(JobContext job,
+                                                String namedOutput) {
+    return job.getConfiguration().getClass(MO_PREFIX + namedOutput + KEY, null,
+      Object.class);
+  }
+
+  // Returns the value class for a named output.
+  private static Class<?> getNamedOutputValueClass(
+      JobContext job, String namedOutput) {
+    return job.getConfiguration().getClass(MO_PREFIX + namedOutput + VALUE,
+      null, Object.class);
+  }
+
+  /**
+   * Adds a named output for the job.
+   * <p/>
+   *
+   * @param job               job to add the named output
+   * @param namedOutput       named output name, it has to be a word, letters
+   *                          and numbers only, cannot be the word 'part' as
+   *                          that is reserved for the default output.
+   * @param outputFormatClass OutputFormat class.
+   * @param keyClass          key class
+   * @param valueClass        value class
+   */
+  @SuppressWarnings("unchecked")
+  public static void addNamedOutput(Job job, String namedOutput,
+      Class<? extends OutputFormat> outputFormatClass,
+      Class<?> keyClass, Class<?> valueClass) {
+    checkNamedOutputName(job, namedOutput, true);
+    Configuration conf = job.getConfiguration();
+    conf.set(MULTIPLE_OUTPUTS,
+      conf.get(MULTIPLE_OUTPUTS, "") + " " + namedOutput);
+    conf.setClass(MO_PREFIX + namedOutput + FORMAT, outputFormatClass,
+      OutputFormat.class);
+    conf.setClass(MO_PREFIX + namedOutput + KEY, keyClass, Object.class);
+    conf.setClass(MO_PREFIX + namedOutput + VALUE, valueClass, Object.class);
+  }
+
+  /**
+   * Enables or disables counters for the named outputs.
+   * 
+   * The counters group is the {@link MultipleOutputs} class name.
+   * The names of the counters are the same as the named outputs. These
+   * counters count the number records written to each output name.
+   * By default these counters are disabled.
+   *
+   * @param job    job  to enable counters
+   * @param enabled indicates if the counters will be enabled or not.
+   */
+  public static void setCountersEnabled(Job job, boolean enabled) {
+    job.getConfiguration().setBoolean(COUNTERS_ENABLED, enabled);
+  }
+
+  /**
+   * Returns if the counters for the named outputs are enabled or not.
+   * By default these counters are disabled.
+   *
+   * @param job    the job 
+   * @return TRUE if the counters are enabled, FALSE if they are disabled.
+   */
+  public static boolean getCountersEnabled(JobContext job) {
+    return job.getConfiguration().getBoolean(COUNTERS_ENABLED, false);
+  }
+
+  /**
+   * Wraps RecordWriter to increment counters. 
+   */
+  @SuppressWarnings("unchecked")
+  private static class RecordWriterWithCounter extends RecordWriter {
+    private RecordWriter writer;
+    private String counterName;
+    private TaskInputOutputContext context;
+
+    public RecordWriterWithCounter(RecordWriter writer, String counterName,
+                                   TaskInputOutputContext context) {
+      this.writer = writer;
+      this.counterName = counterName;
+      this.context = context;
+    }
+
+    @SuppressWarnings({"unchecked"})
+    public void write(Object key, Object value) 
+        throws IOException, InterruptedException {
+      context.getCounter(COUNTERS_GROUP, counterName).increment(1);
+      writer.write(key, value);
+    }
+
+    public void close(TaskAttemptContext context) 
+        throws IOException, InterruptedException {
+      writer.close(context);
+    }
+  }
+
+  // instance code, to be used from Mapper/Reducer code
+
+  private TaskInputOutputContext<?, ?, KEYOUT, VALUEOUT> context;
+  private Set<String> namedOutputs;
+  private Map<String, RecordWriter<?, ?>> recordWriters;
+  private boolean countersEnabled;
+  
+  /**
+   * Creates and initializes multiple outputs support,
+   * it should be instantiated in the Mapper/Reducer setup method.
+   *
+   * @param context the TaskInputOutputContext object
+   */
+  public MultipleOutputs(
+      TaskInputOutputContext<?, ?, KEYOUT, VALUEOUT> context) {
+    this.context = context;
+    namedOutputs = Collections.unmodifiableSet(
+      new HashSet<String>(MultipleOutputs.getNamedOutputsList(context)));
+    recordWriters = new HashMap<String, RecordWriter<?, ?>>();
+    countersEnabled = getCountersEnabled(context);
+  }
+
+  /**
+   * Write key and value to the namedOutput.
+   *
+   * Output path is a unique file generated for the namedOutput.
+   * For example, {namedOutput}-(m|r)-{part-number}
+   * 
+   * @param namedOutput the named output name
+   * @param key         the key
+   * @param value       the value
+   */
+  @SuppressWarnings("unchecked")
+  public <K, V> void write(String namedOutput, K key, V value)
+      throws IOException, InterruptedException {
+    write(namedOutput, key, value, namedOutput);
+  }
+
+  /**
+   * Write key and value to baseOutputPath using the namedOutput.
+   * 
+   * @param namedOutput    the named output name
+   * @param key            the key
+   * @param value          the value
+   * @param baseOutputPath base-output path to write the record to.
+   * Note: Framework will generate unique filename for the baseOutputPath
+   */
+  @SuppressWarnings("unchecked")
+  public <K, V> void write(String namedOutput, K key, V value,
+      String baseOutputPath) throws IOException, InterruptedException {
+    checkNamedOutputName(context, namedOutput, false);
+    checkBaseOutputPath(baseOutputPath);
+    if (!namedOutputs.contains(namedOutput)) {
+      throw new IllegalArgumentException("Undefined named output '" +
+        namedOutput + "'");
+    }
+    TaskAttemptContext taskContext = getContext(namedOutput);
+    getRecordWriter(taskContext, baseOutputPath).write(key, value);
+  }
+
+  /**
+   * Write key value to an output file name.
+   * 
+   * Gets the record writer from job's output format.  
+   * Job's output format should be a FileOutputFormat.
+   * 
+   * @param key       the key
+   * @param value     the value
+   * @param baseOutputPath base-output path to write the record to.
+   * Note: Framework will generate unique filename for the baseOutputPath
+   */
+  @SuppressWarnings("unchecked")
+  public void write(KEYOUT key, VALUEOUT value, String baseOutputPath) 
+      throws IOException, InterruptedException {
+    checkBaseOutputPath(baseOutputPath);
+    if (jobOutputFormatContext == null) {
+      jobOutputFormatContext = 
+        new TaskAttemptContext(context.getConfiguration(), 
+                               context.getTaskAttemptID());
+    }
+    getRecordWriter(jobOutputFormatContext, baseOutputPath).write(key, value);
+  }
+
+  // by being synchronized MultipleOutputTask can be use with a
+  // MultithreadedMapper.
+  @SuppressWarnings("unchecked")
+  private synchronized RecordWriter getRecordWriter(
+      TaskAttemptContext taskContext, String baseFileName) 
+      throws IOException, InterruptedException {
+    
+    // look for record-writer in the cache
+    RecordWriter writer = recordWriters.get(baseFileName);
+    
+    // If not in cache, create a new one
+    if (writer == null) {
+      // get the record writer from context output format
+      FileOutputFormat.setOutputName(taskContext, baseFileName);
+      try {
+        writer = ((OutputFormat) ReflectionUtils.newInstance(
+          taskContext.getOutputFormatClass(), taskContext.getConfiguration()))
+          .getRecordWriter(taskContext);
+      } catch (ClassNotFoundException e) {
+        throw new IOException(e);
+      }
+ 
+      // if counters are enabled, wrap the writer with context 
+      // to increment counters 
+      if (countersEnabled) {
+        writer = new RecordWriterWithCounter(writer, baseFileName, context);
+      }
+      
+      // add the record-writer to the cache
+      recordWriters.put(baseFileName, writer);
+    }
+    return writer;
+  }
+
+   // Create a taskAttemptContext for the named output with 
+   // output format and output key/value types put in the context
+  private TaskAttemptContext getContext(String nameOutput) throws IOException {
+      
+    TaskAttemptContext taskContext = taskContexts.get(nameOutput);
+    
+    if (taskContext != null) {
+        return taskContext;
+    }
+    
+    // The following trick leverages the instantiation of a record writer via
+    // the job thus supporting arbitrary output formats.
+    Job job = new Job(context.getConfiguration());
+    job.setOutputFormatClass(getNamedOutputFormatClass(context, nameOutput));
+    job.setOutputKeyClass(getNamedOutputKeyClass(context, nameOutput));
+    job.setOutputValueClass(getNamedOutputValueClass(context, nameOutput));
+    taskContext = new TaskAttemptContext(job.getConfiguration(), context
+        .getTaskAttemptID());
+
+    taskContexts.put(nameOutput, taskContext);
+
+    return taskContext;
+  }
+
+  /**
+   * Closes all the opened outputs.
+   * 
+   * This should be called from cleanup method of map/reduce task.
+   * If overridden subclasses must invoke <code>super.close()</code> at the
+   * end of their <code>close()</code>
+   * 
+   */
+  @SuppressWarnings("unchecked")
+  public void close() throws IOException, InterruptedException {
+    for (RecordWriter writer : recordWriters.values()) {
+      writer.close(context);
+    }
+  }
+}

+ 198 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/output/SequenceFileAsBinaryOutputFormat.java

@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapreduce.lib.output;
+
+import java.io.IOException;
+import java.io.DataOutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.SequenceFile.ValueBytes;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.mapred.InvalidJobConfException;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/** 
+ * An {@link org.apache.hadoop.mapreduce.OutputFormat} that writes keys, 
+ * values to {@link SequenceFile}s in binary(raw) format
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class SequenceFileAsBinaryOutputFormat 
+    extends SequenceFileOutputFormat <BytesWritable,BytesWritable> {
+  public static String KEY_CLASS = "mapreduce.output.seqbinaryoutputformat.key.class"; 
+  public static String VALUE_CLASS = "mapreduce.output.seqbinaryoutputformat.value.class"; 
+
+  /** 
+   * Inner class used for appendRaw
+   */
+  static public class WritableValueBytes implements ValueBytes {
+    private BytesWritable value;
+
+    public WritableValueBytes() {
+      this.value = null;
+    }
+    
+    public WritableValueBytes(BytesWritable value) {
+      this.value = value;
+    }
+
+    public void reset(BytesWritable value) {
+      this.value = value;
+    }
+
+    public void writeUncompressedBytes(DataOutputStream outStream)
+        throws IOException {
+      outStream.write(value.getBytes(), 0, value.getLength());
+    }
+
+    public void writeCompressedBytes(DataOutputStream outStream)
+        throws IllegalArgumentException, IOException {
+      throw new UnsupportedOperationException(
+        "WritableValueBytes doesn't support RECORD compression"); 
+    }
+    
+    public int getSize(){
+      return value.getLength();
+    }
+  }
+
+  /**
+   * Set the key class for the {@link SequenceFile}
+   * <p>This allows the user to specify the key class to be different 
+   * from the actual class ({@link BytesWritable}) used for writing </p>
+   * 
+   * @param job the {@link Job} to modify
+   * @param theClass the SequenceFile output key class.
+   */
+  static public void setSequenceFileOutputKeyClass(Job job, 
+      Class<?> theClass) {
+    job.getConfiguration().setClass(KEY_CLASS,
+      theClass, Object.class);
+  }
+
+  /**
+   * Set the value class for the {@link SequenceFile}
+   * <p>This allows the user to specify the value class to be different 
+   * from the actual class ({@link BytesWritable}) used for writing </p>
+   * 
+   * @param job the {@link Job} to modify
+   * @param theClass the SequenceFile output key class.
+   */
+  static public void setSequenceFileOutputValueClass(Job job, 
+      Class<?> theClass) {
+    job.getConfiguration().setClass(VALUE_CLASS, 
+                  theClass, Object.class);
+  }
+
+  /**
+   * Get the key class for the {@link SequenceFile}
+   * 
+   * @return the key class of the {@link SequenceFile}
+   */
+  static public Class<? extends WritableComparable> 
+      getSequenceFileOutputKeyClass(JobContext job) { 
+    return job.getConfiguration().getClass(KEY_CLASS,
+      job.getOutputKeyClass().asSubclass(WritableComparable.class), 
+      WritableComparable.class);
+  }
+
+  /**
+   * Get the value class for the {@link SequenceFile}
+   * 
+   * @return the value class of the {@link SequenceFile}
+   */
+  static public Class<? extends Writable> getSequenceFileOutputValueClass(
+      JobContext job) { 
+    return job.getConfiguration().getClass(VALUE_CLASS, 
+      job.getOutputValueClass().asSubclass(Writable.class), Writable.class);
+  }
+  
+  @Override 
+  public RecordWriter<BytesWritable, BytesWritable> getRecordWriter(
+      TaskAttemptContext context) throws IOException {
+    final SequenceFile.Writer out = getSequenceWriter(context,
+      getSequenceFileOutputKeyClass(context),
+      getSequenceFileOutputValueClass(context)); 
+
+    return new RecordWriter<BytesWritable, BytesWritable>() {
+      private WritableValueBytes wvaluebytes = new WritableValueBytes();
+
+      public void write(BytesWritable bkey, BytesWritable bvalue)
+        throws IOException {
+        wvaluebytes.reset(bvalue);
+        out.appendRaw(bkey.getBytes(), 0, bkey.getLength(), wvaluebytes);
+        wvaluebytes.reset(null);
+      }
+
+      public void close(TaskAttemptContext context) throws IOException { 
+        out.close();
+      }
+    };
+  }
+
+  protected SequenceFile.Writer getSequenceWriter(TaskAttemptContext context,
+      Class<?> keyClass, Class<?> valueClass)
+      throws IOException {
+    Configuration conf = context.getConfiguration();
+
+    CompressionCodec codec = null;
+    CompressionType compressionType = CompressionType.NONE;
+    if (getCompressOutput(context)) {
+      // find the kind of compression to do
+      compressionType = getOutputCompressionType(context);
+      // find the right codec
+      Class<?> codecClass = getOutputCompressorClass(context,
+                                                     DefaultCodec.class);
+      codec = (CompressionCodec)
+        ReflectionUtils.newInstance(codecClass, conf);
+    }
+    // get the path of the temporary output file
+    Path file = getDefaultWorkFile(context, "");
+    FileSystem fs = file.getFileSystem(conf);
+    return SequenceFile.createWriter(fs, conf, file,
+             keyClass,
+             valueClass,
+             compressionType,
+             codec,
+             context);
+  }
+
+  @Override 
+  public void checkOutputSpecs(JobContext job) throws IOException {
+    super.checkOutputSpecs(job);
+    if (getCompressOutput(job) && 
+        getOutputCompressionType(job) == CompressionType.RECORD ) {
+      throw new InvalidJobConfException("SequenceFileAsBinaryOutputFormat "
+        + "doesn't support Record Compression" );
+    }
+  }
+}

+ 142 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java

@@ -0,0 +1,142 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapreduce.Partitioner;
+
+/**
+ * <p>Partition {@link BinaryComparable} keys using a configurable part of 
+ * the bytes array returned by {@link BinaryComparable#getBytes()}.</p>
+ * 
+ * <p>The subarray to be used for the partitioning can be defined by means
+ * of the following properties:
+ * <ul>
+ *   <li>
+ *     <i>mapred.binary.partitioner.left.offset</i>:
+ *     left offset in array (0 by default)
+ *   </li>
+ *   <li>
+ *     <i>mapred.binary.partitioner.right.offset</i>: 
+ *     right offset in array (-1 by default)
+ *   </li>
+ * </ul>
+ * Like in Python, both negative and positive offsets are allowed, but
+ * the meaning is slightly different. In case of an array of length 5,
+ * for instance, the possible offsets are:
+ * <pre><code>
+ *  +---+---+---+---+---+
+ *  | B | B | B | B | B |
+ *  +---+---+---+---+---+
+ *    0   1   2   3   4
+ *   -5  -4  -3  -2  -1
+ * </code></pre>
+ * The first row of numbers gives the position of the offsets 0...5 in 
+ * the array; the second row gives the corresponding negative offsets. 
+ * Contrary to Python, the specified subarray has byte <code>i</code> 
+ * and <code>j</code> as first and last element, repectively, when 
+ * <code>i</code> and <code>j</code> are the left and right offset.
+ * 
+ * <p>For Hadoop programs written in Java, it is advisable to use one of 
+ * the following static convenience methods for setting the offsets:
+ * <ul>
+ *   <li>{@link #setOffsets}</li>
+ *   <li>{@link #setLeftOffset}</li>
+ *   <li>{@link #setRightOffset}</li>
+ * </ul></p>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BinaryPartitioner<V> extends Partitioner<BinaryComparable, V> 
+  implements Configurable {
+
+  private static final String LEFT_OFFSET_PROPERTY_NAME = 
+    "mapred.binary.partitioner.left.offset";
+  private static final String RIGHT_OFFSET_PROPERTY_NAME = 
+    "mapred.binary.partitioner.right.offset";
+  
+  /**
+   * Set the subarray to be used for partitioning to 
+   * <code>bytes[left:(right+1)]</code> in Python syntax.
+   * 
+   * @param conf configuration object
+   * @param left left Python-style offset
+   * @param right right Python-style offset
+   */
+  public static void setOffsets(Configuration conf, int left, int right) {
+    conf.setInt(LEFT_OFFSET_PROPERTY_NAME, left);
+    conf.setInt(RIGHT_OFFSET_PROPERTY_NAME, right);
+  }
+  
+  /**
+   * Set the subarray to be used for partitioning to 
+   * <code>bytes[offset:]</code> in Python syntax.
+   * 
+   * @param conf configuration object
+   * @param offset left Python-style offset
+   */
+  public static void setLeftOffset(Configuration conf, int offset) {
+    conf.setInt(LEFT_OFFSET_PROPERTY_NAME, offset);
+  }
+  
+  /**
+   * Set the subarray to be used for partitioning to 
+   * <code>bytes[:(offset+1)]</code> in Python syntax.
+   * 
+   * @param conf configuration object
+   * @param offset right Python-style offset
+   */
+  public static void setRightOffset(Configuration conf, int offset) {
+    conf.setInt(RIGHT_OFFSET_PROPERTY_NAME, offset);
+  }
+  
+  
+  private Configuration conf;
+  private int leftOffset, rightOffset;
+  
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    leftOffset = conf.getInt(LEFT_OFFSET_PROPERTY_NAME, 0);
+    rightOffset = conf.getInt(RIGHT_OFFSET_PROPERTY_NAME, -1);
+  }
+  
+  public Configuration getConf() {
+    return conf;
+  }
+  
+  /** 
+   * Use (the specified slice of the array returned by) 
+   * {@link BinaryComparable#getBytes()} to partition. 
+   */
+  @Override
+  public int getPartition(BinaryComparable key, V value, int numPartitions) {
+    int length = key.getLength();
+    int leftIndex = (leftOffset + length) % length;
+    int rightIndex = (rightOffset + length) % length;
+    int hash = WritableComparator.hashBytes(key.getBytes(), 
+      leftIndex, rightIndex - leftIndex + 1);
+    return (hash & Integer.MAX_VALUE) % numPartitions;
+  }
+  
+}

+ 418 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/partition/InputSampler.java

@@ -0,0 +1,418 @@
+/**
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Utility for collecting samples and writing a partition file for
+ * {@link TotalOrderPartitioner}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class InputSampler<K,V> extends Configured implements Tool  {
+
+  private static final Log LOG = LogFactory.getLog(InputSampler.class);
+
+  static int printUsage() {
+    System.out.println("sampler -r <reduces>\n" +
+      "      [-inFormat <input format class>]\n" +
+      "      [-keyClass <map input & output key class>]\n" +
+      "      [-splitRandom <double pcnt> <numSamples> <maxsplits> | " +
+      "             // Sample from random splits at random (general)\n" +
+      "       -splitSample <numSamples> <maxsplits> | " +
+      "             // Sample from first records in splits (random data)\n"+
+      "       -splitInterval <double pcnt> <maxsplits>]" +
+      "             // Sample from splits at intervals (sorted data)");
+    System.out.println("Default sampler: -splitRandom 0.1 10000 10");
+    ToolRunner.printGenericCommandUsage(System.out);
+    return -1;
+  }
+
+  public InputSampler(Configuration conf) {
+    setConf(conf);
+  }
+
+  /**
+   * Interface to sample using an 
+   * {@link org.apache.hadoop.mapreduce.InputFormat}.
+   */
+  public interface Sampler<K,V> {
+    /**
+     * For a given job, collect and return a subset of the keys from the
+     * input data.
+     */
+    K[] getSample(InputFormat<K,V> inf, Job job) 
+    throws IOException, InterruptedException;
+  }
+
+  /**
+   * Samples the first n records from s splits.
+   * Inexpensive way to sample random data.
+   */
+  public static class SplitSampler<K,V> implements Sampler<K,V> {
+
+    private final int numSamples;
+    private final int maxSplitsSampled;
+
+    /**
+     * Create a SplitSampler sampling <em>all</em> splits.
+     * Takes the first numSamples / numSplits records from each split.
+     * @param numSamples Total number of samples to obtain from all selected
+     *                   splits.
+     */
+    public SplitSampler(int numSamples) {
+      this(numSamples, Integer.MAX_VALUE);
+    }
+
+    /**
+     * Create a new SplitSampler.
+     * @param numSamples Total number of samples to obtain from all selected
+     *                   splits.
+     * @param maxSplitsSampled The maximum number of splits to examine.
+     */
+    public SplitSampler(int numSamples, int maxSplitsSampled) {
+      this.numSamples = numSamples;
+      this.maxSplitsSampled = maxSplitsSampled;
+    }
+
+    /**
+     * From each split sampled, take the first numSamples / numSplits records.
+     */
+    @SuppressWarnings("unchecked") // ArrayList::toArray doesn't preserve type
+    public K[] getSample(InputFormat<K,V> inf, Job job) 
+        throws IOException, InterruptedException {
+      List<InputSplit> splits = inf.getSplits(job);
+      ArrayList<K> samples = new ArrayList<K>(numSamples);
+      int splitsToSample = Math.min(maxSplitsSampled, splits.size());
+      int samplesPerSplit = numSamples / splitsToSample;
+      long records = 0;
+      for (int i = 0; i < splitsToSample; ++i) {
+        TaskAttemptContext samplingContext = new TaskAttemptContext(
+            job.getConfiguration(), new TaskAttemptID());
+        RecordReader<K,V> reader = inf.createRecordReader(
+            splits.get(i), samplingContext);
+        reader.initialize(splits.get(i), samplingContext);
+        while (reader.nextKeyValue()) {
+          samples.add(ReflectionUtils.copy(job.getConfiguration(),
+                                           reader.getCurrentKey(), null));
+          ++records;
+          if ((i+1) * samplesPerSplit <= records) {
+            break;
+          }
+        }
+        reader.close();
+      }
+      return (K[])samples.toArray();
+    }
+  }
+
+  /**
+   * Sample from random points in the input.
+   * General-purpose sampler. Takes numSamples / maxSplitsSampled inputs from
+   * each split.
+   */
+  public static class RandomSampler<K,V> implements Sampler<K,V> {
+    private double freq;
+    private final int numSamples;
+    private final int maxSplitsSampled;
+
+    /**
+     * Create a new RandomSampler sampling <em>all</em> splits.
+     * This will read every split at the client, which is very expensive.
+     * @param freq Probability with which a key will be chosen.
+     * @param numSamples Total number of samples to obtain from all selected
+     *                   splits.
+     */
+    public RandomSampler(double freq, int numSamples) {
+      this(freq, numSamples, Integer.MAX_VALUE);
+    }
+
+    /**
+     * Create a new RandomSampler.
+     * @param freq Probability with which a key will be chosen.
+     * @param numSamples Total number of samples to obtain from all selected
+     *                   splits.
+     * @param maxSplitsSampled The maximum number of splits to examine.
+     */
+    public RandomSampler(double freq, int numSamples, int maxSplitsSampled) {
+      this.freq = freq;
+      this.numSamples = numSamples;
+      this.maxSplitsSampled = maxSplitsSampled;
+    }
+
+    /**
+     * Randomize the split order, then take the specified number of keys from
+     * each split sampled, where each key is selected with the specified
+     * probability and possibly replaced by a subsequently selected key when
+     * the quota of keys from that split is satisfied.
+     */
+    @SuppressWarnings("unchecked") // ArrayList::toArray doesn't preserve type
+    public K[] getSample(InputFormat<K,V> inf, Job job) 
+        throws IOException, InterruptedException {
+      List<InputSplit> splits = inf.getSplits(job);
+      ArrayList<K> samples = new ArrayList<K>(numSamples);
+      int splitsToSample = Math.min(maxSplitsSampled, splits.size());
+
+      Random r = new Random();
+      long seed = r.nextLong();
+      r.setSeed(seed);
+      LOG.debug("seed: " + seed);
+      // shuffle splits
+      for (int i = 0; i < splits.size(); ++i) {
+        InputSplit tmp = splits.get(i);
+        int j = r.nextInt(splits.size());
+        splits.set(i, splits.get(j));
+        splits.set(j, tmp);
+      }
+      // our target rate is in terms of the maximum number of sample splits,
+      // but we accept the possibility of sampling additional splits to hit
+      // the target sample keyset
+      for (int i = 0; i < splitsToSample ||
+                     (i < splits.size() && samples.size() < numSamples); ++i) {
+        TaskAttemptContext samplingContext = new TaskAttemptContext(
+            job.getConfiguration(), new TaskAttemptID());
+        RecordReader<K,V> reader = inf.createRecordReader(
+            splits.get(i), samplingContext);
+        reader.initialize(splits.get(i), samplingContext);
+        while (reader.nextKeyValue()) {
+          if (r.nextDouble() <= freq) {
+            if (samples.size() < numSamples) {
+              samples.add(ReflectionUtils.copy(job.getConfiguration(),
+                                               reader.getCurrentKey(), null));
+            } else {
+              // When exceeding the maximum number of samples, replace a
+              // random element with this one, then adjust the frequency
+              // to reflect the possibility of existing elements being
+              // pushed out
+              int ind = r.nextInt(numSamples);
+              if (ind != numSamples) {
+                samples.set(ind, ReflectionUtils.copy(job.getConfiguration(),
+                                 reader.getCurrentKey(), null));
+              }
+              freq *= (numSamples - 1) / (double) numSamples;
+            }
+          }
+        }
+        reader.close();
+      }
+      return (K[])samples.toArray();
+    }
+  }
+
+  /**
+   * Sample from s splits at regular intervals.
+   * Useful for sorted data.
+   */
+  public static class IntervalSampler<K,V> implements Sampler<K,V> {
+    private final double freq;
+    private final int maxSplitsSampled;
+
+    /**
+     * Create a new IntervalSampler sampling <em>all</em> splits.
+     * @param freq The frequency with which records will be emitted.
+     */
+    public IntervalSampler(double freq) {
+      this(freq, Integer.MAX_VALUE);
+    }
+
+    /**
+     * Create a new IntervalSampler.
+     * @param freq The frequency with which records will be emitted.
+     * @param maxSplitsSampled The maximum number of splits to examine.
+     * @see #getSample
+     */
+    public IntervalSampler(double freq, int maxSplitsSampled) {
+      this.freq = freq;
+      this.maxSplitsSampled = maxSplitsSampled;
+    }
+
+    /**
+     * For each split sampled, emit when the ratio of the number of records
+     * retained to the total record count is less than the specified
+     * frequency.
+     */
+    @SuppressWarnings("unchecked") // ArrayList::toArray doesn't preserve type
+    public K[] getSample(InputFormat<K,V> inf, Job job) 
+        throws IOException, InterruptedException {
+      List<InputSplit> splits = inf.getSplits(job);
+      ArrayList<K> samples = new ArrayList<K>();
+      int splitsToSample = Math.min(maxSplitsSampled, splits.size());
+      long records = 0;
+      long kept = 0;
+      for (int i = 0; i < splitsToSample; ++i) {
+        TaskAttemptContext samplingContext = new TaskAttemptContext(
+            job.getConfiguration(), new TaskAttemptID());
+        RecordReader<K,V> reader = inf.createRecordReader(
+            splits.get(i), samplingContext);
+        reader.initialize(splits.get(i), samplingContext);
+        while (reader.nextKeyValue()) {
+          ++records;
+          if ((double) kept / records < freq) {
+            samples.add(ReflectionUtils.copy(job.getConfiguration(),
+                                 reader.getCurrentKey(), null));
+            ++kept;
+          }
+        }
+        reader.close();
+      }
+      return (K[])samples.toArray();
+    }
+  }
+
+  /**
+   * Write a partition file for the given job, using the Sampler provided.
+   * Queries the sampler for a sample keyset, sorts by the output key
+   * comparator, selects the keys for each rank, and writes to the destination
+   * returned from {@link TotalOrderPartitioner#getPartitionFile}.
+   */
+  @SuppressWarnings("unchecked") // getInputFormat, getOutputKeyComparator
+  public static <K,V> void writePartitionFile(Job job, Sampler<K,V> sampler) 
+      throws IOException, ClassNotFoundException, InterruptedException {
+    Configuration conf = job.getConfiguration();
+    final InputFormat inf = 
+        ReflectionUtils.newInstance(job.getInputFormatClass(), conf);
+    int numPartitions = job.getNumReduceTasks();
+    K[] samples = sampler.getSample(inf, job);
+    LOG.info("Using " + samples.length + " samples");
+    RawComparator<K> comparator =
+      (RawComparator<K>) job.getSortComparator();
+    Arrays.sort(samples, comparator);
+    Path dst = new Path(TotalOrderPartitioner.getPartitionFile(conf));
+    FileSystem fs = dst.getFileSystem(conf);
+    if (fs.exists(dst)) {
+      fs.delete(dst, false);
+    }
+    SequenceFile.Writer writer = SequenceFile.createWriter(fs, 
+      conf, dst, job.getMapOutputKeyClass(), NullWritable.class);
+    NullWritable nullValue = NullWritable.get();
+    float stepSize = samples.length / (float) numPartitions;
+    int last = -1;
+    for(int i = 1; i < numPartitions; ++i) {
+      int k = Math.round(stepSize * i);
+      while (last >= k && comparator.compare(samples[last], samples[k]) == 0) {
+        ++k;
+      }
+      writer.append(samples[k], nullValue);
+      last = k;
+    }
+    writer.close();
+  }
+
+  /**
+   * Driver for InputSampler from the command line.
+   * Configures a JobConf instance and calls {@link #writePartitionFile}.
+   */
+  public int run(String[] args) throws Exception {
+    Job job = new Job(getConf());
+    ArrayList<String> otherArgs = new ArrayList<String>();
+    Sampler<K,V> sampler = null;
+    for(int i=0; i < args.length; ++i) {
+      try {
+        if ("-r".equals(args[i])) {
+          job.setNumReduceTasks(Integer.parseInt(args[++i]));
+        } else if ("-inFormat".equals(args[i])) {
+          job.setInputFormatClass(
+              Class.forName(args[++i]).asSubclass(InputFormat.class));
+        } else if ("-keyClass".equals(args[i])) {
+          job.setMapOutputKeyClass(
+              Class.forName(args[++i]).asSubclass(WritableComparable.class));
+        } else if ("-splitSample".equals(args[i])) {
+          int numSamples = Integer.parseInt(args[++i]);
+          int maxSplits = Integer.parseInt(args[++i]);
+          if (0 >= maxSplits) maxSplits = Integer.MAX_VALUE;
+          sampler = new SplitSampler<K,V>(numSamples, maxSplits);
+        } else if ("-splitRandom".equals(args[i])) {
+          double pcnt = Double.parseDouble(args[++i]);
+          int numSamples = Integer.parseInt(args[++i]);
+          int maxSplits = Integer.parseInt(args[++i]);
+          if (0 >= maxSplits) maxSplits = Integer.MAX_VALUE;
+          sampler = new RandomSampler<K,V>(pcnt, numSamples, maxSplits);
+        } else if ("-splitInterval".equals(args[i])) {
+          double pcnt = Double.parseDouble(args[++i]);
+          int maxSplits = Integer.parseInt(args[++i]);
+          if (0 >= maxSplits) maxSplits = Integer.MAX_VALUE;
+          sampler = new IntervalSampler<K,V>(pcnt, maxSplits);
+        } else {
+          otherArgs.add(args[i]);
+        }
+      } catch (NumberFormatException except) {
+        System.out.println("ERROR: Integer expected instead of " + args[i]);
+        return printUsage();
+      } catch (ArrayIndexOutOfBoundsException except) {
+        System.out.println("ERROR: Required parameter missing from " +
+            args[i-1]);
+        return printUsage();
+      }
+    }
+    if (job.getNumReduceTasks() <= 1) {
+      System.err.println("Sampler requires more than one reducer");
+      return printUsage();
+    }
+    if (otherArgs.size() < 2) {
+      System.out.println("ERROR: Wrong number of parameters: ");
+      return printUsage();
+    }
+    if (null == sampler) {
+      sampler = new RandomSampler<K,V>(0.1, 10000, 10);
+    }
+
+    Path outf = new Path(otherArgs.remove(otherArgs.size() - 1));
+    TotalOrderPartitioner.setPartitionFile(getConf(), outf);
+    for (String s : otherArgs) {
+      FileInputFormat.addInputPath(job, new Path(s));
+    }
+    InputSampler.<K,V>writePartitionFile(job, sampler);
+
+    return 0;
+  }
+
+  public static void main(String[] args) throws Exception {
+    InputSampler<?,?> sampler = new InputSampler(new Configuration());
+    int res = ToolRunner.run(sampler, args);
+    System.exit(res);
+  }
+}

+ 374 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.java

@@ -0,0 +1,374 @@
+/**
+ * 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 java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.partition.KeyFieldHelper.KeyDescription;
+
+
+/**
+ * This comparator implementation provides a subset of the features provided
+ * by the Unix/GNU Sort. In particular, the supported features are:
+ * -n, (Sort numerically)
+ * -r, (Reverse the result of comparison)
+ * -k pos1[,pos2], where pos is of the form f[.c][opts], where f is the number
+ *  of the field to use, and c is the number of the first character from the
+ *  beginning of the field. Fields and character posns are numbered starting
+ *  with 1; a character position of zero in pos2 indicates the field's last
+ *  character. If '.c' is omitted from pos1, it defaults to 1 (the beginning
+ *  of the field); if omitted from pos2, it defaults to 0 (the end of the
+ *  field). opts are ordering options (any of 'nr' as described above). 
+ * We assume that the fields in the key are separated by 
+ * {@link JobContext#MAP_OUTPUT_KEY_FIELD_SEPERATOR}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class KeyFieldBasedComparator<K, V> extends WritableComparator 
+    implements Configurable {
+  private KeyFieldHelper keyFieldHelper = new KeyFieldHelper();
+  public static String COMPARATOR_OPTIONS = "mapreduce.partition.keycomparator.options";
+  private static final byte NEGATIVE = (byte)'-';
+  private static final byte ZERO = (byte)'0';
+  private static final byte DECIMAL = (byte)'.';
+  private Configuration conf;
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    String option = conf.get(COMPARATOR_OPTIONS);
+    String keyFieldSeparator = conf.get("mapreduce.map.output.key.field.separator","\t");
+    keyFieldHelper.setKeyFieldSeparator(keyFieldSeparator);
+    keyFieldHelper.parseOption(option);
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+  
+  public KeyFieldBasedComparator() {
+    super(Text.class);
+  }
+    
+  public int compare(byte[] b1, int s1, int l1,
+      byte[] b2, int s2, int l2) {
+    int n1 = WritableUtils.decodeVIntSize(b1[s1]);
+    int n2 = WritableUtils.decodeVIntSize(b2[s2]);
+    List <KeyDescription> allKeySpecs = keyFieldHelper.keySpecs();
+
+    if (allKeySpecs.size() == 0) {
+      return compareBytes(b1, s1 + n1, l1 - n1, b2, s2 + n2, l2 - n2);
+    }
+    
+    int []lengthIndicesFirst = 
+      keyFieldHelper.getWordLengths(b1, s1 + n1, s1 + l1);
+    int []lengthIndicesSecond = 
+      keyFieldHelper.getWordLengths(b2, s2 + n2, s2 + l2);
+    
+    for (KeyDescription keySpec : allKeySpecs) {
+      int startCharFirst = keyFieldHelper.getStartOffset(b1, s1 + n1, s1 + l1,
+        lengthIndicesFirst, keySpec);
+      int endCharFirst = keyFieldHelper.getEndOffset(b1, s1 + n1, s1 + l1, 
+        lengthIndicesFirst, keySpec);
+      int startCharSecond = keyFieldHelper.getStartOffset(b2, s2 + n2, s2 + l2,
+        lengthIndicesSecond, keySpec);
+      int endCharSecond = keyFieldHelper.getEndOffset(b2, s2 + n2, s2 + l2, 
+        lengthIndicesSecond, keySpec);
+      int result;
+      if ((result = compareByteSequence(b1, startCharFirst, endCharFirst, b2, 
+          startCharSecond, endCharSecond, keySpec)) != 0) {
+        return result;
+      }
+    }
+    return 0;
+  }
+  
+  private int compareByteSequence(byte[] first, int start1, int end1, 
+      byte[] second, int start2, int end2, KeyDescription key) {
+    if (start1 == -1) {
+      if (key.reverse) {
+        return 1;
+      }
+      return -1;
+    }
+    if (start2 == -1) {
+      if (key.reverse) {
+        return -1; 
+      }
+      return 1;
+    }
+    int compareResult = 0;
+    if (!key.numeric) {
+      compareResult = compareBytes(first, start1, end1-start1 + 1, second,
+        start2, end2 - start2 + 1);
+    }
+    if (key.numeric) {
+      compareResult = numericalCompare (first, start1, end1, second, start2,
+        end2);
+    }
+    if (key.reverse) {
+      return -compareResult;
+    }
+    return compareResult;
+  }
+  
+  private int numericalCompare (byte[] a, int start1, int end1, 
+      byte[] b, int start2, int end2) {
+    int i = start1;
+    int j = start2;
+    int mul = 1;
+    byte first_a = a[i];
+    byte first_b = b[j];
+    if (first_a == NEGATIVE) {
+      if (first_b != NEGATIVE) {
+        //check for cases like -0.0 and 0.0 (they should be declared equal)
+        return oneNegativeCompare(a, start1 + 1, end1, b, start2, end2);
+      }
+      i++;
+    }
+    if (first_b == NEGATIVE) {
+      if (first_a != NEGATIVE) {
+        //check for cases like 0.0 and -0.0 (they should be declared equal)
+        return -oneNegativeCompare(b, start2+1, end2, a, start1, end1);
+      }
+      j++;
+    }
+    if (first_b == NEGATIVE && first_a == NEGATIVE) {
+      mul = -1;
+    }
+
+    //skip over ZEROs
+    while (i <= end1) {
+      if (a[i] != ZERO) {
+        break;
+      }
+      i++;
+    }
+    while (j <= end2) {
+      if (b[j] != ZERO) {
+        break;
+      }
+      j++;
+    }
+    
+    //skip over equal characters and stopping at the first nondigit char
+    //The nondigit character could be '.'
+    while (i <= end1 && j <= end2) {
+      if (!isdigit(a[i]) || a[i] != b[j]) {
+        break;
+      }
+      i++; j++;
+    }
+    if (i <= end1) {
+      first_a = a[i];
+    }
+    if (j <= end2) {
+      first_b = b[j];
+    }
+    //store the result of the difference. This could be final result if the
+    //number of digits in the mantissa is the same in both the numbers 
+    int firstResult = first_a - first_b;
+    
+    //check whether we hit a decimal in the earlier scan
+    if ((first_a == DECIMAL && (!isdigit(first_b) || j > end2)) ||
+            (first_b == DECIMAL && (!isdigit(first_a) || i > end1))) {
+      return ((mul < 0) ? -decimalCompare(a, i, end1, b, j, end2) : 
+        decimalCompare(a, i, end1, b, j, end2));
+    }
+    //check the number of digits in the mantissa of the numbers
+    int numRemainDigits_a = 0;
+    int numRemainDigits_b = 0;
+    while (i <= end1) {
+      //if we encounter a non-digit treat the corresponding number as being 
+      //smaller      
+      if (isdigit(a[i++])) {
+        numRemainDigits_a++;
+      } else break;
+    }
+    while (j <= end2) {
+      //if we encounter a non-digit treat the corresponding number as being 
+      //smaller
+      if (isdigit(b[j++])) {
+        numRemainDigits_b++;
+      } else break;
+    }
+    int ret = numRemainDigits_a - numRemainDigits_b;
+    if (ret == 0) { 
+      return ((mul < 0) ? -firstResult : firstResult);
+    } else {
+      return ((mul < 0) ? -ret : ret);
+    }
+  }
+  private boolean isdigit(byte b) {
+    if ('0' <= b && b <= '9') {
+      return true;
+    }
+    return false;
+  }
+  private int decimalCompare(byte[] a, int i, int end1, 
+                             byte[] b, int j, int end2) {
+    if (i > end1) {
+      //if a[] has nothing remaining
+      return -decimalCompare1(b, ++j, end2);
+    }
+    if (j > end2) {
+      //if b[] has nothing remaining
+      return decimalCompare1(a, ++i, end1);
+    }
+    if (a[i] == DECIMAL && b[j] == DECIMAL) {
+      while (i <= end1 && j <= end2) {
+        if (a[i] != b[j]) {
+          if (isdigit(a[i]) && isdigit(b[j])) {
+            return a[i] - b[j];
+          }
+          if (isdigit(a[i])) {
+            return 1;
+          }
+          if (isdigit(b[j])) {
+            return -1;
+          }
+          return 0;
+        }
+        i++; j++;
+      }
+      if (i > end1 && j > end2) {
+        return 0;
+      }
+        
+      if (i > end1) {
+        //check whether there is a non-ZERO digit after potentially
+        //a number of ZEROs (e.g., a=.4444, b=.444400004)
+        return -decimalCompare1(b, j, end2);
+      }
+      if (j > end2) {
+        //check whether there is a non-ZERO digit after potentially
+        //a number of ZEROs (e.g., b=.4444, a=.444400004)
+        return decimalCompare1(a, i, end1);
+      }
+    }
+    else if (a[i] == DECIMAL) {
+      return decimalCompare1(a, ++i, end1);
+    }
+    else if (b[j] == DECIMAL) {
+      return -decimalCompare1(b, ++j, end2);
+    }
+    return 0;
+  }
+  
+  private int decimalCompare1(byte[] a, int i, int end) {
+    while (i <= end) {
+      if (a[i] == ZERO) {
+        i++;
+        continue;
+      }
+      if (isdigit(a[i])) {
+        return 1;
+      } else {
+        return 0;
+      }
+    }
+    return 0;
+  }
+  
+  private int oneNegativeCompare(byte[] a, int start1, int end1, 
+      byte[] b, int start2, int end2) {
+    //here a[] is negative and b[] is positive
+    //We have to ascertain whether the number contains any digits.
+    //If it does, then it is a smaller number for sure. If not,
+    //then we need to scan b[] to find out whether b[] has a digit
+    //If b[] does contain a digit, then b[] is certainly
+    //greater. If not, that is, both a[] and b[] don't contain
+    //digits then they should be considered equal.
+    if (!isZero(a, start1, end1)) {
+      return -1;
+    }
+    //reached here - this means that a[] is a ZERO
+    if (!isZero(b, start2, end2)) {
+      return -1;
+    }
+    //reached here - both numbers are basically ZEROs and hence
+    //they should compare equal
+    return 0;
+  }
+  
+  private boolean isZero(byte a[], int start, int end) {
+    //check for zeros in the significand part as well as the decimal part
+    //note that we treat the non-digit characters as ZERO
+    int i = start;
+    //we check the significand for being a ZERO
+    while (i <= end) {
+      if (a[i] != ZERO) {
+        if (a[i] != DECIMAL && isdigit(a[i])) {
+          return false;
+        }
+        break;
+      }
+      i++;
+    }
+
+    if (i != (end+1) && a[i++] == DECIMAL) {
+      //we check the decimal part for being a ZERO
+      while (i <= end) {
+        if (a[i] != ZERO) {
+          if (isdigit(a[i])) {
+            return false;
+          }
+          break;
+        }
+        i++;
+      }
+    }
+    return true;
+  }
+  /**
+   * Set the {@link KeyFieldBasedComparator} options used to compare keys.
+   * 
+   * @param keySpec the key specification of the form -k pos1[,pos2], where,
+   *  pos is of the form f[.c][opts], where f is the number
+   *  of the key field to use, and c is the number of the first character from
+   *  the beginning of the field. Fields and character posns are numbered 
+   *  starting with 1; a character position of zero in pos2 indicates the
+   *  field's last character. If '.c' is omitted from pos1, it defaults to 1
+   *  (the beginning of the field); if omitted from pos2, it defaults to 0 
+   *  (the end of the field). opts are ordering options. The supported options
+   *  are:
+   *    -n, (Sort numerically)
+   *    -r, (Reverse the result of comparison)                 
+   */
+  public static void setKeyFieldComparatorOptions(Job job, String keySpec) {
+    job.getConfiguration().set(COMPARATOR_OPTIONS, keySpec);
+  }
+  
+  /**
+   * Get the {@link KeyFieldBasedComparator} options
+   */
+  public static String getKeyFieldComparatorOption(JobContext job) {
+    return job.getConfiguration().get(COMPARATOR_OPTIONS);
+  }
+
+
+}

+ 156 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedPartitioner.java

@@ -0,0 +1,156 @@
+/**
+ * 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 java.io.UnsupportedEncodingException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.lib.partition.KeyFieldHelper.KeyDescription;
+
+ /**   
+  *  Defines a way to partition keys based on certain key fields (also see
+  *  {@link KeyFieldBasedComparator}.
+  *  The key specification supported is of the form -k pos1[,pos2], where,
+  *  pos is of the form f[.c][opts], where f is the number
+  *  of the key field to use, and c is the number of the first character from
+  *  the beginning of the field. Fields and character posns are numbered 
+  *  starting with 1; a character position of zero in pos2 indicates the
+  *  field's last character. If '.c' is omitted from pos1, it defaults to 1
+  *  (the beginning of the field); if omitted from pos2, it defaults to 0 
+  *  (the end of the field).
+  * 
+  */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class KeyFieldBasedPartitioner<K2, V2> extends Partitioner<K2, V2> 
+    implements Configurable {
+
+  private static final Log LOG = LogFactory.getLog(
+                                   KeyFieldBasedPartitioner.class.getName());
+  public static String PARTITIONER_OPTIONS = 
+    "mapreduce.partition.keypartitioner.options";
+  private int numOfPartitionFields;
+  
+  private KeyFieldHelper keyFieldHelper = new KeyFieldHelper();
+  
+  private Configuration conf;
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    String keyFieldSeparator = 
+      conf.get("mapreduce.map.output.key.field.separator", "\t");
+    keyFieldHelper.setKeyFieldSeparator(keyFieldSeparator);
+    if (conf.get("num.key.fields.for.partition") != null) {
+      LOG.warn("Using deprecated num.key.fields.for.partition. " +
+      		"Use mapreduce.partition.keypartitioner.options instead");
+      this.numOfPartitionFields = conf.getInt("num.key.fields.for.partition",0);
+      keyFieldHelper.setKeyFieldSpec(1,numOfPartitionFields);
+    } else {
+      String option = conf.get(PARTITIONER_OPTIONS);
+      keyFieldHelper.parseOption(option);
+    }
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+  
+  public int getPartition(K2 key, V2 value, int numReduceTasks) {
+    byte[] keyBytes;
+
+    List <KeyDescription> allKeySpecs = keyFieldHelper.keySpecs();
+    if (allKeySpecs.size() == 0) {
+      return getPartition(key.toString().hashCode(), numReduceTasks);
+    }
+
+    try {
+      keyBytes = key.toString().getBytes("UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      throw new RuntimeException("The current system does not " +
+          "support UTF-8 encoding!", e);
+    }
+    // return 0 if the key is empty
+    if (keyBytes.length == 0) {
+      return 0;
+    }
+    
+    int []lengthIndicesFirst = keyFieldHelper.getWordLengths(keyBytes, 0, 
+        keyBytes.length);
+    int currentHash = 0;
+    for (KeyDescription keySpec : allKeySpecs) {
+      int startChar = keyFieldHelper.getStartOffset(keyBytes, 0, 
+        keyBytes.length, lengthIndicesFirst, keySpec);
+       // no key found! continue
+      if (startChar < 0) {
+        continue;
+      }
+      int endChar = keyFieldHelper.getEndOffset(keyBytes, 0, keyBytes.length, 
+          lengthIndicesFirst, keySpec);
+      currentHash = hashCode(keyBytes, startChar, endChar, 
+          currentHash);
+    }
+    return getPartition(currentHash, numReduceTasks);
+  }
+  
+  protected int hashCode(byte[] b, int start, int end, int currentHash) {
+    for (int i = start; i <= end; i++) {
+      currentHash = 31*currentHash + b[i];
+    }
+    return currentHash;
+  }
+
+  protected int getPartition(int hash, int numReduceTasks) {
+    return (hash & Integer.MAX_VALUE) % numReduceTasks;
+  }
+  
+  /**
+   * Set the {@link KeyFieldBasedPartitioner} options used for 
+   * {@link Partitioner}
+   * 
+   * @param keySpec the key specification of the form -k pos1[,pos2], where,
+   *  pos is of the form f[.c][opts], where f is the number
+   *  of the key field to use, and c is the number of the first character from
+   *  the beginning of the field. Fields and character posns are numbered 
+   *  starting with 1; a character position of zero in pos2 indicates the
+   *  field's last character. If '.c' is omitted from pos1, it defaults to 1
+   *  (the beginning of the field); if omitted from pos2, it defaults to 0 
+   *  (the end of the field).
+   */
+  public void setKeyFieldPartitionerOptions(Job job, String keySpec) {
+    job.getConfiguration().set(PARTITIONER_OPTIONS, keySpec);
+  }
+  
+  /**
+   * Get the {@link KeyFieldBasedPartitioner} options
+   */
+  public String getKeyFieldPartitionerOption(JobContext job) {
+    return job.getConfiguration().get(PARTITIONER_OPTIONS);
+  }
+
+
+}

+ 296 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/partition/KeyFieldHelper.java

@@ -0,0 +1,296 @@
+/**
+ * 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 java.io.UnsupportedEncodingException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.util.UTF8ByteArrayUtils;
+
+/**
+ * This is used in {@link KeyFieldBasedComparator} & 
+ * {@link KeyFieldBasedPartitioner}. Defines all the methods
+ * for parsing key specifications. The key specification is of the form:
+ * -k pos1[,pos2], where pos is of the form f[.c][opts], where f is the number
+ *  of the field to use, and c is the number of the first character from the
+ *  beginning of the field. Fields and character posns are numbered starting
+ *  with 1; a character position of zero in pos2 indicates the field's last
+ *  character. If '.c' is omitted from pos1, it defaults to 1 (the beginning
+ *  of the field); if omitted from pos2, it defaults to 0 (the end of the
+ *  field). opts are ordering options (supported options are 'nr'). 
+ */
+
+class KeyFieldHelper {
+  
+  protected static class KeyDescription {
+    int beginFieldIdx = 1;
+    int beginChar = 1;
+    int endFieldIdx = 0;
+    int endChar = 0;
+    boolean numeric;
+    boolean reverse;
+    @Override
+    public String toString() {
+      return "-k" 
+             + beginFieldIdx + "." + beginChar + "," 
+             + endFieldIdx + "." + endChar 
+             + (numeric ? "n" : "") + (reverse ? "r" : "");
+    }
+  }
+  
+  private List<KeyDescription> allKeySpecs = new ArrayList<KeyDescription>();
+  private byte[] keyFieldSeparator;
+  private boolean keySpecSeen = false;
+  
+  public void setKeyFieldSeparator(String keyFieldSeparator) {
+    try {
+      this.keyFieldSeparator =
+        keyFieldSeparator.getBytes("UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      throw new RuntimeException("The current system does not " +
+          "support UTF-8 encoding!", e);
+    }    
+  }
+  
+  /** Required for backcompatibility with num.key.fields.for.partition in
+   * {@link KeyFieldBasedPartitioner} */
+  public void setKeyFieldSpec(int start, int end) {
+    if (end >= start) {
+      KeyDescription k = new KeyDescription();
+      k.beginFieldIdx = start;
+      k.endFieldIdx = end;
+      keySpecSeen = true;
+      allKeySpecs.add(k);
+    }
+  }
+  
+  public List<KeyDescription> keySpecs() {
+    return allKeySpecs;
+  }
+    
+  public int[] getWordLengths(byte []b, int start, int end) {
+    //Given a string like "hello how are you", it returns an array
+    //like [4 5, 3, 3, 3], where the first element is the number of
+	//fields
+    if (!keySpecSeen) {
+      //if there were no key specs, then the whole key is one word
+      return new int[] {1};
+    }
+    int[] lengths = new int[10];
+    int currLenLengths = lengths.length;
+    int idx = 1;
+    int pos;
+    while ((pos = UTF8ByteArrayUtils.findBytes(b, start, end, 
+        keyFieldSeparator)) != -1) {
+      if (++idx == currLenLengths) {
+        int[] temp = lengths;
+        lengths = new int[(currLenLengths = currLenLengths*2)];
+        System.arraycopy(temp, 0, lengths, 0, temp.length);
+      }
+      lengths[idx - 1] = pos - start;
+      start = pos + 1;
+    }
+    
+    if (start != end) {
+      lengths[idx] = end - start;
+    }
+    lengths[0] = idx; //number of words is the first element
+    return lengths;
+  }
+  public int getStartOffset(byte[]b, int start, int end, 
+      int []lengthIndices, KeyDescription k) {
+    //if -k2.5,2 is the keyspec, the startChar is lengthIndices[1] + 5
+    //note that the [0]'th element is the number of fields in the key
+    if (lengthIndices[0] >= k.beginFieldIdx) {
+      int position = 0;
+      for (int i = 1; i < k.beginFieldIdx; i++) {
+        position += lengthIndices[i] + keyFieldSeparator.length; 
+      }
+      if (position + k.beginChar <= (end - start)) {
+        return start + position + k.beginChar - 1; 
+      }
+    }
+    return -1;
+  }
+  public int getEndOffset(byte[]b, int start, int end, 
+      int []lengthIndices, KeyDescription k) {
+    //if -k2,2.8 is the keyspec, the endChar is lengthIndices[1] + 8
+    //note that the [0]'th element is the number of fields in the key
+    if (k.endFieldIdx == 0) {
+      //there is no end field specified for this keyspec. So the remaining
+      //part of the key is considered in its entirety.
+      return end - 1; 
+    }
+    if (lengthIndices[0] >= k.endFieldIdx) {
+      int position = 0;
+      int i;
+      for (i = 1; i < k.endFieldIdx; i++) {
+        position += lengthIndices[i] + keyFieldSeparator.length;
+      }
+      if (k.endChar == 0) { 
+        position += lengthIndices[i];
+      }
+      if (position + k.endChar <= (end - start)) {
+        return start + position + k.endChar - 1;
+      }
+      return end - 1;
+    }
+    return end - 1;
+  }
+  public void parseOption(String option) {
+    if (option == null || option.equals("")) {
+      //we will have only default comparison
+      return;
+    }
+    StringTokenizer args = new StringTokenizer(option);
+    KeyDescription global = new KeyDescription();
+    while (args.hasMoreTokens()) {
+      String arg = args.nextToken();
+      if (arg.equals("-n")) {  
+        global.numeric = true;
+      }
+      if (arg.equals("-r")) {
+        global.reverse = true;
+      }
+      if (arg.equals("-nr")) {
+        global.numeric = true;
+        global.reverse = true;
+      }
+      if (arg.startsWith("-k")) {
+        KeyDescription k = parseKey(arg, args);
+        if (k != null) {
+          allKeySpecs.add(k);
+          keySpecSeen = true;
+        }
+      }
+    }
+    for (KeyDescription key : allKeySpecs) {
+      if (!(key.reverse | key.numeric)) {
+        key.reverse = global.reverse;
+        key.numeric = global.numeric;
+      }
+    }
+    if (allKeySpecs.size() == 0) {
+      allKeySpecs.add(global);
+    }
+  }
+  
+  private KeyDescription parseKey(String arg, StringTokenizer args) {
+    //we allow for -k<arg> and -k <arg>
+    String keyArgs = null;
+    if (arg.length() == 2) {
+      if (args.hasMoreTokens()) {
+        keyArgs = args.nextToken();
+      }
+    } else {
+      keyArgs = arg.substring(2);
+    }
+    if (keyArgs == null || keyArgs.length() == 0) {
+      return null;
+    }
+    StringTokenizer st = new StringTokenizer(keyArgs,"nr.,",true);
+       
+    KeyDescription key = new KeyDescription();
+    
+    String token;
+    //the key is of the form 1[.3][nr][,1.5][nr]
+    if (st.hasMoreTokens()) {
+      token = st.nextToken();
+      //the first token must be a number
+      key.beginFieldIdx = Integer.parseInt(token);
+    }
+    if (st.hasMoreTokens()) {
+      token = st.nextToken();
+      if (token.equals(".")) {
+        token = st.nextToken();
+        key.beginChar = Integer.parseInt(token);
+        if (st.hasMoreTokens()) {
+          token = st.nextToken();
+        } else {
+          return key;
+        }
+      } 
+      do {
+        if (token.equals("n")) {
+          key.numeric = true;
+        }
+        else if (token.equals("r")) {
+          key.reverse = true;
+        }
+        else break;
+        if (st.hasMoreTokens()) {
+          token = st.nextToken();
+        } else {
+          return key;
+        }
+      } while (true);
+      if (token.equals(",")) {
+        token = st.nextToken();
+        //the first token must be a number
+        key.endFieldIdx = Integer.parseInt(token);
+        if (st.hasMoreTokens()) {
+          token = st.nextToken();
+          if (token.equals(".")) {
+            token = st.nextToken();
+            key.endChar = Integer.parseInt(token);
+            if (st.hasMoreTokens()) {
+              token = st.nextToken();
+            } else {
+              return key;
+            }
+          }
+          do {
+            if (token.equals("n")) {
+              key.numeric = true;
+            }
+            else if (token.equals("r")) {
+              key.reverse = true;
+            }
+            else { 
+              throw new IllegalArgumentException("Invalid -k argument. " +
+               "Must be of the form -k pos1,[pos2], where pos is of the form " +
+               "f[.c]nr");
+            }
+            if (st.hasMoreTokens()) {
+              token = st.nextToken();
+            } else {
+              break;
+            }
+          } while (true);
+        }
+        return key;
+      }
+      throw new IllegalArgumentException("Invalid -k argument. " +
+          "Must be of the form -k pos1,[pos2], where pos is of the form " +
+          "f[.c]nr");
+    }
+    return key;
+  }
+  private void printKey(KeyDescription key) {
+    System.out.println("key.beginFieldIdx: " + key.beginFieldIdx);
+    System.out.println("key.beginChar: " + key.beginChar);
+    System.out.println("key.endFieldIdx: " + key.endFieldIdx);
+    System.out.println("key.endChar: " + key.endChar);
+    System.out.println("key.numeric: " + key.numeric);
+    System.out.println("key.reverse: " + key.reverse);
+    System.out.println("parseKey over");
+  }  
+}

+ 411 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/partition/TotalOrderPartitioner.java

@@ -0,0 +1,411 @@
+/**
+ * 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 java.io.IOException;
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Partitioner effecting a total order by reading split points from
+ * an externally generated source.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class TotalOrderPartitioner<K extends WritableComparable<?>,V>
+    extends Partitioner<K,V> implements Configurable {
+
+  private Node partitions;
+  public static final String DEFAULT_PATH = "_partition.lst";
+  public static final String PARTITIONER_PATH = 
+    "mapreduce.totalorderpartitioner.path";
+  public static final String MAX_TRIE_DEPTH = 
+    "mapreduce.totalorderpartitioner.trie.maxdepth"; 
+  public static final String NATURAL_ORDER = 
+    "mapreduce.totalorderpartitioner.naturalorder";
+  Configuration conf;
+  private static final Log LOG = LogFactory.getLog(TotalOrderPartitioner.class);
+
+  public TotalOrderPartitioner() { }
+
+  /**
+   * Read in the partition file and build indexing data structures.
+   * If the keytype is {@link org.apache.hadoop.io.BinaryComparable} and
+   * <tt>total.order.partitioner.natural.order</tt> is not false, a trie
+   * of the first <tt>total.order.partitioner.max.trie.depth</tt>(2) + 1 bytes
+   * will be built. Otherwise, keys will be located using a binary search of
+   * the partition keyset using the {@link org.apache.hadoop.io.RawComparator}
+   * defined for this job. The input file must be sorted with the same
+   * comparator and contain {@link Job#getNumReduceTasks()} - 1 keys.
+   */
+  @SuppressWarnings("unchecked") // keytype from conf not static
+  public void setConf(Configuration conf) {
+    try {
+      this.conf = conf;
+      String parts = getPartitionFile(conf);
+      final Path partFile = new Path(parts);
+      final FileSystem fs = (DEFAULT_PATH.equals(parts))
+        ? FileSystem.getLocal(conf)     // assume in DistributedCache
+        : partFile.getFileSystem(conf);
+
+      Job job = new Job(conf);
+      Class<K> keyClass = (Class<K>)job.getMapOutputKeyClass();
+      K[] splitPoints = readPartitions(fs, partFile, keyClass, conf);
+      if (splitPoints.length != job.getNumReduceTasks() - 1) {
+        throw new IOException("Wrong number of partitions in keyset");
+      }
+      RawComparator<K> comparator =
+        (RawComparator<K>) job.getSortComparator();
+      for (int i = 0; i < splitPoints.length - 1; ++i) {
+        if (comparator.compare(splitPoints[i], splitPoints[i+1]) >= 0) {
+          throw new IOException("Split points are out of order");
+        }
+      }
+      boolean natOrder =
+        conf.getBoolean(NATURAL_ORDER, true);
+      if (natOrder && BinaryComparable.class.isAssignableFrom(keyClass)) {
+        partitions = buildTrie((BinaryComparable[])splitPoints, 0,
+            splitPoints.length, new byte[0],
+            // Now that blocks of identical splitless trie nodes are 
+            // represented reentrantly, and we develop a leaf for any trie
+            // node with only one split point, the only reason for a depth
+            // limit is to refute stack overflow or bloat in the pathological
+            // case where the split points are long and mostly look like bytes 
+            // iii...iixii...iii   .  Therefore, we make the default depth
+            // limit large but not huge.
+            conf.getInt(MAX_TRIE_DEPTH, 200));
+      } else {
+        partitions = new BinarySearchNode(splitPoints, comparator);
+      }
+    } catch (IOException e) {
+      throw new IllegalArgumentException("Can't read partitions file", e);
+    }
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+  
+  // by construction, we know if our keytype
+  @SuppressWarnings("unchecked") // is memcmp-able and uses the trie
+  public int getPartition(K key, V value, int numPartitions) {
+    return partitions.findPartition(key);
+  }
+
+  /**
+   * Set the path to the SequenceFile storing the sorted partition keyset.
+   * It must be the case that for <tt>R</tt> reduces, there are <tt>R-1</tt>
+   * keys in the SequenceFile.
+   */
+  public static void setPartitionFile(Configuration conf, Path p) {
+    conf.set(PARTITIONER_PATH, p.toString());
+  }
+
+  /**
+   * Get the path to the SequenceFile storing the sorted partition keyset.
+   * @see #setPartitionFile(Configuration, Path)
+   */
+  public static String getPartitionFile(Configuration conf) {
+    return conf.get(PARTITIONER_PATH, DEFAULT_PATH);
+  }
+
+  /**
+   * Interface to the partitioner to locate a key in the partition keyset.
+   */
+  interface Node<T> {
+    /**
+     * Locate partition in keyset K, st [Ki..Ki+1) defines a partition,
+     * with implicit K0 = -inf, Kn = +inf, and |K| = #partitions - 1.
+     */
+    int findPartition(T key);
+  }
+
+  /**
+   * Base class for trie nodes. If the keytype is memcomp-able, this builds
+   * tries of the first <tt>total.order.partitioner.max.trie.depth</tt>
+   * bytes.
+   */
+  static abstract class TrieNode implements Node<BinaryComparable> {
+    private final int level;
+    TrieNode(int level) {
+      this.level = level;
+    }
+    int getLevel() {
+      return level;
+    }
+  }
+
+  /**
+   * For types that are not {@link org.apache.hadoop.io.BinaryComparable} or
+   * where disabled by <tt>total.order.partitioner.natural.order</tt>,
+   * search the partition keyset with a binary search.
+   */
+  class BinarySearchNode implements Node<K> {
+    private final K[] splitPoints;
+    private final RawComparator<K> comparator;
+    BinarySearchNode(K[] splitPoints, RawComparator<K> comparator) {
+      this.splitPoints = splitPoints;
+      this.comparator = comparator;
+    }
+    public int findPartition(K key) {
+      final int pos = Arrays.binarySearch(splitPoints, key, comparator) + 1;
+      return (pos < 0) ? -pos : pos;
+    }
+  }
+
+  /**
+   * An inner trie node that contains 256 children based on the next
+   * character.
+   */
+  class InnerTrieNode extends TrieNode {
+    private TrieNode[] child = new TrieNode[256];
+
+    InnerTrieNode(int level) {
+      super(level);
+    }
+    public int findPartition(BinaryComparable key) {
+      int level = getLevel();
+      if (key.getLength() <= level) {
+        return child[0].findPartition(key);
+      }
+      return child[0xFF & key.getBytes()[level]].findPartition(key);
+    }
+  }
+  
+  /**
+   * @param level        the tree depth at this node
+   * @param splitPoints  the full split point vector, which holds
+   *                     the split point or points this leaf node
+   *                     should contain
+   * @param lower        first INcluded element of splitPoints
+   * @param upper        first EXcluded element of splitPoints
+   * @return  a leaf node.  They come in three kinds: no split points 
+   *          [and the findParttion returns a canned index], one split
+   *          point [and we compare with a single comparand], or more
+   *          than one [and we do a binary search].  The last case is
+   *          rare.
+   */
+  private TrieNode LeafTrieNodeFactory
+             (int level, BinaryComparable[] splitPoints, int lower, int upper) {
+      switch (upper - lower) {
+      case 0:
+          return new UnsplitTrieNode(level, lower);
+          
+      case 1:
+          return new SinglySplitTrieNode(level, splitPoints, lower);
+          
+      default:
+          return new LeafTrieNode(level, splitPoints, lower, upper);
+      }
+  }
+
+  /**
+   * A leaf trie node that scans for the key between lower..upper.
+   * 
+   * We don't generate many of these now, since we usually continue trie-ing 
+   * when more than one split point remains at this level. and we make different
+   * objects for nodes with 0 or 1 split point.
+   */
+  private class LeafTrieNode extends TrieNode {
+    final int lower;
+    final int upper;
+    final BinaryComparable[] splitPoints;
+    LeafTrieNode(int level, BinaryComparable[] splitPoints, int lower, int upper) {
+      super(level);
+      this.lower = lower;
+      this.upper = upper;
+      this.splitPoints = splitPoints;
+    }
+    public int findPartition(BinaryComparable key) {
+      final int pos = Arrays.binarySearch(splitPoints, lower, upper, key) + 1;
+      return (pos < 0) ? -pos : pos;
+    }
+  }
+  
+  private class UnsplitTrieNode extends TrieNode {
+      final int result;
+      
+      UnsplitTrieNode(int level, int value) {
+          super(level);
+          this.result = value;
+      }
+      
+      public int findPartition(BinaryComparable key) {
+          return result;
+      }
+  }
+  
+  private class SinglySplitTrieNode extends TrieNode {
+      final int               lower;
+      final BinaryComparable  mySplitPoint;
+      
+      SinglySplitTrieNode(int level, BinaryComparable[] splitPoints, int lower) {
+          super(level);
+          this.lower = lower;
+          this.mySplitPoint = splitPoints[lower];
+      }
+      
+      public int findPartition(BinaryComparable key) {
+          return lower + (key.compareTo(mySplitPoint) < 0 ? 0 : 1);
+      }
+  }
+
+
+  /**
+   * Read the cut points from the given IFile.
+   * @param fs The file system
+   * @param p The path to read
+   * @param keyClass The map output key class
+   * @param job The job config
+   * @throws IOException
+   */
+                                 // matching key types enforced by passing in
+  @SuppressWarnings("unchecked") // map output key class
+  private K[] readPartitions(FileSystem fs, Path p, Class<K> keyClass,
+      Configuration conf) throws IOException {
+    SequenceFile.Reader reader = new SequenceFile.Reader(fs, p, conf);
+    ArrayList<K> parts = new ArrayList<K>();
+    K key = ReflectionUtils.newInstance(keyClass, conf);
+    NullWritable value = NullWritable.get();
+    try {
+      while (reader.next(key, value)) {
+        parts.add(key);
+        key = ReflectionUtils.newInstance(keyClass, conf);
+      }
+      reader.close();
+      reader = null;
+    } finally {
+      IOUtils.cleanup(LOG, reader);
+    }
+    return parts.toArray((K[])Array.newInstance(keyClass, parts.size()));
+  }
+  
+  /**
+   * 
+   * This object contains a TrieNodeRef if there is such a thing that
+   * can be repeated.  Two adjacent trie node slots that contain no 
+   * split points can be filled with the same trie node, even if they
+   * are not on the same level.  See buildTreeRec, below.
+   *
+   */  
+  private class CarriedTrieNodeRef
+  {
+      TrieNode   content;
+      
+      CarriedTrieNodeRef() {
+          content = null;
+      }
+  }
+
+  
+  /**
+   * Given a sorted set of cut points, build a trie that will find the correct
+   * partition quickly.
+   * @param splits the list of cut points
+   * @param lower the lower bound of partitions 0..numPartitions-1
+   * @param upper the upper bound of partitions 0..numPartitions-1
+   * @param prefix the prefix that we have already checked against
+   * @param maxDepth the maximum depth we will build a trie for
+   * @return the trie node that will divide the splits correctly
+   */
+  private TrieNode buildTrie(BinaryComparable[] splits, int lower,
+          int upper, byte[] prefix, int maxDepth) {
+      return buildTrieRec
+               (splits, lower, upper, prefix, maxDepth, new CarriedTrieNodeRef());
+  }
+  
+  /**
+   * This is the core of buildTrie.  The interface, and stub, above, just adds
+   * an empty CarriedTrieNodeRef.  
+   * 
+   * We build trie nodes in depth first order, which is also in key space
+   * order.  Every leaf node is referenced as a slot in a parent internal
+   * node.  If two adjacent slots [in the DFO] hold leaf nodes that have
+   * no split point, then they are not separated by a split point either, 
+   * because there's no place in key space for that split point to exist.
+   * 
+   * When that happens, the leaf nodes would be semantically identical, and
+   * we reuse the object.  A single CarriedTrieNodeRef "ref" lives for the 
+   * duration of the tree-walk.  ref carries a potentially reusable, unsplit
+   * leaf node for such reuse until a leaf node with a split arises, which 
+   * breaks the chain until we need to make a new unsplit leaf node.
+   * 
+   * Note that this use of CarriedTrieNodeRef means that for internal nodes, 
+   * for internal nodes if this code is modified in any way we still need 
+   * to make or fill in the subnodes in key space order.
+   */
+  private TrieNode buildTrieRec(BinaryComparable[] splits, int lower,
+      int upper, byte[] prefix, int maxDepth, CarriedTrieNodeRef ref) {
+    final int depth = prefix.length;
+    // We generate leaves for a single split point as well as for 
+    // no split points.
+    if (depth >= maxDepth || lower >= upper - 1) {
+        // If we have two consecutive requests for an unsplit trie node, we
+        // can deliver the same one the second time.
+        if (lower == upper && ref.content != null) {
+            return ref.content;
+        }
+        TrieNode  result = LeafTrieNodeFactory(depth, splits, lower, upper);
+        ref.content = lower == upper ? result : null;
+        return result;
+    }
+    InnerTrieNode result = new InnerTrieNode(depth);
+    byte[] trial = Arrays.copyOf(prefix, prefix.length + 1);
+    // append an extra byte on to the prefix
+    int         currentBound = lower;
+    for(int ch = 0; ch < 0xFF; ++ch) {
+      trial[depth] = (byte) (ch + 1);
+      lower = currentBound;
+      while (currentBound < upper) {
+        if (splits[currentBound].compareTo(trial, 0, trial.length) >= 0) {
+          break;
+        }
+        currentBound += 1;
+      }
+      trial[depth] = (byte) ch;
+      result.child[0xFF & ch]
+                   = buildTrieRec(splits, lower, currentBound, trial, maxDepth, ref);
+    }
+    // pick up the rest
+    trial[depth] = (byte)0xFF;
+    result.child[0xFF] 
+                 = buildTrieRec(splits, lower, currentBound, trial, maxDepth, ref);
+    
+    return result;
+  }
+}

+ 189 - 0
src/test/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java

@@ -0,0 +1,189 @@
+/**
+ * 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.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.Utils;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+
+/**
+ * A JUnit test to test the Map-Reduce framework's feature to create part
+ * files only if there is an explicit output.collect. This helps in preventing
+ * 0 byte files
+ */
+public class TestMapReduceLazyOutput extends TestCase {
+  private static final int NUM_HADOOP_SLAVES = 3;
+  private static final int NUM_MAPS_PER_NODE = 2;
+  private static final Path INPUT = new Path("/testlazy/input");
+
+  private static final List<String> input = 
+    Arrays.asList("All","Roads","Lead","To","Hadoop");
+
+  public static class TestMapper 
+  extends Mapper<LongWritable, Text, LongWritable, Text>{
+
+    public void map(LongWritable key, Text value, Context context
+    ) throws IOException, InterruptedException {
+      String id = context.getTaskAttemptID().toString();
+      // Mapper 0 does not output anything
+      if (!id.endsWith("0_0")) {
+        context.write(key, value);
+      }
+    }
+  }
+
+
+  public static class TestReducer 
+  extends Reducer<LongWritable,Text,LongWritable,Text> {
+    
+    public void reduce(LongWritable key, Iterable<Text> values, 
+        Context context) throws IOException, InterruptedException {
+      String id = context.getTaskAttemptID().toString();
+      // Reducer 0 does not output anything
+      if (!id.endsWith("0_0")) {
+        for (Text val: values) {
+          context.write(key, val);
+        }
+      }
+    }
+  }
+  
+  private static void runTestLazyOutput(Configuration conf, Path output,
+      int numReducers, boolean createLazily) 
+  throws Exception {
+    Job job = new Job(conf, "Test-Lazy-Output");
+
+    FileInputFormat.setInputPaths(job, INPUT);
+    FileOutputFormat.setOutputPath(job, output);
+
+    job.setJarByClass(TestMapReduceLazyOutput.class);
+    job.setInputFormatClass(TextInputFormat.class);
+    job.setOutputKeyClass(LongWritable.class);
+    job.setOutputValueClass(Text.class);
+    job.setNumReduceTasks(numReducers);
+
+    job.setMapperClass(TestMapper.class);
+    job.setReducerClass(TestReducer.class);
+
+    if (createLazily) {
+      LazyOutputFormat.setOutputFormatClass(job, TextOutputFormat.class);
+    } else {
+      job.setOutputFormatClass(TextOutputFormat.class);
+    }
+    assertTrue(job.waitForCompletion(true));
+  }
+
+  public void createInput(FileSystem fs, int numMappers) throws Exception {
+    for (int i =0; i < numMappers; i++) {
+      OutputStream os = fs.create(new Path(INPUT, 
+        "text" + i + ".txt"));
+      Writer wr = new OutputStreamWriter(os);
+      for(String inp : input) {
+        wr.write(inp+"\n");
+      }
+      wr.close();
+    }
+  }
+
+
+  public void testLazyOutput() throws Exception {
+    MiniDFSCluster dfs = null;
+    MiniMRCluster mr = null;
+    FileSystem fileSys = null;
+    try {
+      Configuration conf = new Configuration();
+
+      // Start the mini-MR and mini-DFS clusters
+      dfs = new MiniDFSCluster(conf, NUM_HADOOP_SLAVES, true, null);
+      fileSys = dfs.getFileSystem();
+      mr = new MiniMRCluster(NUM_HADOOP_SLAVES, fileSys.getUri().toString(), 1);
+
+      int numReducers = 2;
+      int numMappers = NUM_HADOOP_SLAVES * NUM_MAPS_PER_NODE;
+
+      createInput(fileSys, numMappers);
+      Path output1 = new Path("/testlazy/output1");
+
+      // Test 1. 
+      runTestLazyOutput(mr.createJobConf(), output1, 
+          numReducers, true);
+
+      Path[] fileList = 
+        FileUtil.stat2Paths(fileSys.listStatus(output1,
+            new Utils.OutputFileUtils.OutputFilesFilter()));
+      for(int i=0; i < fileList.length; ++i) {
+        System.out.println("Test1 File list[" + i + "]" + ": "+ fileList[i]);
+      }
+      assertTrue(fileList.length == (numReducers - 1));
+
+      // Test 2. 0 Reducers, maps directly write to the output files
+      Path output2 = new Path("/testlazy/output2");
+      runTestLazyOutput(mr.createJobConf(), output2, 0, true);
+
+      fileList =
+        FileUtil.stat2Paths(fileSys.listStatus(output2,
+            new Utils.OutputFileUtils.OutputFilesFilter()));
+      for(int i=0; i < fileList.length; ++i) {
+        System.out.println("Test2 File list[" + i + "]" + ": "+ fileList[i]);
+      }
+
+      assertTrue(fileList.length == numMappers - 1);
+
+      // Test 3. 0 Reducers, but flag is turned off
+      Path output3 = new Path("/testlazy/output3");
+      runTestLazyOutput(mr.createJobConf(), output3, 0, false);
+
+      fileList =
+        FileUtil.stat2Paths(fileSys.listStatus(output3,
+            new Utils.OutputFileUtils.OutputFilesFilter()));
+      for(int i=0; i < fileList.length; ++i) {
+        System.out.println("Test3 File list[" + i + "]" + ": "+ fileList[i]);
+      }
+
+      assertTrue(fileList.length == numMappers);
+
+    } finally {
+      if (dfs != null) { dfs.shutdown(); }
+      if (mr != null) { mr.shutdown();
+      }
+    }
+  }
+
+}

+ 39 - 0
src/test/org/apache/hadoop/mapreduce/lib/db/TestDBJob.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.db;
+
+import java.io.IOException;
+
+import org.apache.hadoop.examples.DBCountPageView;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.util.ToolRunner;
+
+
+public class TestDBJob extends HadoopTestCase {
+
+  public TestDBJob() throws IOException {
+    super(LOCAL_MR, LOCAL_FS, 3, 1);
+  }
+  
+  public void testRun() throws Exception {
+    DBCountPageView testDriver = new DBCountPageView();
+    ToolRunner.run(createJobConf(), testDriver, new String[0]);
+  }
+  
+}

+ 69 - 0
src/test/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java

@@ -0,0 +1,69 @@
+/**
+ * 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.db;
+
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Job;
+
+public class TestDBOutputFormat extends TestCase {
+  
+  private String[] fieldNames = new String[] { "id", "name", "value" };
+  private String[] nullFieldNames = new String[] { null, null, null };
+  private String expected = "INSERT INTO hadoop_output " +
+                             "(id,name,value) VALUES (?,?,?);";
+  private String nullExpected = "INSERT INTO hadoop_output VALUES (?,?,?);"; 
+  
+  private DBOutputFormat<DBWritable, NullWritable> format 
+    = new DBOutputFormat<DBWritable, NullWritable>();
+  
+  public void testConstructQuery() {  
+    String actual = format.constructQuery("hadoop_output", fieldNames);
+    assertEquals(expected, actual);
+    
+    actual = format.constructQuery("hadoop_output", nullFieldNames);
+    assertEquals(nullExpected, actual);
+  }
+  
+  public void testSetOutput() throws IOException {
+    Job job = new Job(new Configuration());
+    DBOutputFormat.setOutput(job, "hadoop_output", fieldNames);
+    
+    DBConfiguration dbConf = new DBConfiguration(job.getConfiguration());
+    String actual = format.constructQuery(dbConf.getOutputTableName()
+        , dbConf.getOutputFieldNames());
+    
+    assertEquals(expected, actual);
+    
+    job = new Job(new Configuration());
+    dbConf = new DBConfiguration(job.getConfiguration());
+    DBOutputFormat.setOutput(job, "hadoop_output", nullFieldNames.length);
+    assertNull(dbConf.getOutputFieldNames());
+    assertEquals(nullFieldNames.length, dbConf.getOutputFieldCount());
+    
+    actual = format.constructQuery(dbConf.getOutputTableName()
+        , new String[dbConf.getOutputFieldCount()]);
+    
+    assertEquals(nullExpected, actual);
+  }
+  
+}

+ 222 - 0
src/test/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java

@@ -0,0 +1,222 @@
+/**
+ * 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.db;
+
+import java.sql.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.examples.DBCountPageView;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.db.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.hadoop.util.StringUtils;
+import org.hsqldb.Server;
+
+/**
+ * Test aspects of DataDrivenDBInputFormat
+ */
+public class TestDataDrivenDBInputFormat extends HadoopTestCase {
+
+  private static final Log LOG = LogFactory.getLog(
+      TestDataDrivenDBInputFormat.class);
+
+  private static final String DB_NAME = "dddbif";
+  private static final String DB_URL = 
+    "jdbc:hsqldb:hsql://localhost/" + DB_NAME;
+  private static final String DRIVER_CLASS = "org.hsqldb.jdbcDriver";
+
+  private Server server;
+  private Connection connection;
+
+  private static final String OUT_DIR;
+
+  public TestDataDrivenDBInputFormat() throws IOException {
+    super(LOCAL_MR, LOCAL_FS, 1, 1);
+  }
+
+  static {
+    OUT_DIR = System.getProperty("test.build.data", "/tmp") + "/dddbifout";
+  }
+
+  private void startHsqldbServer() {
+    if (null == server) {
+      server = new Server();
+      server.setDatabasePath(0,
+          System.getProperty("test.build.data", "/tmp") + "/" + DB_NAME);
+      server.setDatabaseName(0, DB_NAME);
+      server.start();
+    }
+  }
+
+  private void createConnection(String driverClassName,
+      String url) throws Exception {
+
+    Class.forName(driverClassName);
+    connection = DriverManager.getConnection(url);
+    connection.setAutoCommit(false);
+  }
+
+  private void shutdown() {
+    try {
+      connection.commit();
+      connection.close();
+      connection = null;
+    }catch (Throwable ex) {
+      LOG.warn("Exception occurred while closing connection :"
+          + StringUtils.stringifyException(ex));
+    } finally {
+      try {
+        if(server != null) {
+          server.shutdown();
+        }
+      }catch (Throwable ex) {
+        LOG.warn("Exception occurred while shutting down HSQLDB :"
+            + StringUtils.stringifyException(ex));
+      }
+      server = null;
+    }
+  }
+
+  private void initialize(String driverClassName, String url)
+      throws Exception {
+    startHsqldbServer();
+    createConnection(driverClassName, url);
+  }
+
+  public void setUp() throws Exception {
+    initialize(DRIVER_CLASS, DB_URL);
+    super.setUp();
+  }
+
+  public void tearDown() throws Exception {
+    super.tearDown();
+    shutdown();
+  }
+
+
+
+  public static class DateCol implements DBWritable, WritableComparable {
+    Date d;
+
+    public String toString() {
+      return d.toString();
+    }
+
+    public void readFields(ResultSet rs) throws SQLException {
+      d = rs.getDate(1);
+    }
+
+    public void write(PreparedStatement ps) {
+      // not needed.
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      long v = in.readLong();
+      d = new Date(v);
+    }
+
+    public void write(DataOutput out) throws IOException {
+      out.writeLong(d.getTime());
+    }
+
+    @Override
+    public int hashCode() {
+      return (int) d.getTime();
+    }
+
+    @Override
+    public int compareTo(Object o) {
+      if (o instanceof DateCol) {
+        Long v = Long.valueOf(d.getTime());
+        Long other = Long.valueOf(((DateCol) o).d.getTime());
+        return v.compareTo(other);
+      } else {
+        return -1;
+      }
+    }
+  }
+
+  public static class ValMapper
+      extends Mapper<Object, Object, Object, NullWritable> {
+    public void map(Object k, Object v, Context c)
+        throws IOException, InterruptedException {
+      c.write(v, NullWritable.get());
+    }
+  }
+
+  public void testDateSplits() throws Exception {
+    Statement s = connection.createStatement();
+    final String DATE_TABLE = "datetable";
+    final String COL = "foo";
+    try {
+      // delete the table if it already exists.
+      s.executeUpdate("DROP TABLE " + DATE_TABLE);
+    } catch (SQLException e) {
+    }
+
+    // Create the table.
+    s.executeUpdate("CREATE TABLE " + DATE_TABLE + "(" + COL + " TIMESTAMP)");
+    s.executeUpdate("INSERT INTO " + DATE_TABLE + " VALUES('2010-04-01')");
+    s.executeUpdate("INSERT INTO " + DATE_TABLE + " VALUES('2010-04-02')");
+    s.executeUpdate("INSERT INTO " + DATE_TABLE + " VALUES('2010-05-01')");
+    s.executeUpdate("INSERT INTO " + DATE_TABLE + " VALUES('2011-04-01')");
+
+    // commit this tx.
+    connection.commit();
+
+    Configuration conf = new Configuration();
+    conf.set("fs.defaultFS", "file:///");
+    FileSystem fs = FileSystem.getLocal(conf);
+    fs.delete(new Path(OUT_DIR), true);
+
+    // now do a dd import
+    Job job = new Job(conf);
+    job.setMapperClass(ValMapper.class);
+    job.setReducerClass(Reducer.class);
+    job.setMapOutputKeyClass(DateCol.class);
+    job.setMapOutputValueClass(NullWritable.class);
+    job.setOutputKeyClass(DateCol.class);
+    job.setOutputValueClass(NullWritable.class);
+    job.setNumReduceTasks(1);
+    job.getConfiguration().setInt("mapreduce.map.tasks", 2);
+    FileOutputFormat.setOutputPath(job, new Path(OUT_DIR));
+    DBConfiguration.configureDB(job.getConfiguration(), DRIVER_CLASS,
+        DB_URL, null, null);
+    DataDrivenDBInputFormat.setInput(job, DateCol.class, DATE_TABLE, null,
+        COL, COL);
+
+    boolean ret = job.waitForCompletion(true);
+    assertTrue("job failed", ret);
+
+    // Check to see that we imported as much as we thought we did.
+    assertEquals("Did not get all the records", 4,
+        job.getCounters().findCounter("org.apache.hadoop.mapred.Task$Counter",
+          "REDUCE_OUTPUT_RECORDS").getValue());
+  }
+}

+ 106 - 0
src/test/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java

@@ -0,0 +1,106 @@
+/**
+ * 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.db;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+public class TestIntegerSplitter extends TestCase {
+  private long [] toLongArray(List<Long> in) {
+    long [] out = new long[in.size()];
+    for (int i = 0; i < in.size(); i++) {
+      out[i] = in.get(i).longValue();
+    }
+
+    return out;
+  }
+
+  public String formatLongArray(long [] ar) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("[");
+    boolean first = true;
+    for (long val : ar) {
+      if (!first) {
+        sb.append(", ");
+      }
+
+      sb.append(Long.toString(val));
+      first = false;
+    }
+
+    sb.append("]");
+    return sb.toString();
+  }
+
+  public void assertLongArrayEquals(long [] expected, long [] actual) {
+    for (int i = 0; i < expected.length; i++) {
+      try {
+        assertEquals("Failure at position " + i + "; got " + actual[i]
+            + " instead of " + expected[i] + "; actual array is " + formatLongArray(actual),
+            expected[i], actual[i]);
+      } catch (ArrayIndexOutOfBoundsException oob) {
+        fail("Expected array with " + expected.length + " elements; got " + actual.length
+            + ". Actual array is " + formatLongArray(actual));
+      }
+    }
+
+    if (actual.length > expected.length) {
+      fail("Actual array has " + actual.length + " elements; expected " + expected.length
+          + ". ACtual array is " + formatLongArray(actual));
+    }
+  }
+
+  public void testEvenSplits() throws SQLException {
+    List<Long> splits = new IntegerSplitter().split(10, 0, 100);
+    long [] expected = { 0, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100 };
+    assertLongArrayEquals(expected, toLongArray(splits));
+  }
+
+  public void testOddSplits() throws SQLException {
+    List<Long> splits = new IntegerSplitter().split(10, 0, 95);
+    long [] expected = { 0, 9, 18, 27, 36, 45, 54, 63, 72, 81, 90, 95 };
+    assertLongArrayEquals(expected, toLongArray(splits));
+
+  }
+
+  public void testSingletonSplit() throws SQLException {
+    List<Long> splits = new IntegerSplitter().split(1, 5, 5);
+    long [] expected = { 5, 5 };
+    assertLongArrayEquals(expected, toLongArray(splits));
+  }
+
+  public void testSingletonSplit2() throws SQLException {
+    // Same test, but overly-high numSplits
+    List<Long> splits = new IntegerSplitter().split(5, 5, 5);
+    long [] expected = { 5, 5 };
+    assertLongArrayEquals(expected, toLongArray(splits));
+  }
+
+  public void testTooManySplits() throws SQLException {
+    List<Long> splits = new IntegerSplitter().split(5, 3, 5);
+    long [] expected = { 3, 4, 5 };
+    assertLongArrayEquals(expected, toLongArray(splits));
+  }
+
+}
+

+ 128 - 0
src/test/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java

@@ -0,0 +1,128 @@
+/**
+ * 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.db;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+public class TestTextSplitter extends TestCase {
+
+  public String formatArray(Object [] ar) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("[");
+    boolean first = true;
+    for (Object val : ar) {
+      if (!first) {
+        sb.append(", ");
+      }
+
+      sb.append(val.toString());
+      first = false;
+    }
+
+    sb.append("]");
+    return sb.toString();
+  }
+
+  public void assertArrayEquals(Object [] expected, Object [] actual) {
+    for (int i = 0; i < expected.length; i++) {
+      try {
+        assertEquals("Failure at position " + i + "; got " + actual[i]
+            + " instead of " + expected[i] + "; actual array is " + formatArray(actual),
+            expected[i], actual[i]);
+      } catch (ArrayIndexOutOfBoundsException oob) {
+        fail("Expected array with " + expected.length + " elements; got " + actual.length
+            + ". Actual array is " + formatArray(actual));
+      }
+    }
+
+    if (actual.length > expected.length) {
+      fail("Actual array has " + actual.length + " elements; expected " + expected.length
+          + ". Actual array is " + formatArray(actual));
+    }
+  }
+
+  public void testStringConvertEmpty() {
+    TextSplitter splitter = new TextSplitter();
+    BigDecimal emptyBigDec = splitter.stringToBigDecimal("");
+    assertEquals(BigDecimal.ZERO, emptyBigDec);
+  }
+
+  public void testBigDecConvertEmpty() {
+    TextSplitter splitter = new TextSplitter();
+    String emptyStr = splitter.bigDecimalToString(BigDecimal.ZERO);
+    assertEquals("", emptyStr);
+  }
+
+  public void testConvertA() {
+    TextSplitter splitter = new TextSplitter();
+    String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("A"));
+    assertEquals("A", out);
+  }
+
+  public void testConvertZ() {
+    TextSplitter splitter = new TextSplitter();
+    String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("Z"));
+    assertEquals("Z", out);
+  }
+
+  public void testConvertThreeChars() {
+    TextSplitter splitter = new TextSplitter();
+    String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("abc"));
+    assertEquals("abc", out);
+  }
+
+  public void testConvertStr() {
+    TextSplitter splitter = new TextSplitter();
+    String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("big str"));
+    assertEquals("big str", out);
+  }
+
+  public void testConvertChomped() {
+    TextSplitter splitter = new TextSplitter();
+    String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("AVeryLongStringIndeed"));
+    assertEquals("AVeryLon", out);
+  }
+
+  public void testAlphabetSplit() throws SQLException {
+    // This should give us 25 splits, one per letter.
+    TextSplitter splitter = new TextSplitter();
+    List<String> splits = splitter.split(25, "A", "Z", "");
+    String [] expected = { "A", "B", "C", "D", "E", "F", "G", "H", "I", "J", "K",
+        "L", "M", "N", "O", "P", "Q", "R", "S", "T", "U", "V", "W", "X", "Y", "Z" };
+    assertArrayEquals(expected, splits.toArray(new String [0]));
+  }
+
+  public void testCommonPrefix() throws SQLException {
+    // Splits between 'Hand' and 'Hardy'
+    TextSplitter splitter = new TextSplitter();
+    List<String> splits = splitter.split(5, "nd", "rdy", "Ha");
+    // Don't check for exact values in the middle, because the splitter generates some
+    // ugly Unicode-isms. But do check that we get multiple splits and that it starts
+    // and ends on the correct points.
+    assertEquals("Hand", splits.get(0));
+    assertEquals("Hardy", splits.get(splits.size() -1));
+    assertEquals(6, splits.size());
+  }
+}
+

+ 124 - 0
src/test/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java

@@ -0,0 +1,124 @@
+/**
+ * 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.fieldsel;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+
+import junit.framework.TestCase;
+import java.text.NumberFormat;
+
+public class TestMRFieldSelection extends TestCase {
+
+private static NumberFormat idFormat = NumberFormat.getInstance();
+  static {
+    idFormat.setMinimumIntegerDigits(4);
+    idFormat.setGroupingUsed(false);
+  }
+
+  public void testFieldSelection() throws Exception {
+    launch();
+  }
+  private static Path testDir = new Path(
+    System.getProperty("test.build.data", "/tmp"), "field");
+  
+  public static void launch() throws Exception {
+    Configuration conf = new Configuration();
+    FileSystem fs = FileSystem.get(conf);
+    int numOfInputLines = 10;
+
+    Path outDir = new Path(testDir, "output_for_field_selection_test");
+    Path inDir = new Path(testDir, "input_for_field_selection_test");
+
+    StringBuffer inputData = new StringBuffer();
+    StringBuffer expectedOutput = new StringBuffer();
+    constructInputOutputData(inputData, expectedOutput, numOfInputLines);
+    
+    conf.set(FieldSelectionHelper.DATA_FIELD_SEPERATOR, "-");
+    conf.set(FieldSelectionHelper.MAP_OUTPUT_KEY_VALUE_SPEC, "6,5,1-3:0-");
+    conf.set(
+      FieldSelectionHelper.REDUCE_OUTPUT_KEY_VALUE_SPEC, ":4,3,2,1,0,0-");
+    Job job = MapReduceTestUtil.createJob(conf, inDir, outDir,
+      1, 1, inputData.toString());
+    job.setMapperClass(FieldSelectionMapper.class);
+    job.setReducerClass(FieldSelectionReducer.class);
+    job.setOutputKeyClass(Text.class);
+    job.setOutputValueClass(Text.class);
+    job.setNumReduceTasks(1);
+
+    job.waitForCompletion(true);
+    assertTrue("Job Failed!", job.isSuccessful());
+
+    //
+    // Finally, we compare the reconstructed answer key with the
+    // original one.  Remember, we need to ignore zero-count items
+    // in the original key.
+    //
+    String outdata = MapReduceTestUtil.readOutput(outDir, conf);
+    assertEquals("Outputs doesnt match.",expectedOutput.toString(), outdata);
+    fs.delete(outDir, true);
+  }
+
+  public static void constructInputOutputData(StringBuffer inputData,
+      StringBuffer expectedOutput, int numOfInputLines) {
+    for (int i = 0; i < numOfInputLines; i++) {
+      inputData.append(idFormat.format(i));
+      inputData.append("-").append(idFormat.format(i+1));
+      inputData.append("-").append(idFormat.format(i+2));
+      inputData.append("-").append(idFormat.format(i+3));
+      inputData.append("-").append(idFormat.format(i+4));
+      inputData.append("-").append(idFormat.format(i+5));
+      inputData.append("-").append(idFormat.format(i+6));
+      inputData.append("\n");
+
+      expectedOutput.append(idFormat.format(i+3));
+      expectedOutput.append("-" ).append (idFormat.format(i+2));
+      expectedOutput.append("-" ).append (idFormat.format(i+1));
+      expectedOutput.append("-" ).append (idFormat.format(i+5));
+      expectedOutput.append("-" ).append (idFormat.format(i+6));
+
+      expectedOutput.append("-" ).append (idFormat.format(i+6));
+      expectedOutput.append("-" ).append (idFormat.format(i+5));
+      expectedOutput.append("-" ).append (idFormat.format(i+1));
+      expectedOutput.append("-" ).append (idFormat.format(i+2));
+      expectedOutput.append("-" ).append (idFormat.format(i+3));
+      expectedOutput.append("-" ).append (idFormat.format(i+0));
+      expectedOutput.append("-" ).append (idFormat.format(i+1));
+      expectedOutput.append("-" ).append (idFormat.format(i+2));
+      expectedOutput.append("-" ).append (idFormat.format(i+3));
+      expectedOutput.append("-" ).append (idFormat.format(i+4));
+      expectedOutput.append("-" ).append (idFormat.format(i+5));
+      expectedOutput.append("-" ).append (idFormat.format(i+6));
+      expectedOutput.append("\n");
+    }
+    System.out.println("inputData:");
+    System.out.println(inputData.toString());
+    System.out.println("ExpectedData:");
+    System.out.println(expectedOutput.toString());
+  }
+  
+  /**
+   * Launches all the tasks in order.
+   */
+  public static void main(String[] argv) throws Exception {
+    launch();
+  }
+}

+ 1174 - 0
src/test/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java

@@ -0,0 +1,1174 @@
+/**
+ * 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.OutputStream;
+import java.net.URI;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.zip.GZIPOutputStream;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+
+public class TestCombineFileInputFormat extends TestCase {
+
+  private static final String rack1[] = new String[] {
+    "/r1"
+  };
+  private static final String hosts1[] = new String[] {
+    "host1.rack1.com"
+  };
+  private static final String rack2[] = new String[] {
+    "/r2"
+  };
+  private static final String hosts2[] = new String[] {
+    "host2.rack2.com"
+  };
+  private static final String rack3[] = new String[] {
+    "/r3"
+  };
+  private static final String hosts3[] = new String[] {
+    "host3.rack3.com"
+  };
+  final Path inDir = new Path("/racktesting");
+  final Path outputPath = new Path("/output");
+  final Path dir1 = new Path(inDir, "/dir1");
+  final Path dir2 = new Path(inDir, "/dir2");
+  final Path dir3 = new Path(inDir, "/dir3");
+  final Path dir4 = new Path(inDir, "/dir4");
+  final Path dir5 = new Path(inDir, "/dir5");
+
+  static final int BLOCKSIZE = 1024;
+  static final byte[] databuf = new byte[BLOCKSIZE];
+
+  /** Dummy class to extend CombineFileInputFormat*/
+  private class DummyInputFormat extends CombineFileInputFormat<Text, Text> {
+    @Override
+    public RecordReader<Text,Text> createRecordReader(InputSplit split, 
+        TaskAttemptContext context) throws IOException {
+      return null;
+    }
+  }
+
+  /** Dummy class to extend CombineFileInputFormat. It allows 
+   * non-existent files to be passed into the CombineFileInputFormat, allows
+   * for easy testing without having to create real files.
+   */
+  private class DummyInputFormat1 extends DummyInputFormat {
+    @Override
+    protected List<FileStatus> listStatus(JobContext job) throws IOException {
+      Path[] files = getInputPaths(job);
+      List<FileStatus> results = new ArrayList<FileStatus>();
+      for (int i = 0; i < files.length; i++) {
+        Path p = files[i];
+        FileSystem fs = p.getFileSystem(job.getConfiguration());
+        results.add(fs.getFileStatus(p));
+      }
+      return results;
+    }
+  }
+
+  /** Dummy class to extend CombineFileInputFormat. It allows
+   * testing with files having missing blocks without actually removing replicas.
+   */
+  public static class MissingBlockFileSystem extends DistributedFileSystem {
+    String fileWithMissingBlocks;
+
+    @Override
+    public void initialize(URI name, Configuration conf) throws IOException {
+      fileWithMissingBlocks = "";
+      super.initialize(name, conf);
+    }
+
+    @Override
+    public BlockLocation[] getFileBlockLocations(
+        FileStatus stat, long start, long len) throws IOException {
+      if (stat.isDir()) {
+        return null;
+      }
+      System.out.println("File " + stat.getPath());
+      String name = stat.getPath().toUri().getPath();
+      BlockLocation[] locs =
+        super.getFileBlockLocations(stat, start, len);
+      if (name.equals(fileWithMissingBlocks)) {
+        System.out.println("Returing missing blocks for " + fileWithMissingBlocks);
+        locs[0] = new BlockLocation(new String[0], new String[0],
+            locs[0].getOffset(), locs[0].getLength());
+      }
+      return locs;
+    }
+
+    public void setFileWithMissingBlocks(String f) {
+      fileWithMissingBlocks = f;
+    }
+  }
+
+  private static final String DUMMY_KEY = "dummy.rr.key";
+
+  private static class DummyRecordReader extends RecordReader<Text, Text> {
+    private TaskAttemptContext context;
+    private CombineFileSplit s;
+    private int idx;
+    private boolean used;
+
+    public DummyRecordReader(CombineFileSplit split, TaskAttemptContext context,
+        Integer i) {
+      this.context = context;
+      this.idx = i;
+      this.s = split;
+      this.used = true;
+    }
+
+    /** @return a value specified in the context to check whether the
+     * context is properly updated by the initialize() method.
+     */
+    public String getDummyConfVal() {
+      return this.context.getConfiguration().get(DUMMY_KEY);
+    }
+
+    public void initialize(InputSplit split, TaskAttemptContext context) {
+      this.context = context;
+      this.s = (CombineFileSplit) split;
+
+      // By setting used to true in the c'tor, but false in initialize,
+      // we can check that initialize() is always called before use
+      // (e.g., in testReinit()).
+      this.used = false;
+    }
+
+    public boolean nextKeyValue() {
+      boolean ret = !used;
+      this.used = true;
+      return ret;
+    }
+
+    public Text getCurrentKey() {
+      return new Text(this.context.getConfiguration().get(DUMMY_KEY));
+    }
+
+    public Text getCurrentValue() {
+      return new Text(this.s.getPath(idx).toString());
+    }
+
+    public float getProgress() {
+      return used ? 1.0f : 0.0f;
+    }
+
+    public void close() {
+    }
+  }
+
+  /** Extend CFIF to use CFRR with DummyRecordReader */
+  private class ChildRRInputFormat extends CombineFileInputFormat<Text, Text> {
+    @SuppressWarnings("unchecked")
+    @Override
+    public RecordReader<Text,Text> createRecordReader(InputSplit split, 
+        TaskAttemptContext context) throws IOException {
+      return new CombineFileRecordReader((CombineFileSplit) split, context,
+          (Class) DummyRecordReader.class);
+    }
+  }
+
+  public void testRecordReaderInit() throws InterruptedException, IOException {
+    // Test that we properly initialize the child recordreader when
+    // CombineFileInputFormat and CombineFileRecordReader are used.
+
+    TaskAttemptID taskId = new TaskAttemptID("jt", 0, true, 0, 0);
+    Configuration conf1 = new Configuration();
+    conf1.set(DUMMY_KEY, "STATE1");
+    TaskAttemptContext context1 = new TaskAttemptContext(conf1, taskId);
+
+    // This will create a CombineFileRecordReader that itself contains a
+    // DummyRecordReader.
+    InputFormat inputFormat = new ChildRRInputFormat();
+
+    Path [] files = { new Path("file1") };
+    long [] lengths = { 1 };
+
+    CombineFileSplit split = new CombineFileSplit(files, lengths);
+
+    RecordReader rr = inputFormat.createRecordReader(split, context1);
+    assertTrue("Unexpected RR type!", rr instanceof CombineFileRecordReader);
+
+    // Verify that the initial configuration is the one being used.
+    // Right after construction the dummy key should have value "STATE1"
+    assertEquals("Invalid initial dummy key value", "STATE1",
+      rr.getCurrentKey().toString());
+
+    // Switch the active context for the RecordReader...
+    Configuration conf2 = new Configuration();
+    conf2.set(DUMMY_KEY, "STATE2");
+    TaskAttemptContext context2 = new TaskAttemptContext(conf2, taskId);
+    rr.initialize(split, context2);
+
+    // And verify that the new context is updated into the child record reader.
+    assertEquals("Invalid secondary dummy key value", "STATE2",
+      rr.getCurrentKey().toString());
+  }
+
+  public void testReinit() throws Exception {
+    // Test that a split containing multiple files works correctly,
+    // with the child RecordReader getting its initialize() method
+    // called a second time.
+    TaskAttemptID taskId = new TaskAttemptID("jt", 0, true, 0, 0);
+    Configuration conf = new Configuration();
+    TaskAttemptContext context = new TaskAttemptContext(conf, taskId);
+
+    // This will create a CombineFileRecordReader that itself contains a
+    // DummyRecordReader.
+    InputFormat inputFormat = new ChildRRInputFormat();
+
+    Path [] files = { new Path("file1"), new Path("file2") };
+    long [] lengths = { 1, 1 };
+
+    CombineFileSplit split = new CombineFileSplit(files, lengths);
+    RecordReader rr = inputFormat.createRecordReader(split, context);
+    assertTrue("Unexpected RR type!", rr instanceof CombineFileRecordReader);
+
+    // first initialize() call comes from MapTask. We'll do it here.
+    rr.initialize(split, context);
+
+    // First value is first filename.
+    assertTrue(rr.nextKeyValue());
+    assertEquals("file1", rr.getCurrentValue().toString());
+
+    // The inner RR will return false, because it only emits one (k, v) pair.
+    // But there's another sub-split to process. This returns true to us.
+    assertTrue(rr.nextKeyValue());
+    
+    // And the 2nd rr will have its initialize method called correctly.
+    assertEquals("file2", rr.getCurrentValue().toString());
+    
+    // But after both child RR's have returned their singleton (k, v), this
+    // should also return false.
+    assertFalse(rr.nextKeyValue());
+  }
+
+  public void testSplitPlacement() throws IOException {
+    MiniDFSCluster dfs = null;
+    FileSystem fileSys = null;
+    try {
+      /* Start 3 datanodes, one each in rack r1, r2, r3. Create five files
+       * 1) file1 and file5, just after starting the datanode on r1, with 
+       *    a repl factor of 1, and,
+       * 2) file2, just after starting the datanode on r2, with 
+       *    a repl factor of 2, and,
+       * 3) file3, file4 after starting the all three datanodes, with a repl 
+       *    factor of 3.
+       * At the end, file1, file5 will be present on only datanode1, file2 will 
+       * be present on datanode 1 and datanode2 and 
+       * file3, file4 will be present on all datanodes. 
+       */
+      Configuration conf = new Configuration();
+      conf.setBoolean("dfs.replication.considerLoad", false);
+      dfs = new MiniDFSCluster(conf, 1, true, rack1, hosts1);
+      dfs.waitActive();
+
+      fileSys = dfs.getFileSystem();
+      if (!fileSys.mkdirs(inDir)) {
+        throw new IOException("Mkdirs failed to create " + inDir.toString());
+      }
+      Path file1 = new Path(dir1 + "/file1");
+      writeFile(conf, file1, (short)1, 1);
+      // create another file on the same datanode
+      Path file5 = new Path(dir5 + "/file5");
+      writeFile(conf, file5, (short)1, 1);
+      // split it using a CombinedFile input format
+      DummyInputFormat inFormat = new DummyInputFormat();
+      Job job = new Job(conf);
+      FileInputFormat.setInputPaths(job, dir1 + "," + dir5);
+      List<InputSplit> splits = inFormat.getSplits(job);
+      System.out.println("Made splits(Test0): " + splits.size());
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test0): " + split);
+      }
+      assertEquals(splits.size(), 1);
+      CombineFileSplit fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(2, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+      assertEquals(file5.getName(), fileSplit.getPath(1).getName());
+      assertEquals(0, fileSplit.getOffset(1));
+      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+      
+      dfs.startDataNodes(conf, 1, true, null, rack2, hosts2, null);
+      dfs.waitActive();
+
+      // create file on two datanodes.
+      Path file2 = new Path(dir2 + "/file2");
+      writeFile(conf, file2, (short)2, 2);
+
+      // split it using a CombinedFile input format
+      inFormat = new DummyInputFormat();
+      FileInputFormat.setInputPaths(job, dir1 + "," + dir2);
+      inFormat.setMinSplitSizeRack(BLOCKSIZE);
+      splits = inFormat.getSplits(job);
+      System.out.println("Made splits(Test1): " + splits.size());
+
+      // make sure that each split has different locations
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test1): " + split);
+      }
+      assertEquals(splits.size(), 2);
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(fileSplit.getNumPaths(), 2);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(fileSplit.getNumPaths(), 1);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file1.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
+
+      // create another file on 3 datanodes and 3 racks.
+      dfs.startDataNodes(conf, 1, true, null, rack3, hosts3, null);
+      dfs.waitActive();
+      Path file3 = new Path(dir3 + "/file3");
+      writeFile(conf, new Path(dir3 + "/file3"), (short)3, 3);
+      inFormat = new DummyInputFormat();
+      FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3);
+      inFormat.setMinSplitSizeRack(BLOCKSIZE);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test2): " + split);
+      }
+      assertEquals(splits.size(), 3);
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(fileSplit.getNumPaths(), 3);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(2).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
+      assertEquals(fileSplit.getLength(2), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], hosts3[0]); // should be on r3
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(fileSplit.getNumPaths(), 2);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(fileSplit.getNumPaths(), 1);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file1.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
+
+      // create file4 on all three racks
+      Path file4 = new Path(dir4 + "/file4");
+      writeFile(conf, file4, (short)3, 3);
+      inFormat = new DummyInputFormat();
+      FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      inFormat.setMinSplitSizeRack(BLOCKSIZE);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test3): " + split);
+      }
+      assertEquals(splits.size(), 3);
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(fileSplit.getNumPaths(), 6);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(2).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
+      assertEquals(fileSplit.getLength(2), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], hosts3[0]); // should be on r3
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(fileSplit.getNumPaths(), 2);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(fileSplit.getNumPaths(), 1);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file1.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
+
+      // maximum split size is 2 blocks 
+      inFormat = new DummyInputFormat();
+      inFormat.setMinSplitSizeNode(BLOCKSIZE);
+      inFormat.setMaxSplitSize(2*BLOCKSIZE);
+      FileInputFormat.setInputPaths(job, 
+        dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test4): " + split);
+      }
+      assertEquals(splits.size(), 5);
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(fileSplit.getNumPaths(), 2);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(fileSplit.getPath(0).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(0), 2 * BLOCKSIZE);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file4.getName());
+      assertEquals(fileSplit.getOffset(1), 0);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(fileSplit.getNumPaths(), 2);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file4.getName());
+      assertEquals(fileSplit.getOffset(0), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file4.getName());
+      assertEquals(fileSplit.getOffset(1), 2 * BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
+
+      // maximum split size is 3 blocks 
+      inFormat = new DummyInputFormat();
+      inFormat.setMinSplitSizeNode(BLOCKSIZE);
+      inFormat.setMaxSplitSize(3*BLOCKSIZE);
+      FileInputFormat.setInputPaths(job, 
+        dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test5): " + split);
+      }
+      assertEquals(splits.size(), 4);
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(fileSplit.getNumPaths(), 3);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(2).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
+      assertEquals(fileSplit.getLength(2), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(fileSplit.getPath(0).getName(), file4.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file4.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(2).getName(), file4.getName());
+      assertEquals(fileSplit.getOffset(2),  2 * BLOCKSIZE);
+      assertEquals(fileSplit.getLength(2), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(fileSplit.getNumPaths(), 2);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], "host2.rack2.com");
+      fileSplit = (CombineFileSplit) splits.get(3);
+      assertEquals(fileSplit.getNumPaths(), 1);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file1.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], "host1.rack1.com");
+
+      // maximum split size is 4 blocks 
+      inFormat = new DummyInputFormat();
+      inFormat.setMaxSplitSize(4*BLOCKSIZE);
+      FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test6): " + split);
+      }
+      assertEquals(splits.size(), 3);
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(fileSplit.getNumPaths(), 4);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(2).getName(), file3.getName());
+      assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
+      assertEquals(fileSplit.getLength(2), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(fileSplit.getNumPaths(), 4);
+      assertEquals(fileSplit.getPath(0).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(1).getName(), file2.getName());
+      assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(1), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(2).getName(), file4.getName());
+      assertEquals(fileSplit.getOffset(2), BLOCKSIZE);
+      assertEquals(fileSplit.getLength(2), BLOCKSIZE);
+      assertEquals(fileSplit.getPath(3).getName(), file4.getName());
+      assertEquals(fileSplit.getOffset(3),  2 * BLOCKSIZE);
+      assertEquals(fileSplit.getLength(3), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], "host2.rack2.com");
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(fileSplit.getNumPaths(), 1);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getPath(0).getName(), file1.getName());
+      assertEquals(fileSplit.getOffset(0), 0);
+      assertEquals(fileSplit.getLength(0), BLOCKSIZE);
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
+
+      // maximum split size is 7 blocks and min is 3 blocks
+      inFormat = new DummyInputFormat();
+      inFormat.setMaxSplitSize(7*BLOCKSIZE);
+      inFormat.setMinSplitSizeNode(3*BLOCKSIZE);
+      inFormat.setMinSplitSizeRack(3*BLOCKSIZE);
+      FileInputFormat.setInputPaths(job, 
+        dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test7): " + split);
+      }
+      assertEquals(splits.size(), 2);
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(fileSplit.getNumPaths(), 6);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(fileSplit.getNumPaths(), 3);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getLocations()[0], "host1.rack1.com");
+
+      // Rack 1 has file1, file2 and file3 and file4
+      // Rack 2 has file2 and file3 and file4
+      // Rack 3 has file3 and file4
+      // setup a filter so that only file1 and file2 can be combined
+      inFormat = new DummyInputFormat();
+      FileInputFormat.addInputPath(job, inDir);
+      inFormat.setMinSplitSizeRack(1); // everything is at least rack local
+      inFormat.createPool(new TestFilter(dir1), 
+                          new TestFilter(dir2));
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test1): " + split);
+      }
+      assertEquals(splits.size(), 3);
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(fileSplit.getNumPaths(), 2);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(fileSplit.getNumPaths(), 1);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(fileSplit.getNumPaths(), 6);
+      assertEquals(fileSplit.getLocations().length, 1);
+      assertEquals(fileSplit.getLocations()[0], hosts3[0]); // should be on r3
+
+      // measure performance when there are multiple pools and
+      // many files in each pool.
+      int numPools = 100;
+      int numFiles = 1000;
+      DummyInputFormat1 inFormat1 = new DummyInputFormat1();
+      for (int i = 0; i < numFiles; i++) {
+        FileInputFormat.setInputPaths(job, file1);
+      }
+      inFormat1.setMinSplitSizeRack(1); // everything is at least rack local
+      final Path dirNoMatch1 = new Path(inDir, "/dirxx");
+      final Path dirNoMatch2 = new Path(inDir, "/diryy");
+      for (int i = 0; i < numPools; i++) {
+        inFormat1.createPool(new TestFilter(dirNoMatch1), 
+                            new TestFilter(dirNoMatch2));
+      }
+      long start = System.currentTimeMillis();
+      splits = inFormat1.getSplits(job);
+      long end = System.currentTimeMillis();
+      System.out.println("Elapsed time for " + numPools + " pools " +
+                         " and " + numFiles + " files is " + 
+                         ((end - start)/1000) + " seconds.");
+
+      // This file has three whole blocks. If the maxsplit size is
+      // half the block size, then there should be six splits.
+      inFormat = new DummyInputFormat();
+      inFormat.setMaxSplitSize(BLOCKSIZE/2);
+      FileInputFormat.setInputPaths(job, dir3);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test8): " + split);
+      }
+      assertEquals(6, splits.size());
+
+    } finally {
+      if (dfs != null) {
+        dfs.shutdown();
+      }
+    }
+  }
+
+  static void writeFile(Configuration conf, Path name,
+      short replication, int numBlocks) throws IOException {
+    FileSystem fileSys = FileSystem.get(conf);
+
+    FSDataOutputStream stm = fileSys.create(name, true,
+                                            conf.getInt("io.file.buffer.size", 4096),
+                                            replication, (long)BLOCKSIZE);
+    writeDataAndSetReplication(fileSys, name, stm, replication, numBlocks);
+  }
+
+  // Creates the gzip file and return the FileStatus
+  static FileStatus writeGzipFile(Configuration conf, Path name,
+      short replication, int numBlocks) throws IOException {
+    FileSystem fileSys = FileSystem.get(conf);
+
+    GZIPOutputStream out = new GZIPOutputStream(fileSys.create(name, true, conf
+        .getInt("io.file.buffer.size", 4096), replication, (long) BLOCKSIZE));
+    writeDataAndSetReplication(fileSys, name, out, replication, numBlocks);
+    return fileSys.getFileStatus(name);
+  }
+
+  private static void writeDataAndSetReplication(FileSystem fileSys, Path name,
+      OutputStream out, short replication, int numBlocks) throws IOException {
+    for (int i = 0; i < numBlocks; i++) {
+      out.write(databuf);
+    }
+    out.close();
+    DFSTestUtil.waitReplication(fileSys, name, replication);
+  }
+  
+  public void testSplitPlacementForCompressedFiles() throws IOException {
+    MiniDFSCluster dfs = null;
+    FileSystem fileSys = null;
+    try {
+      /* Start 3 datanodes, one each in rack r1, r2, r3. Create five gzipped
+       *  files
+       * 1) file1 and file5, just after starting the datanode on r1, with 
+       *    a repl factor of 1, and,
+       * 2) file2, just after starting the datanode on r2, with 
+       *    a repl factor of 2, and,
+       * 3) file3, file4 after starting the all three datanodes, with a repl 
+       *    factor of 3.
+       * At the end, file1, file5 will be present on only datanode1, file2 will 
+       * be present on datanode 1 and datanode2 and 
+       * file3, file4 will be present on all datanodes. 
+       */
+      Configuration conf = new Configuration();
+      conf.setBoolean("dfs.replication.considerLoad", false);
+      dfs = new MiniDFSCluster(conf, 1, true, rack1, hosts1);
+      dfs.waitActive();
+
+      fileSys = dfs.getFileSystem();
+      if (!fileSys.mkdirs(inDir)) {
+        throw new IOException("Mkdirs failed to create " + inDir.toString());
+      }
+      Path file1 = new Path(dir1 + "/file1.gz");
+      FileStatus f1 = writeGzipFile(conf, file1, (short)1, 1);
+      // create another file on the same datanode
+      Path file5 = new Path(dir5 + "/file5.gz");
+      FileStatus f5 = writeGzipFile(conf, file5, (short)1, 1);
+      // split it using a CombinedFile input format
+      DummyInputFormat inFormat = new DummyInputFormat();
+      Job job = new Job(conf);
+      FileInputFormat.setInputPaths(job, dir1 + "," + dir5);
+      List<InputSplit> splits = inFormat.getSplits(job);
+      System.out.println("Made splits(Test0): " + splits.size());
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test0): " + split);
+      }
+      assertEquals(splits.size(), 1);
+      CombineFileSplit fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(2, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f1.getLen(), fileSplit.getLength(0));
+      assertEquals(file5.getName(), fileSplit.getPath(1).getName());
+      assertEquals(0, fileSplit.getOffset(1));
+      assertEquals(f5.getLen(), fileSplit.getLength(1));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+      
+      dfs.startDataNodes(conf, 1, true, null, rack2, hosts2, null);
+      dfs.waitActive();
+
+      // create file on two datanodes.
+      Path file2 = new Path(dir2 + "/file2.gz");
+      FileStatus f2 = writeGzipFile(conf, file2, (short)2, 2);
+
+      // split it using a CombinedFile input format
+      inFormat = new DummyInputFormat();
+      FileInputFormat.setInputPaths(job, dir1 + "," + dir2);
+      inFormat.setMinSplitSizeRack(f1.getLen());
+      splits = inFormat.getSplits(job);
+      System.out.println("Made splits(Test1): " + splits.size());
+
+      // make sure that each split has different locations
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test1): " + split);
+      }
+      assertEquals(2, splits.size());
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f2.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f1.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      // create another file on 3 datanodes and 3 racks.
+      dfs.startDataNodes(conf, 1, true, null, rack3, hosts3, null);
+      dfs.waitActive();
+      Path file3 = new Path(dir3 + "/file3.gz");
+      FileStatus f3 = writeGzipFile(conf, file3, (short)3, 3);
+      inFormat = new DummyInputFormat();
+      FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3);
+      inFormat.setMinSplitSizeRack(f1.getLen());
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test2): " + split);
+      }
+      assertEquals(3, splits.size());
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f3.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f2.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f1.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      // create file4 on all three racks
+      Path file4 = new Path(dir4 + "/file4.gz");
+      FileStatus f4 = writeGzipFile(conf, file4, (short)3, 3);
+      inFormat = new DummyInputFormat();
+      FileInputFormat.setInputPaths(job,
+          dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      inFormat.setMinSplitSizeRack(f1.getLen());
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test3): " + split);
+      }
+      assertEquals(3, splits.size());
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(2, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f3.getLen(), fileSplit.getLength(0));
+      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
+      assertEquals(0, fileSplit.getOffset(1));
+      assertEquals(f4.getLen(), fileSplit.getLength(1));
+      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f2.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f1.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      // maximum split size is file1's length
+      inFormat = new DummyInputFormat();
+      inFormat.setMinSplitSizeNode(f1.getLen());
+      inFormat.setMaxSplitSize(f1.getLen());
+      FileInputFormat.setInputPaths(job, 
+        dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test4): " + split);
+      }
+      assertEquals(4, splits.size());
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f3.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(file4.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f4.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f2.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(3);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f1.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      // maximum split size is twice file1's length
+      inFormat = new DummyInputFormat();
+      inFormat.setMinSplitSizeNode(f1.getLen());
+      inFormat.setMaxSplitSize(2 * f1.getLen());
+      FileInputFormat.setInputPaths(job, 
+        dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test5): " + split);
+      }
+      assertEquals(3, splits.size());
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(2, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f3.getLen(), fileSplit.getLength(0));
+      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
+      assertEquals(0, fileSplit.getOffset(1));
+      assertEquals(f4.getLen(), fileSplit.getLength(1));
+      assertEquals(hosts3[0], fileSplit.getLocations()[0]);
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f2.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f1.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      // maximum split size is 4 times file1's length 
+      inFormat = new DummyInputFormat();
+      inFormat.setMinSplitSizeNode(2 * f1.getLen());
+      inFormat.setMaxSplitSize(4 * f1.getLen());
+      FileInputFormat.setInputPaths(job,
+          dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test6): " + split);
+      }
+      assertEquals(2, splits.size());
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(2, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f3.getLen(), fileSplit.getLength(0));
+      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
+      assertEquals(0, fileSplit.getOffset(1));
+      assertEquals(f4.getLen(), fileSplit.getLength(1));
+      assertEquals(hosts3[0], fileSplit.getLocations()[0]);
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(2, fileSplit.getNumPaths());
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(f1.getLen(), fileSplit.getLength(0));
+      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
+      assertEquals(0, fileSplit.getOffset(1), BLOCKSIZE);
+      assertEquals(f2.getLen(), fileSplit.getLength(1));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      // maximum split size and min-split-size per rack is 4 times file1's length
+      inFormat = new DummyInputFormat();
+      inFormat.setMaxSplitSize(4 * f1.getLen());
+      inFormat.setMinSplitSizeRack(4 * f1.getLen());
+      FileInputFormat.setInputPaths(job, 
+        dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test7): " + split);
+      }
+      assertEquals(1, splits.size());
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(4, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+
+      // minimum split size per node is 4 times file1's length
+      inFormat = new DummyInputFormat();
+      inFormat.setMinSplitSizeNode(4 * f1.getLen());
+      FileInputFormat.setInputPaths(job, 
+        dir1 + "," + dir2 + "," + dir3 + "," + dir4);
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test8): " + split);
+      }
+      assertEquals(1, splits.size());
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(4, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+
+      // Rack 1 has file1, file2 and file3 and file4
+      // Rack 2 has file2 and file3 and file4
+      // Rack 3 has file3 and file4
+      // setup a filter so that only file1 and file2 can be combined
+      inFormat = new DummyInputFormat();
+      FileInputFormat.addInputPath(job, inDir);
+      inFormat.setMinSplitSizeRack(1); // everything is at least rack local
+      inFormat.createPool(new TestFilter(dir1), 
+                          new TestFilter(dir2));
+      splits = inFormat.getSplits(job);
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test9): " + split);
+      }
+      assertEquals(3, splits.size());
+      fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
+      fileSplit = (CombineFileSplit) splits.get(1);
+      assertEquals(1, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+      fileSplit = (CombineFileSplit) splits.get(2);
+      assertEquals(2, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
+
+      // measure performance when there are multiple pools and
+      // many files in each pool.
+      int numPools = 100;
+      int numFiles = 1000;
+      DummyInputFormat1 inFormat1 = new DummyInputFormat1();
+      for (int i = 0; i < numFiles; i++) {
+        FileInputFormat.setInputPaths(job, file1);
+      }
+      inFormat1.setMinSplitSizeRack(1); // everything is at least rack local
+      final Path dirNoMatch1 = new Path(inDir, "/dirxx");
+      final Path dirNoMatch2 = new Path(inDir, "/diryy");
+      for (int i = 0; i < numPools; i++) {
+        inFormat1.createPool(new TestFilter(dirNoMatch1), 
+                            new TestFilter(dirNoMatch2));
+      }
+      long start = System.currentTimeMillis();
+      splits = inFormat1.getSplits(job);
+      long end = System.currentTimeMillis();
+      System.out.println("Elapsed time for " + numPools + " pools " +
+                         " and " + numFiles + " files is " + 
+                         ((end - start)) + " milli seconds.");
+    } finally {
+      if (dfs != null) {
+        dfs.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test that CFIF can handle missing blocks.
+   */
+  public void testMissingBlocks() throws IOException {
+    String namenode = null;
+    MiniDFSCluster dfs = null;
+    FileSystem fileSys = null;
+    String testName = "testMissingBlocks";
+    try {
+      Configuration conf = new Configuration();
+      conf.set("fs.hdfs.impl", MissingBlockFileSystem.class.getName());
+      conf.setBoolean("dfs.replication.considerLoad", false);
+      dfs = new MiniDFSCluster(conf, 1, true, rack1, hosts1);
+      dfs.waitActive();
+
+      namenode = (dfs.getFileSystem()).getUri().getHost() + ":" +
+                 (dfs.getFileSystem()).getUri().getPort();
+
+      fileSys = dfs.getFileSystem();
+      if (!fileSys.mkdirs(inDir)) {
+        throw new IOException("Mkdirs failed to create " + inDir.toString());
+      }
+
+      Path file1 = new Path(dir1 + "/file1");
+      writeFile(conf, file1, (short)1, 1);
+      // create another file on the same datanode
+      Path file5 = new Path(dir5 + "/file5");
+      writeFile(conf, file5, (short)1, 1);
+
+      ((MissingBlockFileSystem)fileSys).setFileWithMissingBlocks(file1.toUri().getPath());
+      // split it using a CombinedFile input format
+      DummyInputFormat inFormat = new DummyInputFormat();
+      Job job = new Job(conf);
+      FileInputFormat.setInputPaths(job, dir1 + "," + dir5);
+      List<InputSplit> splits = inFormat.getSplits(job);
+      System.out.println("Made splits(Test0): " + splits.size());
+      for (InputSplit split : splits) {
+        System.out.println("File split(Test0): " + split);
+      }
+      assertEquals(splits.size(), 1);
+      CombineFileSplit fileSplit = (CombineFileSplit) splits.get(0);
+      assertEquals(2, fileSplit.getNumPaths());
+      assertEquals(1, fileSplit.getLocations().length);
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
+      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+      assertEquals(file5.getName(), fileSplit.getPath(1).getName());
+      assertEquals(0, fileSplit.getOffset(1));
+      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+
+    } finally {
+      if (dfs != null) {
+        dfs.shutdown();
+      }
+    }
+  }
+
+  static class TestFilter implements PathFilter {
+    private Path p;
+
+    // store a path prefix in this TestFilter
+    public TestFilter(Path p) {
+      this.p = p;
+    }
+
+    // returns true if the specified path matches the prefix stored
+    // in this TestFilter.
+    public boolean accept(Path path) {
+      if (path.toString().indexOf(p.toString()) == 0) {
+        return true;
+      }
+      return false;
+    }
+
+    public String toString() {
+      return "PathFilter:" + p;
+    }
+  }
+
+  /*
+   * Prints out the input splits for the specified files
+   */
+  private void splitRealFiles(String[] args) throws IOException {
+    Configuration conf = new Configuration();
+    Job job = new Job();
+    FileSystem fs = FileSystem.get(conf);
+    if (!(fs instanceof DistributedFileSystem)) {
+      throw new IOException("Wrong file system: " + fs.getClass().getName());
+    }
+    int blockSize = conf.getInt("dfs.block.size", 128 * 1024 * 1024);
+
+    DummyInputFormat inFormat = new DummyInputFormat();
+    for (int i = 0; i < args.length; i++) {
+      FileInputFormat.addInputPaths(job, args[i]);
+    }
+    inFormat.setMinSplitSizeRack(blockSize);
+    inFormat.setMaxSplitSize(10 * blockSize);
+
+    List<InputSplit> splits = inFormat.getSplits(job);
+    System.out.println("Total number of splits " + splits.size());
+    for (int i = 0; i < splits.size(); ++i) {
+      CombineFileSplit fileSplit = (CombineFileSplit) splits.get(i);
+      System.out.println("Split[" + i + "] " + fileSplit);
+    }
+  }
+
+  public static void main(String[] args) throws Exception{
+
+    // if there are some parameters specified, then use those paths
+    if (args.length != 0) {
+      TestCombineFileInputFormat test = new TestCombineFileInputFormat();
+      test.splitRealFiles(args);
+    } else {
+      TestCombineFileInputFormat test = new TestCombineFileInputFormat();
+      test.testSplitPlacement();
+    }
+  }
+}

+ 115 - 0
src/test/org/apache/hadoop/mapreduce/lib/input/TestDelegatingInputFormat.java

@@ -0,0 +1,115 @@
+/**
+ * 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.DataOutputStream;
+import java.io.IOException;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+
+public class TestDelegatingInputFormat extends TestCase {
+
+  @SuppressWarnings("unchecked")
+  public void testSplitting() throws Exception {
+    Job job = new Job();
+    MiniDFSCluster dfs = null;
+    try {
+      dfs = new MiniDFSCluster(job.getConfiguration(), 4, true, new String[] { "/rack0",
+         "/rack0", "/rack1", "/rack1" }, new String[] { "host0", "host1",
+         "host2", "host3" });
+      FileSystem fs = dfs.getFileSystem();
+
+      Path path = getPath("/foo/bar", fs);
+      Path path2 = getPath("/foo/baz", fs);
+      Path path3 = getPath("/bar/bar", fs);
+      Path path4 = getPath("/bar/baz", fs);
+
+      final int numSplits = 100;
+
+      FileInputFormat.setMaxInputSplitSize(job, 
+              fs.getFileStatus(path).getLen() / numSplits);
+      MultipleInputs.addInputPath(job, path, TextInputFormat.class,
+         MapClass.class);
+      MultipleInputs.addInputPath(job, path2, TextInputFormat.class,
+         MapClass2.class);
+      MultipleInputs.addInputPath(job, path3, KeyValueTextInputFormat.class,
+         MapClass.class);
+      MultipleInputs.addInputPath(job, path4, TextInputFormat.class,
+         MapClass2.class);
+      DelegatingInputFormat inFormat = new DelegatingInputFormat();
+
+      int[] bins = new int[3];
+      for (InputSplit split : (List<InputSplit>)inFormat.getSplits(job)) {
+       assertTrue(split instanceof TaggedInputSplit);
+       final TaggedInputSplit tis = (TaggedInputSplit) split;
+       int index = -1;
+
+       if (tis.getInputFormatClass().equals(KeyValueTextInputFormat.class)) {
+         // path3
+         index = 0;
+       } else if (tis.getMapperClass().equals(MapClass.class)) {
+         // path
+         index = 1;
+       } else {
+         // path2 and path4
+         index = 2;
+       }
+
+       bins[index]++;
+      }
+
+      assertEquals("count is not equal to num splits", numSplits, bins[0]);
+      assertEquals("count is not equal to num splits", numSplits, bins[1]);
+      assertEquals("count is not equal to 2 * num splits",
+        numSplits * 2, bins[2]);
+    } finally {
+      if (dfs != null) {
+       dfs.shutdown();
+      }
+    }
+  }
+
+  static Path getPath(final String location, final FileSystem fs)
+      throws IOException {
+    Path path = new Path(location);
+
+    // create a multi-block file on hdfs
+    DataOutputStream out = fs.create(path, true, 4096, (short) 2, 512, null);
+    for (int i = 0; i < 1000; ++i) {
+      out.writeChars("Hello\n");
+    }
+    out.close();
+
+    return path;
+  }
+
+  static class MapClass extends Mapper<String, String, String, String> {
+  }
+
+  static class MapClass2 extends MapClass {
+  }
+
+}

+ 227 - 0
src/test/org/apache/hadoop/mapreduce/lib/input/TestKeyValueTextInputFormat.java

@@ -0,0 +1,227 @@
+/**
+ * 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.*;
+import java.util.*;
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.compress.*;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.util.LineReader;
+import org.apache.hadoop.util.ReflectionUtils;
+
+public class TestKeyValueTextInputFormat extends TestCase {
+  private static final Log LOG =
+    LogFactory.getLog(TestKeyValueTextInputFormat.class.getName());
+
+  private static int MAX_LENGTH = 10000;
+  
+  private static Configuration defaultConf = new Configuration();
+  private static FileSystem localFs = null; 
+  static {
+    try {
+      localFs = FileSystem.getLocal(defaultConf);
+    } catch (IOException e) {
+      throw new RuntimeException("init failure", e);
+    }
+  }
+  private static Path workDir = 
+    new Path(new Path(System.getProperty("test.build.data", "."), "data"),
+             "TestKeyValueTextInputFormat");
+  
+  public void testFormat() throws Exception {
+    Job job = new Job(defaultConf);
+    Path file = new Path(workDir, "test.txt");
+
+    int seed = new Random().nextInt();
+    LOG.info("seed = "+seed);
+    Random random = new Random(seed);
+
+    localFs.delete(workDir, true);
+    FileInputFormat.setInputPaths(job, workDir);
+
+    // for a variety of lengths
+    for (int length = 0; length < MAX_LENGTH;
+         length+= random.nextInt(MAX_LENGTH/10)+1) {
+
+      LOG.debug("creating; entries = " + length);
+
+      // create a file with length entries
+      Writer writer = new OutputStreamWriter(localFs.create(file));
+      try {
+        for (int i = 0; i < length; i++) {
+          writer.write(Integer.toString(i*2));
+          writer.write("\t");
+          writer.write(Integer.toString(i));
+          writer.write("\n");
+        }
+      } finally {
+        writer.close();
+      }
+
+      KeyValueTextInputFormat format = new KeyValueTextInputFormat();
+      JobContext jobContext = new JobContext(job.getConfiguration(), new JobID());
+      List<InputSplit> splits = format.getSplits(jobContext);
+      LOG.debug("splitting: got =        " + splits.size());
+      
+      TaskAttemptContext context = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
+
+      // check each split
+      BitSet bits = new BitSet(length);
+      for (InputSplit split : splits) {
+        LOG.debug("split= " + split);
+        RecordReader<Text, Text> reader =
+          format.createRecordReader(split, context);
+        Class readerClass = reader.getClass();
+        assertEquals("reader class is KeyValueLineRecordReader.", KeyValueLineRecordReader.class, readerClass);        
+
+        reader.initialize(split, context);
+        try {
+          int count = 0;
+          while (reader.nextKeyValue()) {
+            int v = Integer.parseInt(reader.getCurrentValue().toString());
+            LOG.debug("read " + v);
+            if (bits.get(v)) {
+              LOG.warn("conflict with " + v + 
+                       " in split " + split +
+                       " at "+reader.getProgress());
+            }
+            assertFalse("Key in multiple partitions.", bits.get(v));
+            bits.set(v);
+            count++;
+          }
+          LOG.debug("split="+split+" count=" + count);
+        } finally {
+          reader.close();
+        }
+      }
+      assertEquals("Some keys in no partition.", length, bits.cardinality());
+
+    }
+  }
+  private LineReader makeStream(String str) throws IOException {
+    return new LineReader(new ByteArrayInputStream
+                                           (str.getBytes("UTF-8")), 
+                                           defaultConf);
+  }
+  
+  public void testUTF8() throws Exception {
+    LineReader in = makeStream("abcd\u20acbdcd\u20ac");
+    Text line = new Text();
+    in.readLine(line);
+    assertEquals("readLine changed utf8 characters", 
+                 "abcd\u20acbdcd\u20ac", line.toString());
+    in = makeStream("abc\u200axyz");
+    in.readLine(line);
+    assertEquals("split on fake newline", "abc\u200axyz", line.toString());
+  }
+
+  public void testNewLines() throws Exception {
+    LineReader in = makeStream("a\nbb\n\nccc\rdddd\r\neeeee");
+    Text out = new Text();
+    in.readLine(out);
+    assertEquals("line1 length", 1, out.getLength());
+    in.readLine(out);
+    assertEquals("line2 length", 2, out.getLength());
+    in.readLine(out);
+    assertEquals("line3 length", 0, out.getLength());
+    in.readLine(out);
+    assertEquals("line4 length", 3, out.getLength());
+    in.readLine(out);
+    assertEquals("line5 length", 4, out.getLength());
+    in.readLine(out);
+    assertEquals("line5 length", 5, out.getLength());
+    assertEquals("end of file", 0, in.readLine(out));
+  }
+  
+  private static void writeFile(FileSystem fs, Path name, 
+                                CompressionCodec codec,
+                                String contents) throws IOException {
+    OutputStream stm;
+    if (codec == null) {
+      stm = fs.create(name);
+    } else {
+      stm = codec.createOutputStream(fs.create(name));
+    }
+    stm.write(contents.getBytes());
+    stm.close();
+  }
+  
+  private static List<Text> readSplit(KeyValueTextInputFormat format, 
+                                      InputSplit split, 
+                                      TaskAttemptContext context) throws IOException, InterruptedException {
+    List<Text> result = new ArrayList<Text>();
+    RecordReader<Text, Text> reader = format.createRecordReader(split, context);
+    reader.initialize(split, context);
+    while (reader.nextKeyValue()) {
+      result.add(new Text(reader.getCurrentValue()));
+    }
+    return result;
+  }
+  
+  /**
+   * Test using the gzip codec for reading
+   */
+  public static void testGzip() throws Exception {
+    Job job = new Job();
+    CompressionCodec gzip = new GzipCodec();
+    ReflectionUtils.setConf(gzip, job.getConfiguration());
+    localFs.delete(workDir, true);
+    writeFile(localFs, new Path(workDir, "part1.txt.gz"), gzip, 
+              "line-1\tthe quick\nline-2\tbrown\nline-3\tfox jumped\nline-4\tover\nline-5\t the lazy\nline-6\t dog\n");
+    writeFile(localFs, new Path(workDir, "part2.txt.gz"), gzip,
+              "line-1\tthis is a test\nline-1\tof gzip\n");
+    FileInputFormat.setInputPaths(job, workDir);
+    
+    KeyValueTextInputFormat format = new KeyValueTextInputFormat();
+    JobContext jobContext = new JobContext(job.getConfiguration(), new JobID());
+    TaskAttemptContext context = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
+    List<InputSplit> splits = format.getSplits(jobContext);
+    assertEquals("compressed splits == 2", 2, splits.size());
+    FileSplit tmp = (FileSplit) splits.get(0);
+    if (tmp.getPath().getName().equals("part2.txt.gz")) {
+      splits.set(0, splits.get(1));
+      splits.set(1, tmp);
+    }
+    List<Text> results = readSplit(format, splits.get(0), context);
+    assertEquals("splits[0] length", 6, results.size());
+    assertEquals("splits[0][5]", " dog", results.get(5).toString());
+    results = readSplit(format, splits.get(1), context);
+    assertEquals("splits[1] length", 2, results.size());
+    assertEquals("splits[1][0]", "this is a test", 
+                 results.get(0).toString());    
+    assertEquals("splits[1][1]", "of gzip", 
+                 results.get(1).toString());    
+  }
+  
+  public static void main(String[] args) throws Exception {
+    new TestKeyValueTextInputFormat().testFormat();
+  }
+}

+ 115 - 0
src/test/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java

@@ -0,0 +1,115 @@
+/**
+ * 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.Random;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import junit.framework.TestCase;
+
+public class TestMRSequenceFileAsBinaryInputFormat extends TestCase {
+  private static final int RECORDS = 10000;
+
+  public void testBinary() throws IOException, InterruptedException {
+    Job job = new Job();
+    FileSystem fs = FileSystem.getLocal(job.getConfiguration());
+    Path dir = new Path(System.getProperty("test.build.data",".") + "/mapred");
+    Path file = new Path(dir, "testbinary.seq");
+    Random r = new Random();
+    long seed = r.nextLong();
+    r.setSeed(seed);
+
+    fs.delete(dir, true);
+    FileInputFormat.setInputPaths(job, dir);
+
+    Text tkey = new Text();
+    Text tval = new Text();
+
+    SequenceFile.Writer writer = new SequenceFile.Writer(fs,
+      job.getConfiguration(), file, Text.class, Text.class);
+    try {
+      for (int i = 0; i < RECORDS; ++i) {
+        tkey.set(Integer.toString(r.nextInt(), 36));
+        tval.set(Long.toString(r.nextLong(), 36));
+        writer.append(tkey, tval);
+      }
+    } finally {
+      writer.close();
+    }
+    TaskAttemptContext context = MapReduceTestUtil.
+      createDummyMapTaskAttemptContext(job.getConfiguration());
+    InputFormat<BytesWritable,BytesWritable> bformat =
+      new SequenceFileAsBinaryInputFormat();
+
+    int count = 0;
+    r.setSeed(seed);
+    BytesWritable bkey = new BytesWritable();
+    BytesWritable bval = new BytesWritable();
+    Text cmpkey = new Text();
+    Text cmpval = new Text();
+    DataInputBuffer buf = new DataInputBuffer();
+    FileInputFormat.setInputPaths(job, file);
+    for (InputSplit split : bformat.getSplits(job)) {
+      RecordReader<BytesWritable, BytesWritable> reader =
+            bformat.createRecordReader(split, context);
+      MapContext<BytesWritable, BytesWritable, BytesWritable, BytesWritable> 
+        mcontext = new MapContext<BytesWritable, BytesWritable,
+          BytesWritable, BytesWritable>(job.getConfiguration(), 
+          context.getTaskAttemptID(), reader, null, null, 
+          MapReduceTestUtil.createDummyReporter(), 
+          split);
+      reader.initialize(split, mcontext);
+      try {
+        while (reader.nextKeyValue()) {
+          bkey = reader.getCurrentKey();
+          bval = reader.getCurrentValue();
+          tkey.set(Integer.toString(r.nextInt(), 36));
+          tval.set(Long.toString(r.nextLong(), 36));
+          buf.reset(bkey.getBytes(), bkey.getLength());
+          cmpkey.readFields(buf);
+          buf.reset(bval.getBytes(), bval.getLength());
+          cmpval.readFields(buf);
+          assertTrue(
+            "Keys don't match: " + "*" + cmpkey.toString() + ":" +
+            tkey.toString() + "*",
+            cmpkey.toString().equals(tkey.toString()));
+          assertTrue(
+            "Vals don't match: " + "*" + cmpval.toString() + ":" +
+            tval.toString() + "*",
+            cmpval.toString().equals(tval.toString()));
+          ++count;
+        }
+      } finally {
+        reader.close();
+      }
+    }
+    assertEquals("Some records not found", RECORDS, count);
+  }
+
+}

+ 118 - 0
src/test/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java

@@ -0,0 +1,118 @@
+/**
+ * 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.util.*;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.conf.*;
+
+public class TestMRSequenceFileAsTextInputFormat extends TestCase {
+  private static int MAX_LENGTH = 10000;
+  private static Configuration conf = new Configuration();
+
+  public void testFormat() throws Exception {
+    Job job = new Job(conf);
+    FileSystem fs = FileSystem.getLocal(conf);
+    Path dir = new Path(System.getProperty("test.build.data",".") + "/mapred");
+    Path file = new Path(dir, "test.seq");
+    
+    int seed = new Random().nextInt();
+    Random random = new Random(seed);
+
+    fs.delete(dir, true);
+
+    FileInputFormat.setInputPaths(job, dir);
+
+    // for a variety of lengths
+    for (int length = 0; length < MAX_LENGTH;
+         length += random.nextInt(MAX_LENGTH / 10) + 1) {
+
+      // create a file with length entries
+      SequenceFile.Writer writer =
+        SequenceFile.createWriter(fs, conf, file,
+          IntWritable.class, LongWritable.class);
+      try {
+        for (int i = 0; i < length; i++) {
+          IntWritable key = new IntWritable(i);
+          LongWritable value = new LongWritable(10 * i);
+          writer.append(key, value);
+        }
+      } finally {
+        writer.close();
+      }
+
+      TaskAttemptContext context = MapReduceTestUtil.
+        createDummyMapTaskAttemptContext(job.getConfiguration());
+      // try splitting the file in a variety of sizes
+      InputFormat<Text, Text> format =
+        new SequenceFileAsTextInputFormat();
+      
+      for (int i = 0; i < 3; i++) {
+        // check each split
+        BitSet bits = new BitSet(length);
+        int numSplits =
+          random.nextInt(MAX_LENGTH / (SequenceFile.SYNC_INTERVAL / 20)) + 1;
+        FileInputFormat.setMaxInputSplitSize(job, 
+          fs.getFileStatus(file).getLen() / numSplits);
+        for (InputSplit split : format.getSplits(job)) {
+          RecordReader<Text, Text> reader =
+            format.createRecordReader(split, context);
+          MapContext<Text, Text, Text, Text> mcontext = 
+            new MapContext<Text, Text, Text, Text>(job.getConfiguration(), 
+            context.getTaskAttemptID(), reader, null, null, 
+            MapReduceTestUtil.createDummyReporter(), 
+            split);
+          reader.initialize(split, mcontext);
+          Class<?> readerClass = reader.getClass();
+          assertEquals("reader class is SequenceFileAsTextRecordReader.",
+            SequenceFileAsTextRecordReader.class, readerClass);        
+          Text key;
+          try {
+            int count = 0;
+            while (reader.nextKeyValue()) {
+              key = reader.getCurrentKey();
+              int keyInt = Integer.parseInt(key.toString());
+              assertFalse("Key in multiple partitions.", bits.get(keyInt));
+              bits.set(keyInt);
+              count++;
+            }
+          } finally {
+            reader.close();
+          }
+        }
+        assertEquals("Some keys in no partition.", length, bits.cardinality());
+      }
+
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    new TestMRSequenceFileAsTextInputFormat().testFormat();
+  }
+}

+ 194 - 0
src/test/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java

@@ -0,0 +1,194 @@
+/**
+ * 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.*;
+import java.util.*;
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.*;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.conf.*;
+
+public class TestMRSequenceFileInputFilter extends TestCase {
+  private static final Log LOG = 
+    LogFactory.getLog(TestMRSequenceFileInputFilter.class.getName());
+
+  private static final int MAX_LENGTH = 15000;
+  private static final Configuration conf = new Configuration();
+  private static final Job job;
+  private static final FileSystem fs;
+  private static final Path inDir = 
+    new Path(System.getProperty("test.build.data",".") + "/mapred");
+  private static final Path inFile = new Path(inDir, "test.seq");
+  private static final Random random = new Random(1);
+  
+  static {
+    try {
+      job = new Job(conf);
+      FileInputFormat.setInputPaths(job, inDir);
+      fs = FileSystem.getLocal(conf);
+    } catch (IOException e) {
+      e.printStackTrace();
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void createSequenceFile(int numRecords) throws Exception {
+    // create a file with length entries
+    SequenceFile.Writer writer =
+      SequenceFile.createWriter(fs, conf, inFile,
+                                Text.class, BytesWritable.class);
+    try {
+      for (int i = 1; i <= numRecords; i++) {
+        Text key = new Text(Integer.toString(i));
+        byte[] data = new byte[random.nextInt(10)];
+        random.nextBytes(data);
+        BytesWritable value = new BytesWritable(data);
+        writer.append(key, value);
+      }
+    } finally {
+      writer.close();
+    }
+  }
+
+
+  private int countRecords(int numSplits) 
+      throws IOException, InterruptedException {
+    InputFormat<Text, BytesWritable> format =
+      new SequenceFileInputFilter<Text, BytesWritable>();
+    if (numSplits == 0) {
+      numSplits =
+        random.nextInt(MAX_LENGTH / (SequenceFile.SYNC_INTERVAL / 20)) + 1;
+    }
+    FileInputFormat.setMaxInputSplitSize(job, 
+      fs.getFileStatus(inFile).getLen() / numSplits);
+    TaskAttemptContext context = MapReduceTestUtil.
+      createDummyMapTaskAttemptContext(job.getConfiguration());
+    // check each split
+    int count = 0;
+    for (InputSplit split : format.getSplits(job)) {
+      RecordReader<Text, BytesWritable> reader =
+        format.createRecordReader(split, context);
+      MapContext<Text, BytesWritable, Text, BytesWritable> mcontext = 
+        new MapContext<Text, BytesWritable, Text, BytesWritable>(
+        job.getConfiguration(), 
+        context.getTaskAttemptID(), reader, null, null, 
+        MapReduceTestUtil.createDummyReporter(), split);
+      reader.initialize(split, mcontext);
+      try {
+        while (reader.nextKeyValue()) {
+          LOG.info("Accept record " + reader.getCurrentKey().toString());
+          count++;
+        }
+      } finally {
+        reader.close();
+      }
+    }
+    return count;
+  }
+  
+  public void testRegexFilter() throws Exception {
+    // set the filter class
+    LOG.info("Testing Regex Filter with patter: \\A10*");
+    SequenceFileInputFilter.setFilterClass(job, 
+      SequenceFileInputFilter.RegexFilter.class);
+    SequenceFileInputFilter.RegexFilter.setPattern(
+      job.getConfiguration(), "\\A10*");
+    
+    // clean input dir
+    fs.delete(inDir, true);
+  
+    // for a variety of lengths
+    for (int length = 1; length < MAX_LENGTH;
+         length += random.nextInt(MAX_LENGTH / 10) + 1) {
+      LOG.info("******Number of records: " + length);
+      createSequenceFile(length);
+      int count = countRecords(0);
+      assertEquals(count, length==0 ? 0 : (int)Math.log10(length) + 1);
+    }
+    
+    // clean up
+    fs.delete(inDir, true);
+  }
+
+  public void testPercentFilter() throws Exception {
+    LOG.info("Testing Percent Filter with frequency: 1000");
+    // set the filter class
+    SequenceFileInputFilter.setFilterClass(job, 
+      SequenceFileInputFilter.PercentFilter.class);
+    SequenceFileInputFilter.PercentFilter.setFrequency(
+      job.getConfiguration(), 1000);
+      
+    // clean input dir
+    fs.delete(inDir, true);
+    
+    // for a variety of lengths
+    for (int length = 0; length < MAX_LENGTH;
+         length += random.nextInt(MAX_LENGTH / 10) + 1) {
+      LOG.info("******Number of records: "+length);
+      createSequenceFile(length);
+      int count = countRecords(1);
+      LOG.info("Accepted " + count + " records");
+      int expectedCount = length / 1000;
+      if (expectedCount * 1000 != length)
+        expectedCount++;
+      assertEquals(count, expectedCount);
+    }
+      
+    // clean up
+    fs.delete(inDir, true);
+  }
+  
+  public void testMD5Filter() throws Exception {
+    // set the filter class
+    LOG.info("Testing MD5 Filter with frequency: 1000");
+    SequenceFileInputFilter.setFilterClass(job, 
+      SequenceFileInputFilter.MD5Filter.class);
+    SequenceFileInputFilter.MD5Filter.setFrequency(
+      job.getConfiguration(), 1000);
+      
+    // clean input dir
+    fs.delete(inDir, true);
+    
+    // for a variety of lengths
+    for (int length = 0; length < MAX_LENGTH;
+         length += random.nextInt(MAX_LENGTH / 10) + 1) {
+      LOG.info("******Number of records: " + length);
+      createSequenceFile(length);
+      LOG.info("Accepted " + countRecords(0) + " records");
+    }
+    // clean up
+    fs.delete(inDir, true);
+  }
+
+  public static void main(String[] args) throws Exception {
+    TestMRSequenceFileInputFilter filter = new TestMRSequenceFileInputFilter();
+    filter.testRegexFilter();
+  }
+}

+ 72 - 0
src/test/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java

@@ -0,0 +1,72 @@
+/**
+ * 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.Map;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.Mapper;
+
+/**
+ * @see TestDelegatingInputFormat
+ */
+public class TestMultipleInputs extends TestCase {
+  
+  public void testAddInputPathWithFormat() throws IOException {
+    final Job job = new Job();
+    MultipleInputs.addInputPath(job, new Path("/foo"), TextInputFormat.class);
+    MultipleInputs.addInputPath(job, new Path("/bar"),
+        KeyValueTextInputFormat.class);
+    final Map<Path, InputFormat> inputs = MultipleInputs
+       .getInputFormatMap(new JobContext(job.getConfiguration(), new JobID()));
+    assertEquals(TextInputFormat.class, inputs.get(new Path("/foo")).getClass());
+    assertEquals(KeyValueTextInputFormat.class, inputs.get(new Path("/bar"))
+       .getClass());
+  }
+
+  public void testAddInputPathWithMapper() throws IOException {
+    final Job job = new Job();
+    MultipleInputs.addInputPath(job, new Path("/foo"), TextInputFormat.class,
+       MapClass.class);
+    MultipleInputs.addInputPath(job, new Path("/bar"),
+       KeyValueTextInputFormat.class, MapClass2.class);
+    final Map<Path, InputFormat> inputs = MultipleInputs
+       .getInputFormatMap(job);
+    final Map<Path, Class<? extends Mapper>> maps = MultipleInputs
+       .getMapperTypeMap(job);
+
+    assertEquals(TextInputFormat.class, inputs.get(new Path("/foo")).getClass());
+    assertEquals(KeyValueTextInputFormat.class, inputs.get(new Path("/bar"))
+       .getClass());
+    assertEquals(MapClass.class, maps.get(new Path("/foo")));
+    assertEquals(MapClass2.class, maps.get(new Path("/bar")));
+  }
+
+  static class MapClass extends Mapper<String, String, String, String> {
+  }
+
+  static class MapClass2 extends MapClass {
+  }
+}

+ 114 - 0
src/test/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java

@@ -0,0 +1,114 @@
+/**
+ * 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.*;
+import java.util.*;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.*;
+
+public class TestNLineInputFormat extends TestCase {
+  private static int MAX_LENGTH = 200;
+  
+  private static Configuration conf = new Configuration();
+  private static FileSystem localFs = null; 
+
+  static {
+    try {
+      localFs = FileSystem.getLocal(conf);
+    } catch (IOException e) {
+      throw new RuntimeException("init failure", e);
+    }
+  }
+
+  private static Path workDir = 
+    new Path(new Path(System.getProperty("test.build.data", "."), "data"),
+             "TestNLineInputFormat");
+  
+  public void testFormat() throws Exception {
+    Job job = new Job(conf);
+    Path file = new Path(workDir, "test.txt");
+
+    int seed = new Random().nextInt();
+    Random random = new Random(seed);
+
+    localFs.delete(workDir, true);
+    FileInputFormat.setInputPaths(job, workDir);
+    int numLinesPerMap = 5;
+    NLineInputFormat.setNumLinesPerSplit(job, numLinesPerMap);
+    // for a variety of lengths
+    for (int length = 0; length < MAX_LENGTH;
+         length += random.nextInt(MAX_LENGTH / 10) + 1) {
+      // create a file with length entries
+      Writer writer = new OutputStreamWriter(localFs.create(file));
+      try {
+        for (int i = 0; i < length; i++) {
+          writer.write(Integer.toString(i));
+          writer.write("\n");
+        }
+      } finally {
+        writer.close();
+      }
+      checkFormat(job, numLinesPerMap);
+    }
+  }
+
+  void checkFormat(Job job, int expectedN) 
+      throws IOException, InterruptedException {
+    NLineInputFormat format = new NLineInputFormat();
+    List<InputSplit> splits = format.getSplits(job);
+    // check all splits except last one
+    int count = 0;
+    for (int i = 0; i < splits.size() -1; i++) {
+      assertEquals("There are no split locations", 0,
+                   splits.get(i).getLocations().length);
+      TaskAttemptContext context = MapReduceTestUtil.
+        createDummyMapTaskAttemptContext(job.getConfiguration());
+      RecordReader<LongWritable, Text> reader = format.createRecordReader(
+        splits.get(i), context);
+      Class<?> clazz = reader.getClass();
+      assertEquals("reader class is LineRecordReader.", 
+        LineRecordReader.class, clazz);
+      MapContext<LongWritable, Text, LongWritable, Text> mcontext = 
+        new MapContext<LongWritable, Text, LongWritable, Text>(
+          job.getConfiguration(), context.getTaskAttemptID(), reader, null,
+          null, MapReduceTestUtil.createDummyReporter(), splits.get(i));
+      reader.initialize(splits.get(i), mcontext);
+         
+      try {
+        count = 0;
+        while (reader.nextKeyValue()) {
+          count++;
+        }
+      } finally {
+        reader.close();
+      }
+      assertEquals("number of lines in split is " + expectedN ,
+                   expectedN, count);
+    }
+  }
+  
+  public static void main(String[] args) throws Exception {
+    new TestNLineInputFormat().testFormat();
+  }
+}

+ 46 - 0
src/test/org/apache/hadoop/mapreduce/lib/jobcontrol/TestControlledJob.java

@@ -0,0 +1,46 @@
+/**
+ * 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.jobcontrol;
+
+import static org.junit.Assert.assertFalse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+/**
+ */
+public class TestControlledJob {
+  
+  @Test
+  public void testAddingDependingJobToRunningJobFails() throws Exception {
+    Configuration conf = new Configuration();
+    ControlledJob job1 = new ControlledJob(conf);
+    job1.setJobState(ControlledJob.State.RUNNING);
+    assertFalse(job1.addDependingJob(new ControlledJob(conf)));
+  }
+
+  @Test
+  public void testAddingDependingJobToCompletedJobFails() throws Exception {
+    Configuration conf = new Configuration();
+    ControlledJob job1 = new ControlledJob(conf);
+    job1.setJobState(ControlledJob.State.SUCCESS);
+    assertFalse(job1.addDependingJob(new ControlledJob(conf)));
+  }
+
+}

+ 142 - 0
src/test/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java

@@ -0,0 +1,142 @@
+/**
+ * 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.jobcontrol;
+
+import static junit.framework.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.junit.Test;
+
+/**
+ * Tests the JobControl API using mock and stub Job instances.
+ */
+public class TestMapReduceJobControlWithMocks {
+
+  @Test
+  public void testSuccessfulJobs() throws Exception {
+    JobControl jobControl = new JobControl("Test");
+    
+    ControlledJob job1 = createSuccessfulControlledJob(jobControl);
+    ControlledJob job2 = createSuccessfulControlledJob(jobControl);
+    ControlledJob job3 = createSuccessfulControlledJob(jobControl, job1, job2);
+    ControlledJob job4 = createSuccessfulControlledJob(jobControl, job3);
+    
+    runJobControl(jobControl);
+    
+    assertEquals("Success list", 4, jobControl.getSuccessfulJobList().size());
+    assertEquals("Failed list", 0, jobControl.getFailedJobList().size());
+    
+    assertTrue(job1.getJobState() == ControlledJob.State.SUCCESS);
+    assertTrue(job2.getJobState() == ControlledJob.State.SUCCESS);
+    assertTrue(job3.getJobState() == ControlledJob.State.SUCCESS);
+    assertTrue(job4.getJobState() == ControlledJob.State.SUCCESS);
+    
+    jobControl.stop();
+  }
+  
+  @Test
+  public void testFailedJob() throws Exception {
+    JobControl jobControl = new JobControl("Test");
+    
+    ControlledJob job1 = createFailedControlledJob(jobControl);
+    ControlledJob job2 = createSuccessfulControlledJob(jobControl);
+    ControlledJob job3 = createSuccessfulControlledJob(jobControl, job1, job2);
+    ControlledJob job4 = createSuccessfulControlledJob(jobControl, job3);
+    
+    runJobControl(jobControl);
+    
+    assertEquals("Success list", 1, jobControl.getSuccessfulJobList().size());
+    assertEquals("Failed list", 3, jobControl.getFailedJobList().size());
+
+    assertTrue(job1.getJobState() == ControlledJob.State.FAILED);
+    assertTrue(job2.getJobState() == ControlledJob.State.SUCCESS);
+    assertTrue(job3.getJobState() == ControlledJob.State.DEPENDENT_FAILED);
+    assertTrue(job4.getJobState() == ControlledJob.State.DEPENDENT_FAILED);
+    
+    jobControl.stop();
+  }
+  
+  @Test
+  public void testKillJob() throws Exception {
+    JobControl jobControl = new JobControl("Test");
+    
+    ControlledJob job = createFailedControlledJob(jobControl);
+    
+    job.killJob();
+
+    // Verify that killJob() was called on the mock Job
+    verify(job.getJob()).killJob();
+  }
+  
+  private Job createJob(boolean complete, boolean successful)
+  	throws IOException, InterruptedException {
+    // Create a stub Job that responds in a controlled way
+    Job mockJob = mock(Job.class);
+    when(mockJob.getConfiguration()).thenReturn(new Configuration());
+    when(mockJob.isComplete()).thenReturn(complete);
+    when(mockJob.isSuccessful()).thenReturn(successful);
+    return mockJob;
+  }
+  
+  private ControlledJob createControlledJob(JobControl jobControl,
+      	boolean successful, ControlledJob... dependingJobs)
+      	throws IOException, InterruptedException {
+    List<ControlledJob> dependingJobsList = dependingJobs == null ? null :
+      Arrays.asList(dependingJobs);
+    ControlledJob job = new ControlledJob(createJob(true, successful),
+	dependingJobsList);
+    jobControl.addJob(job);
+    return job;
+  }
+  
+  private ControlledJob createSuccessfulControlledJob(JobControl jobControl,
+      ControlledJob... dependingJobs) throws IOException, InterruptedException {
+    return createControlledJob(jobControl, true, dependingJobs);
+  }
+
+  private ControlledJob createFailedControlledJob(JobControl jobControl,
+      ControlledJob... dependingJobs) throws IOException, InterruptedException {
+    return createControlledJob(jobControl, false, dependingJobs);
+  }
+
+  private void runJobControl(JobControl jobControl) {
+    Thread controller = new Thread(jobControl);
+    controller.start();
+    waitTillAllFinished(jobControl);
+  }
+
+  private void waitTillAllFinished(JobControl jobControl) {
+    while (!jobControl.allFinished()) {
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException e) {
+	// ignore
+      }
+    }
+  }
+}

+ 212 - 0
src/test/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.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.lib.output;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.mapred.InvalidJobConfException;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+
+import junit.framework.TestCase;
+import org.apache.commons.logging.*;
+
+public class TestMRSequenceFileAsBinaryOutputFormat extends TestCase {
+  private static final Log LOG =
+    LogFactory.getLog(TestMRSequenceFileAsBinaryOutputFormat.class.getName());
+
+  private static final int RECORDS = 10000;
+  
+  public void testBinary() throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    Job job = new Job(conf);
+    
+    Path outdir = new Path(System.getProperty("test.build.data", "/tmp"),
+                    "outseq");
+    Random r = new Random();
+    long seed = r.nextLong();
+    r.setSeed(seed);
+
+    FileOutputFormat.setOutputPath(job, outdir);
+
+    SequenceFileAsBinaryOutputFormat.setSequenceFileOutputKeyClass(job, 
+                                          IntWritable.class );
+    SequenceFileAsBinaryOutputFormat.setSequenceFileOutputValueClass(job, 
+                                          DoubleWritable.class ); 
+
+    SequenceFileAsBinaryOutputFormat.setCompressOutput(job, true);
+    SequenceFileAsBinaryOutputFormat.setOutputCompressionType(job, 
+                                                       CompressionType.BLOCK);
+
+    BytesWritable bkey = new BytesWritable();
+    BytesWritable bval = new BytesWritable();
+
+    TaskAttemptContext context = 
+      MapReduceTestUtil.createDummyMapTaskAttemptContext(job.getConfiguration());
+    OutputFormat<BytesWritable, BytesWritable> outputFormat = 
+      new SequenceFileAsBinaryOutputFormat();
+    OutputCommitter committer = outputFormat.getOutputCommitter(context);
+    committer.setupJob(job);
+    RecordWriter<BytesWritable, BytesWritable> writer = outputFormat.
+      getRecordWriter(context);
+
+    IntWritable iwritable = new IntWritable();
+    DoubleWritable dwritable = new DoubleWritable();
+    DataOutputBuffer outbuf = new DataOutputBuffer();
+    LOG.info("Creating data by SequenceFileAsBinaryOutputFormat");
+    try {
+      for (int i = 0; i < RECORDS; ++i) {
+        iwritable = new IntWritable(r.nextInt());
+        iwritable.write(outbuf);
+        bkey.set(outbuf.getData(), 0, outbuf.getLength());
+        outbuf.reset();
+        dwritable = new DoubleWritable(r.nextDouble());
+        dwritable.write(outbuf);
+        bval.set(outbuf.getData(), 0, outbuf.getLength());
+        outbuf.reset();
+        writer.write(bkey, bval);
+      }
+    } finally {
+      writer.close(context);
+    }
+    committer.commitTask(context);
+    committer.commitJob(job);
+
+    InputFormat<IntWritable, DoubleWritable> iformat =
+      new SequenceFileInputFormat<IntWritable, DoubleWritable>();
+    int count = 0;
+    r.setSeed(seed);
+    SequenceFileInputFormat.setInputPaths(job, outdir);
+    LOG.info("Reading data by SequenceFileInputFormat");
+    for (InputSplit split : iformat.getSplits(job)) {
+      RecordReader<IntWritable, DoubleWritable> reader =
+        iformat.createRecordReader(split, context);
+      MapContext<IntWritable, DoubleWritable, BytesWritable, BytesWritable> 
+        mcontext = new MapContext<IntWritable, DoubleWritable,
+          BytesWritable, BytesWritable>(job.getConfiguration(), 
+          context.getTaskAttemptID(), reader, null, null, 
+          MapReduceTestUtil.createDummyReporter(), 
+          split);
+      reader.initialize(split, mcontext);
+      try {
+        int sourceInt;
+        double sourceDouble;
+        while (reader.nextKeyValue()) {
+          sourceInt = r.nextInt();
+          sourceDouble = r.nextDouble();
+          iwritable = reader.getCurrentKey();
+          dwritable = reader.getCurrentValue();
+          assertEquals(
+              "Keys don't match: " + "*" + iwritable.get() + ":" + 
+                                           sourceInt + "*",
+              sourceInt, iwritable.get());
+          assertTrue(
+              "Vals don't match: " + "*" + dwritable.get() + ":" +
+                                           sourceDouble + "*",
+              Double.compare(dwritable.get(), sourceDouble) == 0 );
+          ++count;
+        }
+      } finally {
+        reader.close();
+      }
+    }
+    assertEquals("Some records not found", RECORDS, count);
+  }
+
+  public void testSequenceOutputClassDefaultsToMapRedOutputClass() 
+         throws IOException {
+    Job job = new Job();
+    // Setting Random class to test getSequenceFileOutput{Key,Value}Class
+    job.setOutputKeyClass(FloatWritable.class);
+    job.setOutputValueClass(BooleanWritable.class);
+
+    assertEquals("SequenceFileOutputKeyClass should default to ouputKeyClass", 
+      FloatWritable.class,
+      SequenceFileAsBinaryOutputFormat.getSequenceFileOutputKeyClass(job));
+    assertEquals("SequenceFileOutputValueClass should default to " 
+      + "ouputValueClass", 
+      BooleanWritable.class,
+      SequenceFileAsBinaryOutputFormat.getSequenceFileOutputValueClass(job));
+
+    SequenceFileAsBinaryOutputFormat.setSequenceFileOutputKeyClass(job, 
+      IntWritable.class );
+    SequenceFileAsBinaryOutputFormat.setSequenceFileOutputValueClass(job, 
+      DoubleWritable.class ); 
+
+    assertEquals("SequenceFileOutputKeyClass not updated", 
+      IntWritable.class,
+      SequenceFileAsBinaryOutputFormat.getSequenceFileOutputKeyClass(job));
+    assertEquals("SequenceFileOutputValueClass not updated", 
+      DoubleWritable.class,
+      SequenceFileAsBinaryOutputFormat.getSequenceFileOutputValueClass(job));
+  }
+
+  public void testcheckOutputSpecsForbidRecordCompression() 
+      throws IOException {
+    Job job = new Job();
+    FileSystem fs = FileSystem.getLocal(job.getConfiguration());
+    Path outputdir = new Path(System.getProperty("test.build.data", "/tmp") 
+                              + "/output");
+    fs.delete(outputdir, true);
+
+    // Without outputpath, FileOutputFormat.checkoutputspecs will throw 
+    // InvalidJobConfException
+    FileOutputFormat.setOutputPath(job, outputdir);
+
+    // SequenceFileAsBinaryOutputFormat doesn't support record compression
+    // It should throw an exception when checked by checkOutputSpecs
+    SequenceFileAsBinaryOutputFormat.setCompressOutput(job, true);
+
+    SequenceFileAsBinaryOutputFormat.setOutputCompressionType(job, 
+      CompressionType.BLOCK);
+    try {
+      new SequenceFileAsBinaryOutputFormat().checkOutputSpecs(job);
+    } catch (Exception e) {
+      fail("Block compression should be allowed for " 
+        + "SequenceFileAsBinaryOutputFormat:Caught " + e.getClass().getName());
+    }
+
+    SequenceFileAsBinaryOutputFormat.setOutputCompressionType(job, 
+      CompressionType.RECORD);
+    try {
+      new SequenceFileAsBinaryOutputFormat().checkOutputSpecs(job);
+      fail("Record compression should not be allowed for " 
+        + "SequenceFileAsBinaryOutputFormat");
+    } catch (InvalidJobConfException ie) {
+      // expected
+    } catch (Exception e) {
+      fail("Expected " + InvalidJobConfException.class.getName() 
+        + "but caught " + e.getClass().getName() );
+    }
+  }
+}

+ 232 - 0
src/test/org/apache/hadoop/mapreduce/lib/output/TestMultipleOutputs.java

@@ -0,0 +1,232 @@
+/**
+ * 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.fs.FileStatus;
+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.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.CounterGroup;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+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.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+public class TestMultipleOutputs extends HadoopTestCase {
+
+  public TestMultipleOutputs() throws IOException {
+    super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
+  }
+
+  public void testWithoutCounters() throws Exception {
+    _testMultipleOutputs(false);
+  }
+
+  public void testWithCounters() throws Exception {
+    _testMultipleOutputs(true);
+  }
+
+  private static final Path ROOT_DIR = new Path("testing/mo");
+  private static final Path IN_DIR = new Path(ROOT_DIR, "input");
+  private static final Path OUT_DIR = new Path(ROOT_DIR, "output");
+
+  private Path getDir(Path dir) {
+    // Hack for local FS that does not have the concept of a 'mounting point'
+    if (isLocalFS()) {
+      String localPathRoot = System.getProperty("test.build.data", "/tmp")
+        .replace(' ', '+');
+      dir = new Path(localPathRoot, dir);
+    }
+    return dir;
+  }
+
+  public void setUp() throws Exception {
+    super.setUp();
+    Path rootDir = getDir(ROOT_DIR);
+    Path inDir = getDir(IN_DIR);
+
+    JobConf conf = createJobConf();
+    FileSystem fs = FileSystem.get(conf);
+    fs.delete(rootDir, true);
+    if (!fs.mkdirs(inDir)) {
+      throw new IOException("Mkdirs failed to create " + inDir.toString());
+    }
+  }
+
+  public void tearDown() throws Exception {
+    Path rootDir = getDir(ROOT_DIR);
+
+    JobConf conf = createJobConf();
+    FileSystem fs = FileSystem.get(conf);
+    fs.delete(rootDir, true);
+    super.tearDown();
+  }
+
+  protected void _testMultipleOutputs(boolean withCounters) throws Exception {
+    Path inDir = getDir(IN_DIR);
+    Path outDir = getDir(OUT_DIR);
+
+    JobConf conf = createJobConf();
+    FileSystem fs = FileSystem.get(conf);
+
+    DataOutputStream file = fs.create(new Path(inDir, "part-0"));
+    file.writeBytes("a\nb\n\nc\nd\ne");
+    file.close();
+
+    file = fs.create(new Path(inDir, "part-1"));
+    file.writeBytes("a\nb\n\nc\nd\ne");
+    file.close();
+
+    Job job = new Job(conf);
+    job.setJobName("mo");
+    job.setInputFormatClass(TextInputFormat.class);
+
+    job.setOutputKeyClass(LongWritable.class);
+    job.setOutputValueClass(Text.class);
+
+    job.setMapOutputKeyClass(LongWritable.class);
+    job.setMapOutputValueClass(Text.class);
+
+    job.setOutputFormatClass(TextOutputFormat.class);
+    job.setOutputKeyClass(LongWritable.class);
+    job.setOutputValueClass(Text.class);
+
+    MultipleOutputs.addNamedOutput(job, "text", TextOutputFormat.class,
+      LongWritable.class, Text.class);
+
+    MultipleOutputs.setCountersEnabled(job, withCounters);
+
+    job.setMapperClass(MOMap.class);
+    job.setReducerClass(MOReduce.class);
+
+    FileInputFormat.setInputPaths(job, inDir);
+    FileOutputFormat.setOutputPath(job, outDir);
+
+    job.waitForCompletion(false);
+
+    // assert number of named output part files
+    int namedOutputCount = 0;
+    FileStatus[] statuses = fs.listStatus(outDir);
+    
+    for (FileStatus status : statuses) {
+      if (status.getPath().getName().equals("text-m-00000") ||
+        status.getPath().getName().equals("text-m-00001") ||
+        status.getPath().getName().equals("text-r-00000")) {
+        namedOutputCount++;
+      }
+    }
+    assertEquals(3, namedOutputCount);
+
+    // assert TextOutputFormat files correctness
+    JobContext jobContext = new JobContext(job.getConfiguration(), new JobID());
+    BufferedReader reader = new BufferedReader(
+      new InputStreamReader(fs.open(
+        new Path(FileOutputFormat.getOutputPath(jobContext), "text-r-00000"))));
+    int count = 0;
+    String line = reader.readLine();
+    while (line != null) {
+      assertTrue(line.endsWith("text"));
+      line = reader.readLine();
+      count++;
+    }
+    reader.close();
+    assertFalse(count == 0);
+
+    CounterGroup counters =
+      job.getCounters().getGroup(MultipleOutputs.class.getName());
+    if (!withCounters) {
+      assertEquals(0, counters.size());
+    }
+    else {
+      assertEquals(1, counters.size());
+      assertEquals(4, counters.findCounter("text").getValue());
+    }
+
+  }
+
+  @SuppressWarnings({"unchecked"})
+  public static class MOMap extends Mapper<LongWritable, Text, LongWritable,
+    Text> {
+
+    private MultipleOutputs mos;
+
+    @Override
+    protected void setup(Context context) {
+      mos = new MultipleOutputs(context);
+    }
+    
+    @Override
+    public void map(LongWritable key, Text value, Context context)
+      throws IOException, InterruptedException {
+      if (!value.toString().equals("a")) {
+        context.write(key, value);
+      } else {
+        mos.write("text", key, new Text("text"));
+      }
+    }
+    
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+      mos.close();
+    }
+  }
+
+  @SuppressWarnings({"unchecked"})
+  public static class MOReduce extends Reducer<LongWritable, Text,
+    LongWritable, Text> {
+
+    private MultipleOutputs mos;
+
+    @Override
+    protected void setup(Context context) {
+      mos = new MultipleOutputs(context);
+    }
+
+    @Override
+    public void reduce(LongWritable key, Iterable<Text> values, Context context)
+      throws IOException, InterruptedException {
+      for (Text value : values) {
+        if (!value.toString().equals("b")) {
+          context.write(key, value);
+        } else {
+          mos.write("text", key, new Text("text"));
+        }
+      }
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+      mos.close();
+    }
+  }
+
+}

+ 103 - 0
src/test/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.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.lib.partition;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import junit.framework.TestCase;
+
+public class TestBinaryPartitioner extends TestCase {
+
+  public void testDefaultOffsets() {
+    Configuration conf = new Configuration();
+    BinaryPartitioner<?> partitioner = 
+      ReflectionUtils.newInstance(BinaryPartitioner.class, conf);
+    
+    BinaryComparable key1 = new BytesWritable(new byte[] { 1, 2, 3, 4, 5 }); 
+    BinaryComparable key2 = new BytesWritable(new byte[] { 1, 2, 3, 4, 5 });
+    int partition1 = partitioner.getPartition(key1, null, 10);
+    int partition2 = partitioner.getPartition(key2, null, 10);
+    assertEquals(partition1, partition2);
+    
+    key1 = new BytesWritable(new byte[] { 1, 2, 3, 4, 5 }); 
+    key2 = new BytesWritable(new byte[] { 6, 2, 3, 4, 5 });
+    partition1 = partitioner.getPartition(key1, null, 10);
+    partition2 = partitioner.getPartition(key2, null, 10);
+    assertTrue(partition1 != partition2);
+    
+    key1 = new BytesWritable(new byte[] { 1, 2, 3, 4, 5 }); 
+    key2 = new BytesWritable(new byte[] { 1, 2, 3, 4, 6 });
+    partition1 = partitioner.getPartition(key1, null, 10);
+    partition2 = partitioner.getPartition(key2, null, 10);
+    assertTrue(partition1 != partition2);
+  }
+  
+  public void testCustomOffsets() {
+    Configuration conf = new Configuration();
+    BinaryComparable key1 = new BytesWritable(new byte[] { 1, 2, 3, 4, 5 }); 
+    BinaryComparable key2 = new BytesWritable(new byte[] { 6, 2, 3, 7, 8 });
+    
+    BinaryPartitioner.setOffsets(conf, 1, -3);
+    BinaryPartitioner<?> partitioner = 
+      ReflectionUtils.newInstance(BinaryPartitioner.class, conf);
+    int partition1 = partitioner.getPartition(key1, null, 10);
+    int partition2 = partitioner.getPartition(key2, null, 10);
+    assertEquals(partition1, partition2);
+    
+    BinaryPartitioner.setOffsets(conf, 1, 2);
+    partitioner = ReflectionUtils.newInstance(BinaryPartitioner.class, conf);
+    partition1 = partitioner.getPartition(key1, null, 10);
+    partition2 = partitioner.getPartition(key2, null, 10);
+    assertEquals(partition1, partition2);
+    
+    BinaryPartitioner.setOffsets(conf, -4, -3);
+    partitioner = ReflectionUtils.newInstance(BinaryPartitioner.class, conf);
+    partition1 = partitioner.getPartition(key1, null, 10);
+    partition2 = partitioner.getPartition(key2, null, 10);
+    assertEquals(partition1, partition2);
+  }
+  
+  public void testLowerBound() {
+    Configuration conf = new Configuration();
+    BinaryPartitioner.setLeftOffset(conf, 0);
+    BinaryPartitioner<?> partitioner = 
+      ReflectionUtils.newInstance(BinaryPartitioner.class, conf);
+    BinaryComparable key1 = new BytesWritable(new byte[] { 1, 2, 3, 4, 5 }); 
+    BinaryComparable key2 = new BytesWritable(new byte[] { 6, 2, 3, 4, 5 });
+    int partition1 = partitioner.getPartition(key1, null, 10);
+    int partition2 = partitioner.getPartition(key2, null, 10);
+    assertTrue(partition1 != partition2);
+  }
+  
+  public void testUpperBound() {
+    Configuration conf = new Configuration();
+    BinaryPartitioner.setRightOffset(conf, 4);
+    BinaryPartitioner<?> partitioner = 
+      ReflectionUtils.newInstance(BinaryPartitioner.class, conf);
+    BinaryComparable key1 = new BytesWritable(new byte[] { 1, 2, 3, 4, 5 }); 
+    BinaryComparable key2 = new BytesWritable(new byte[] { 1, 2, 3, 4, 6 });
+    int partition1 = partitioner.getPartition(key1, null, 10);
+    int partition2 = partitioner.getPartition(key2, null, 10);
+    assertTrue(partition1 != partition2);
+  }
+  
+}

+ 149 - 0
src/test/org/apache/hadoop/mapreduce/lib/partition/TestInputSampler.java

@@ -0,0 +1,149 @@
+/**
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TestInputSampler {
+
+  static class SequentialSplit extends InputSplit {
+    private int i;
+    SequentialSplit(int i) {
+      this.i = i;
+    }
+    public long getLength() { return 0; }
+    public String[] getLocations() { return new String[0]; }
+    public int getInit() { return i; }
+  }
+
+  static class TestInputSamplerIF
+      extends InputFormat<IntWritable,NullWritable> {
+
+    final int maxDepth;
+    final ArrayList<InputSplit> splits = new ArrayList<InputSplit>();
+
+    TestInputSamplerIF(int maxDepth, int numSplits, int... splitInit) {
+      this.maxDepth = maxDepth;
+      assert splitInit.length == numSplits;
+      for (int i = 0; i < numSplits; ++i) {
+        splits.add(new SequentialSplit(splitInit[i]));
+      }
+    }
+
+    public List<InputSplit> getSplits(JobContext context)
+        throws IOException, InterruptedException {
+      return splits;
+    }
+
+    public RecordReader<IntWritable,NullWritable> createRecordReader(
+        final InputSplit split, TaskAttemptContext context)
+        throws IOException, InterruptedException {
+      return new RecordReader<IntWritable,NullWritable>() {
+        private int maxVal;
+        private final IntWritable i = new IntWritable();
+        public void initialize(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+          i.set(((SequentialSplit)split).getInit() - 1);
+          maxVal = i.get() + maxDepth + 1;
+        }
+        public boolean nextKeyValue() {
+          i.set(i.get() + 1);
+          return i.get() < maxVal;
+        }
+        public IntWritable getCurrentKey() { return i; }
+        public NullWritable getCurrentValue() { return NullWritable.get(); }
+        public float getProgress() { return 1.0f; }
+        public void close() { }
+      };
+    }
+
+  }
+
+  /**
+   * Verify SplitSampler contract, that an equal number of records are taken
+   * from the first splits.
+   */
+  @Test
+  @SuppressWarnings("unchecked") // IntWritable comparator not typesafe
+  public void testSplitSampler() throws Exception {
+    final int TOT_SPLITS = 15;
+    final int NUM_SPLITS = 5;
+    final int STEP_SAMPLE = 5;
+    final int NUM_SAMPLES = NUM_SPLITS * STEP_SAMPLE;
+    InputSampler.Sampler<IntWritable,NullWritable> sampler =
+      new InputSampler.SplitSampler<IntWritable,NullWritable>(
+          NUM_SAMPLES, NUM_SPLITS);
+    int inits[] = new int[TOT_SPLITS];
+    for (int i = 0; i < TOT_SPLITS; ++i) {
+      inits[i] = i * STEP_SAMPLE;
+    }
+    Job ignored = new Job();
+    Object[] samples = sampler.getSample(
+        new TestInputSamplerIF(100000, TOT_SPLITS, inits), ignored);
+    assertEquals(NUM_SAMPLES, samples.length);
+    Arrays.sort(samples, new IntWritable.Comparator());
+    for (int i = 0; i < NUM_SAMPLES; ++i) {
+      assertEquals(i, ((IntWritable)samples[i]).get());
+    }
+  }
+
+  /**
+   * Verify IntervalSampler contract, that samples are taken at regular
+   * intervals from the given splits.
+   */
+  @Test
+  @SuppressWarnings("unchecked") // IntWritable comparator not typesafe
+  public void testIntervalSampler() throws Exception {
+    final int TOT_SPLITS = 16;
+    final int PER_SPLIT_SAMPLE = 4;
+    final int NUM_SAMPLES = TOT_SPLITS * PER_SPLIT_SAMPLE;
+    final double FREQ = 1.0 / TOT_SPLITS;
+    InputSampler.Sampler<IntWritable,NullWritable> sampler =
+      new InputSampler.IntervalSampler<IntWritable,NullWritable>(
+          FREQ, NUM_SAMPLES);
+    int inits[] = new int[TOT_SPLITS];
+    for (int i = 0; i < TOT_SPLITS; ++i) {
+      inits[i] = i;
+    }
+    Job ignored = new Job();
+    Object[] samples = sampler.getSample(new TestInputSamplerIF(
+          NUM_SAMPLES, TOT_SPLITS, inits), ignored);
+    assertEquals(NUM_SAMPLES, samples.length);
+    Arrays.sort(samples, new IntWritable.Comparator());
+    for (int i = 0; i < NUM_SAMPLES; ++i) {
+      assertEquals(i, ((IntWritable)samples[i]).get());
+    }
+  }
+
+}

+ 425 - 0
src/test/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java

@@ -0,0 +1,425 @@
+/**
+ * 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import junit.framework.TestCase;
+
+public class TestKeyFieldHelper extends TestCase {
+  private static final Log LOG = LogFactory.getLog(TestKeyFieldHelper.class);
+  /**
+   * Test is key-field-helper's parse option.
+   */
+  public void testparseOption() throws Exception {
+    KeyFieldHelper helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    String keySpecs = "-k1.2,3.4";
+    String eKeySpecs = keySpecs;
+    helper.parseOption(keySpecs);
+    String actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    // test -k a.b
+    keySpecs = "-k 1.2";
+    eKeySpecs = "-k1.2,0.0";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-nr -k1.2,3.4";
+    eKeySpecs = "-k1.2,3.4nr";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-nr -k1.2,3.4n";
+    eKeySpecs = "-k1.2,3.4n";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-nr -k1.2,3.4r";
+    eKeySpecs = "-k1.2,3.4r";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-nr -k1.2,3.4 -k5.6,7.8n -k9.10,11.12r -k13.14,15.16nr";
+    //1st
+    eKeySpecs = "-k1.2,3.4nr";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    // 2nd
+    eKeySpecs = "-k5.6,7.8n";
+    actKeySpecs = helper.keySpecs().get(1).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    //3rd
+    eKeySpecs = "-k9.10,11.12r";
+    actKeySpecs = helper.keySpecs().get(2).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    //4th
+    eKeySpecs = "-k13.14,15.16nr";
+    actKeySpecs = helper.keySpecs().get(3).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-k1.2n,3.4";
+    eKeySpecs = "-k1.2,3.4n";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-k1.2r,3.4";
+    eKeySpecs = "-k1.2,3.4r";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-k1.2nr,3.4";
+    eKeySpecs = "-k1.2,3.4nr";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-k1.2,3.4n";
+    eKeySpecs = "-k1.2,3.4n";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-k1.2,3.4r";
+    eKeySpecs = "-k1.2,3.4r";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-k1.2,3.4nr";
+    eKeySpecs = "-k1.2,3.4nr";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-nr -k1.2,3.4 -k5.6,7.8";
+    eKeySpecs = "-k1.2,3.4nr";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    eKeySpecs = "-k5.6,7.8nr";
+    actKeySpecs = helper.keySpecs().get(1).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-n -k1.2,3.4 -k5.6,7.8";
+    eKeySpecs = "-k1.2,3.4n";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    eKeySpecs = "-k5.6,7.8n";
+    actKeySpecs = helper.keySpecs().get(1).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-r -k1.2,3.4 -k5.6,7.8";
+    eKeySpecs = "-k1.2,3.4r";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    eKeySpecs = "-k5.6,7.8r";
+    actKeySpecs = helper.keySpecs().get(1).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-k1.2,3.4n -k5.6,7.8";
+    eKeySpecs = "-k1.2,3.4n";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    eKeySpecs = "-k5.6,7.8";
+    actKeySpecs = helper.keySpecs().get(1).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-k1.2,3.4r -k5.6,7.8";
+    eKeySpecs = "-k1.2,3.4r";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    eKeySpecs = "-k5.6,7.8";
+    actKeySpecs = helper.keySpecs().get(1).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-k1.2,3.4nr -k5.6,7.8";
+    eKeySpecs = "-k1.2,3.4nr";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    eKeySpecs = "-k5.6,7.8";
+    actKeySpecs = helper.keySpecs().get(1).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-n";
+    eKeySpecs = "-k1.1,0.0n";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-r";
+    eKeySpecs = "-k1.1,0.0r";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    
+    keySpecs = "-nr";
+    eKeySpecs = "-k1.1,0.0nr";
+    helper = new KeyFieldHelper();
+    helper.parseOption(keySpecs);
+    actKeySpecs = helper.keySpecs().get(0).toString();
+    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+  }
+  
+  /**
+   * Test is key-field-helper's getWordLengths.
+   */
+  public void testGetWordLengths() throws Exception {
+    KeyFieldHelper helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    // test getWordLengths with unspecified key-specifications
+    String input = "hi";
+    int[] result = helper.getWordLengths(input.getBytes(), 0, 2);
+    assertTrue(equals(result, new int[] {1}));
+    
+    // set the key specs
+    helper.setKeyFieldSpec(1, 2);
+    
+    // test getWordLengths with 3 words
+    input = "hi\thello there";
+    result = helper.getWordLengths(input.getBytes(), 0, input.length());
+    assertTrue(equals(result, new int[] {2, 2, 11}));
+    
+    // test getWordLengths with 4 words but with a different separator
+    helper.setKeyFieldSeparator(" ");
+    input = "hi hello\tthere you";
+    result = helper.getWordLengths(input.getBytes(), 0, input.length());
+    assertTrue(equals(result, new int[] {3, 2, 11, 3}));
+    
+    // test with non zero start index
+    input = "hi hello there you where me there";
+    //                 .....................
+    result = helper.getWordLengths(input.getBytes(), 10, 33);
+    assertTrue(equals(result, new int[] {5, 4, 3, 5, 2, 3}));
+    
+    input = "hi hello there you where me ";
+    //                 ..................
+    result = helper.getWordLengths(input.getBytes(), 10, input.length());
+    assertTrue(equals(result, new int[] {5, 4, 3, 5, 2, 0}));
+    
+    input = "";
+    result = helper.getWordLengths(input.getBytes(), 0, 0);
+    assertTrue(equals(result, new int[] {1, 0}));
+    
+    input = "  abc";
+    result = helper.getWordLengths(input.getBytes(), 0, 5);
+    assertTrue(equals(result, new int[] {3, 0, 0, 3}));
+    
+    input = "  abc";
+    result = helper.getWordLengths(input.getBytes(), 0, 2);
+    assertTrue(equals(result, new int[] {3, 0, 0, 0}));
+    
+    input = " abc ";
+    result = helper.getWordLengths(input.getBytes(), 0, 2);
+    assertTrue(equals(result, new int[] {2, 0, 1}));
+    
+    helper.setKeyFieldSeparator("abcd");
+    input = "abc";
+    result = helper.getWordLengths(input.getBytes(), 0, 3);
+    assertTrue(equals(result, new int[] {1, 3}));
+  }
+  
+  /**
+   * Test is key-field-helper's getStartOffset/getEndOffset.
+   */
+  public void testgetStartEndOffset() throws Exception {
+    KeyFieldHelper helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    // test getStartOffset with -k1,2
+    helper.setKeyFieldSpec(1, 2);
+    String input = "hi\thello";
+    String expectedOutput = input;
+    testKeySpecs(input, expectedOutput, helper);
+    
+    // test getStartOffset with -k1.0,0 .. should result into start = -1
+    helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k1.0,0");
+    testKeySpecs(input, null, helper);
+    
+    // test getStartOffset with -k1,0
+    helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k1,0");
+    expectedOutput = input;
+    testKeySpecs(input, expectedOutput, helper);
+    
+    // test getStartOffset with -k1.2,0
+    helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k1.2,0");
+    expectedOutput = "i\thello";
+    testKeySpecs(input, expectedOutput, helper);
+    
+    // test getWordLengths with -k1.0,2.3
+    helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k1.1,2.3");
+    expectedOutput = "hi\thel";
+    testKeySpecs(input, expectedOutput, helper);
+    
+    // test getWordLengths with -k1.2,2.3
+    helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k1.2,2.3");
+    expectedOutput = "i\thel";
+    testKeySpecs(input, expectedOutput, helper);
+    
+    // test getStartOffset with -k1.2,3.0
+    helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k1.2,3.0");
+    expectedOutput = "i\thello";
+    testKeySpecs(input, expectedOutput, helper);
+    
+    // test getStartOffset with -k2,2
+    helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k2,2");
+    expectedOutput = "hello";
+    testKeySpecs(input, expectedOutput, helper);
+    
+    // test getStartOffset with -k3.0,4.0
+    helper = new KeyFieldHelper();
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k3.1,4.0");
+    testKeySpecs(input, null, helper);
+    
+    // test getStartOffset with -k2.1
+    helper = new KeyFieldHelper();
+    input = "123123123123123hi\thello\thow";
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k2.1");
+    expectedOutput = "hello\thow";
+    testKeySpecs(input, expectedOutput, helper, 15, input.length());
+    
+    // test getStartOffset with -k2.1,4 with end ending on \t
+    helper = new KeyFieldHelper();
+    input = "123123123123123hi\thello\t\thow\tare";
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k2.1,3");
+    expectedOutput = "hello\t";
+    testKeySpecs(input, expectedOutput, helper, 17, input.length());
+    
+    // test getStartOffset with -k2.1 with end ending on \t
+    helper = new KeyFieldHelper();
+    input = "123123123123123hi\thello\thow\tare";
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k2.1");
+    expectedOutput = "hello\thow\t";
+    testKeySpecs(input, expectedOutput, helper, 17, 28);
+    
+    // test getStartOffset with -k2.1,3 with smaller length
+    helper = new KeyFieldHelper();
+    input = "123123123123123hi\thello\thow";
+    helper.setKeyFieldSeparator("\t");
+    helper.parseOption("-k2.1,3");
+    expectedOutput = "hello";
+    testKeySpecs(input, expectedOutput, helper, 15, 23);
+  }
+  
+  private void testKeySpecs(String input, String expectedOutput, 
+                            KeyFieldHelper helper) {
+    testKeySpecs(input, expectedOutput, helper, 0, -1);
+  }
+  
+  private void testKeySpecs(String input, String expectedOutput, 
+                            KeyFieldHelper helper, int s1, int e1) {
+    LOG.info("input : " + input);
+    String keySpecs = helper.keySpecs().get(0).toString();
+    LOG.info("keyspecs : " + keySpecs);
+    byte[] inputBytes = input.getBytes(); // get the input bytes
+    if (e1 == -1) {
+      e1 = inputBytes.length;
+    }
+    LOG.info("length : " + e1);
+    // get the word lengths
+    int[] indices = helper.getWordLengths(inputBytes, s1, e1);
+    // get the start index
+    int start = helper.getStartOffset(inputBytes, s1, e1, indices, 
+                                      helper.keySpecs().get(0));
+    LOG.info("start : " + start);
+    if (expectedOutput == null) {
+      assertEquals("Expected -1 when the start index is invalid", -1, start);
+      return;
+    }
+    // get the end index
+    int end = helper.getEndOffset(inputBytes, s1, e1, indices, 
+                                  helper.keySpecs().get(0));
+    LOG.info("end : " + end);
+    //my fix
+    end = (end >= inputBytes.length) ? inputBytes.length -1 : end;
+    int length = end + 1 - start;
+    LOG.info("length : " + length);
+    byte[] outputBytes = new byte[length];
+    System.arraycopy(inputBytes, start, outputBytes, 0, length);
+    String output = new String(outputBytes);
+    LOG.info("output : " + output);
+    LOG.info("expected-output : " + expectedOutput);
+    assertEquals(keySpecs + " failed on input '" + input + "'", 
+                 expectedOutput, output);
+  }
+
+  // check for equality of 2 int arrays
+  private boolean equals(int[] test, int[] expected) {
+    // check array length
+    if (test[0] != expected[0]) {
+      return false;
+    }
+    // if length is same then check the contents
+    for (int i = 0; i < test[0] && i < expected[0]; ++i) {
+      if (test[i] != expected[i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

+ 125 - 0
src/test/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java

@@ -0,0 +1,125 @@
+/**
+ * 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.conf.Configuration;
+import org.apache.hadoop.io.Text;
+
+import junit.framework.TestCase;
+
+public class TestMRKeyFieldBasedPartitioner extends TestCase {
+
+  /**
+   * Test is key-field-based partitioned works with empty key.
+   */
+  public void testEmptyKey() throws Exception {
+    int numReducers = 10;
+    KeyFieldBasedPartitioner<Text, Text> kfbp = 
+      new KeyFieldBasedPartitioner<Text, Text>();
+    Configuration conf = new Configuration();
+    conf.setInt("num.key.fields.for.partition", 10);
+    kfbp.setConf(conf);
+    assertEquals("Empty key should map to 0th partition", 
+                 0, kfbp.getPartition(new Text(), new Text(), numReducers));
+    
+    // check if the hashcode is correct when no keyspec is specified
+    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
+    conf = new Configuration();
+    kfbp.setConf(conf);
+    String input = "abc\tdef\txyz";
+    int hashCode = input.hashCode();
+    int expectedPartition = kfbp.getPartition(hashCode, numReducers);
+    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
+                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    
+    // check if the hashcode is correct with specified keyspec
+    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
+    conf = new Configuration();
+    conf.set(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS, "-k2,2");
+    kfbp.setConf(conf);
+    String expectedOutput = "def";
+    byte[] eBytes = expectedOutput.getBytes();
+    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
+    expectedPartition = kfbp.getPartition(hashCode, numReducers);
+    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
+                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    
+    // test with invalid end index in keyspecs
+    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
+    conf = new Configuration();
+    conf.set(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS, "-k2,5");
+    kfbp.setConf(conf);
+    expectedOutput = "def\txyz";
+    eBytes = expectedOutput.getBytes();
+    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
+    expectedPartition = kfbp.getPartition(hashCode, numReducers);
+    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
+                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    
+    // test with 0 end index in keyspecs
+    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
+    conf = new Configuration();
+    conf.set(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS, "-k2");
+    kfbp.setConf(conf);
+    expectedOutput = "def\txyz";
+    eBytes = expectedOutput.getBytes();
+    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
+    expectedPartition = kfbp.getPartition(hashCode, numReducers);
+    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
+                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    
+    // test with invalid keyspecs
+    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
+    conf = new Configuration();
+    conf.set(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS, "-k10");
+    kfbp.setConf(conf);
+    assertEquals("Partitioner doesnt work as expected", 0, 
+                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    
+    // test with multiple keyspecs
+    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
+    conf = new Configuration();
+    conf.set(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS, "-k2,2 -k4,4");
+    kfbp.setConf(conf);
+    input = "abc\tdef\tpqr\txyz";
+    expectedOutput = "def";
+    eBytes = expectedOutput.getBytes();
+    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
+    expectedOutput = "xyz";
+    eBytes = expectedOutput.getBytes();
+    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, hashCode);
+    expectedPartition = kfbp.getPartition(hashCode, numReducers);
+    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
+                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    
+    // test with invalid start index in keyspecs
+    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
+    conf = new Configuration();
+    conf.set(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS, "-k2,2 -k30,21 -k4,4 -k5");
+    kfbp.setConf(conf);
+    expectedOutput = "def";
+    eBytes = expectedOutput.getBytes();
+    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
+    expectedOutput = "xyz";
+    eBytes = expectedOutput.getBytes();
+    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, hashCode);
+    expectedPartition = kfbp.getPartition(hashCode, numReducers);
+    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
+                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+  }
+}

+ 187 - 0
src/test/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java

@@ -0,0 +1,187 @@
+/**
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+
+public class TestTotalOrderPartitioner extends TestCase {
+
+  private static final Text[] splitStrings = new Text[] {
+    // -inf            // 0
+    new Text("aabbb"), // 1
+    new Text("babbb"), // 2
+    new Text("daddd"), // 3
+    new Text("dddee"), // 4
+    new Text("ddhee"), // 5
+    new Text("dingo"), // 6
+    new Text("hijjj"), // 7
+    new Text("n"),     // 8
+    new Text("yak"),   // 9
+  };
+
+  static class Check<T> {
+    T data;
+    int part;
+    Check(T data, int part) {
+      this.data = data;
+      this.part = part;
+    }
+  }
+
+  private static final ArrayList<Check<Text>> testStrings =
+    new ArrayList<Check<Text>>();
+  static {
+    testStrings.add(new Check<Text>(new Text("aaaaa"), 0));
+    testStrings.add(new Check<Text>(new Text("aaabb"), 0));
+    testStrings.add(new Check<Text>(new Text("aabbb"), 1));
+    testStrings.add(new Check<Text>(new Text("aaaaa"), 0));
+    testStrings.add(new Check<Text>(new Text("babbb"), 2));
+    testStrings.add(new Check<Text>(new Text("baabb"), 1));
+    testStrings.add(new Check<Text>(new Text("yai"), 8));
+    testStrings.add(new Check<Text>(new Text("yak"), 9));
+    testStrings.add(new Check<Text>(new Text("z"), 9));
+    testStrings.add(new Check<Text>(new Text("ddngo"), 5));
+    testStrings.add(new Check<Text>(new Text("hi"), 6));
+  };
+
+  private static <T extends WritableComparable<?>> Path writePartitionFile(
+      String testname, Configuration conf, T[] splits) throws IOException {
+    final FileSystem fs = FileSystem.getLocal(conf);
+    final Path testdir = new Path(System.getProperty("test.build.data", "/tmp")
+                                 ).makeQualified(fs);
+    Path p = new Path(testdir, testname + "/_partition.lst");
+    TotalOrderPartitioner.setPartitionFile(conf, p);
+    conf.setInt("mapred.reduce.tasks", splits.length + 1);
+    SequenceFile.Writer w = null;
+    try {
+      w = SequenceFile.createWriter(fs, conf, p,
+          splits[0].getClass(), NullWritable.class,
+          SequenceFile.CompressionType.NONE);
+      for (int i = 0; i < splits.length; ++i) {
+        w.append(splits[i], NullWritable.get());
+      }
+    } finally {
+      if (null != w)
+        w.close();
+    }
+    return p;
+  }
+
+  public void testTotalOrderMemCmp() throws Exception {
+    TotalOrderPartitioner<Text,NullWritable> partitioner =
+      new TotalOrderPartitioner<Text,NullWritable>();
+    Configuration conf = new Configuration();
+    Path p = TestTotalOrderPartitioner.<Text>writePartitionFile(
+        "totalordermemcmp", conf, splitStrings);
+    conf.setClass("mapred.mapoutput.key.class", Text.class, Object.class);
+    try {
+      partitioner.setConf(conf);
+      NullWritable nw = NullWritable.get();
+      for (Check<Text> chk : testStrings) {
+        assertEquals(chk.data.toString(), chk.part,
+            partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
+      }
+    } finally {
+      p.getFileSystem(conf).delete(p, true);
+    }
+  }
+
+  public void testTotalOrderBinarySearch() throws Exception {
+    TotalOrderPartitioner<Text,NullWritable> partitioner =
+      new TotalOrderPartitioner<Text,NullWritable>();
+    Configuration conf = new Configuration();
+    Path p = TestTotalOrderPartitioner.<Text>writePartitionFile(
+        "totalorderbinarysearch", conf, splitStrings);
+    conf.setBoolean(TotalOrderPartitioner.NATURAL_ORDER, false);
+    conf.setClass("mapred.mapoutput.key.class", Text.class, Object.class);
+    try {
+      partitioner.setConf(conf);
+      NullWritable nw = NullWritable.get();
+      for (Check<Text> chk : testStrings) {
+        assertEquals(chk.data.toString(), chk.part,
+            partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
+      }
+    } finally {
+      p.getFileSystem(conf).delete(p, true);
+    }
+  }
+
+  public static class ReverseStringComparator implements RawComparator<Text> {
+    public int compare(Text a, Text b) {
+      return -a.compareTo(b);
+    }
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+      int n1 = WritableUtils.decodeVIntSize(b1[s1]);
+      int n2 = WritableUtils.decodeVIntSize(b2[s2]);
+      return -1 * WritableComparator.compareBytes(b1, s1+n1, l1-n1,
+                                                  b2, s2+n2, l2-n2);
+    }
+  }
+
+  public void testTotalOrderCustomComparator() throws Exception {
+    TotalOrderPartitioner<Text,NullWritable> partitioner =
+      new TotalOrderPartitioner<Text,NullWritable>();
+    Configuration conf = new Configuration();
+    Text[] revSplitStrings = Arrays.copyOf(splitStrings, splitStrings.length);
+    Arrays.sort(revSplitStrings, new ReverseStringComparator());
+    Path p = TestTotalOrderPartitioner.<Text>writePartitionFile(
+        "totalordercustomcomparator", conf, revSplitStrings);
+    conf.setBoolean(TotalOrderPartitioner.NATURAL_ORDER, false);
+    conf.setClass("mapred.mapoutput.key.class", Text.class, Object.class);
+    conf.setClass("mapred.output.key.comparator.class",
+      ReverseStringComparator.class, RawComparator.class);
+    ArrayList<Check<Text>> revCheck = new ArrayList<Check<Text>>();
+    revCheck.add(new Check<Text>(new Text("aaaaa"), 9));
+    revCheck.add(new Check<Text>(new Text("aaabb"), 9));
+    revCheck.add(new Check<Text>(new Text("aabbb"), 9));
+    revCheck.add(new Check<Text>(new Text("aaaaa"), 9));
+    revCheck.add(new Check<Text>(new Text("babbb"), 8));
+    revCheck.add(new Check<Text>(new Text("baabb"), 8));
+    revCheck.add(new Check<Text>(new Text("yai"), 1));
+    revCheck.add(new Check<Text>(new Text("yak"), 1));
+    revCheck.add(new Check<Text>(new Text("z"), 0));
+    revCheck.add(new Check<Text>(new Text("ddngo"), 4));
+    revCheck.add(new Check<Text>(new Text("hi"), 3));
+    try {
+      partitioner.setConf(conf);
+      NullWritable nw = NullWritable.get();
+      for (Check<Text> chk : revCheck) {
+        assertEquals(chk.data.toString(), chk.part,
+            partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
+      }
+    } finally {
+      p.getFileSystem(conf).delete(p, true);
+    }
+  }
+}