hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cutt...@apache.org
Subject svn commit: r762509 [1/2] - in /hadoop/core/trunk: ./ ivy/ src/contrib/ src/contrib/mrunit/ src/contrib/mrunit/doc/ src/contrib/mrunit/ivy/ src/contrib/mrunit/src/ src/contrib/mrunit/src/java/ src/contrib/mrunit/src/java/org/ src/contrib/mrunit/src/jav...
Date Mon, 06 Apr 2009 21:23:27 GMT
Author: cutting
Date: Mon Apr  6 21:23:26 2009
New Revision: 762509

URL: http://svn.apache.org/viewvc?rev=762509&view=rev
Log:
HADOOP-5518.  Add contrib/mrunit, a MapReduce unit test framework.  Contributed by Aaron Kimball.

Added:
    hadoop/core/trunk/src/contrib/mrunit/
    hadoop/core/trunk/src/contrib/mrunit/README.txt
    hadoop/core/trunk/src/contrib/mrunit/build.xml
    hadoop/core/trunk/src/contrib/mrunit/doc/
    hadoop/core/trunk/src/contrib/mrunit/doc/overview.html
    hadoop/core/trunk/src/contrib/mrunit/ivy/
    hadoop/core/trunk/src/contrib/mrunit/ivy.xml
    hadoop/core/trunk/src/contrib/mrunit/ivy/libraries.properties
    hadoop/core/trunk/src/contrib/mrunit/src/
    hadoop/core/trunk/src/contrib/mrunit/src/java/
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapDriver.java
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapReduceDriver.java
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/ReduceDriver.java
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/TestDriver.java
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockInputSplit.java
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockOutputCollector.java
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockReporter.java
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/testutil/
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/testutil/ExtendedAssert.java
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/types/
    hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/types/Pair.java
    hadoop/core/trunk/src/contrib/mrunit/src/test/
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/AllTests.java
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestExample.java
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapDriver.java
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapReduceDriver.java
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestReduceDriver.java
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestTestDriver.java
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mock/
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mock/TestMockReporter.java
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/AllTests.java
    hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/TestPair.java
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/ivy/libraries.properties
    hadoop/core/trunk/src/contrib/build.xml

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=762509&r1=762508&r2=762509&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Mon Apr  6 21:23:26 2009
@@ -77,6 +77,9 @@
     HADOOP-5450. Add application-specific data types to streaming's typed bytes
     interface. (Klaas Bosteels via omalley)
 
+    HADOOP-5518. Add contrib/mrunit, a MapReduce unit test framework.
+    (Aaron Kimball via cutting)
+
   IMPROVEMENTS
 
     HADOOP-4565. Added CombineFileInputFormat to use data locality information

Modified: hadoop/core/trunk/ivy/libraries.properties
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/ivy/libraries.properties?rev=762509&r1=762508&r2=762509&view=diff
==============================================================================
--- hadoop/core/trunk/ivy/libraries.properties (original)
+++ hadoop/core/trunk/ivy/libraries.properties Mon Apr  6 21:23:26 2009
@@ -45,7 +45,7 @@
 jets3t.version=0.6.1
 jetty.version=6.1.14
 jetty-util.version=6.1.14
-junit.version=3.8.1
+junit.version=4.5
 jdiff.version=1.0.9
 json.version=1.0
 

Modified: hadoop/core/trunk/src/contrib/build.xml
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/build.xml?rev=762509&r1=762508&r2=762509&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/build.xml (original)
+++ hadoop/core/trunk/src/contrib/build.xml Mon Apr  6 21:23:26 2009
@@ -50,6 +50,7 @@
       <fileset dir="." includes="streaming/build.xml"/>
       <fileset dir="." includes="fairscheduler/build.xml"/>
       <fileset dir="." includes="capacity-scheduler/build.xml"/>
+      <fileset dir="." includes="mrunit/build.xml"/>
     </subant>
   </target>
   

Added: hadoop/core/trunk/src/contrib/mrunit/README.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/README.txt?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/README.txt (added)
+++ hadoop/core/trunk/src/contrib/mrunit/README.txt Mon Apr  6 21:23:26 2009
@@ -0,0 +1,3 @@
+
+MRUnit is a unit test driver for MapReduce programs for use with JUnit. See the overview
+in the Javadoc for more details.

Added: hadoop/core/trunk/src/contrib/mrunit/build.xml
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/build.xml?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/build.xml (added)
+++ hadoop/core/trunk/src/contrib/mrunit/build.xml Mon Apr  6 21:23:26 2009
@@ -0,0 +1,28 @@
+<?xml version="1.0"?>
+
+<!--
+   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.
+-->
+
+<!-- 
+Before you can run these subtargets directly, you need 
+to call at top-level: ant deploy-contrib compile-core-test
+-->
+<project name="mrunit" default="jar">
+
+  <import file="../build-contrib.xml"/>
+
+</project>

Added: hadoop/core/trunk/src/contrib/mrunit/doc/overview.html
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/doc/overview.html?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/doc/overview.html (added)
+++ hadoop/core/trunk/src/contrib/mrunit/doc/overview.html Mon Apr  6 21:23:26 2009
@@ -0,0 +1,129 @@
+<body>
+<!--
+   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.
+-->
+<p>
+MRUnit is a library designed to allow easy testing of Mapper and Reducer
+classes using existing tools such as JUnit. MRUnit provides mock
+implementations of OutputCollector and Reporter for use in calling <tt>Mapper.map()</tt>
+and <tt>Reducer.reduce()</tt>, as well as a set of "driver" classes that manage
+delivery of key/value pair inputs to tasks, and comparison of actual task
+outputs with expected outputs.
+</p>
+
+<p>
+The primary advantage of MRUnit is that it allows you to test the outputs
+of individual maps and reduces, as well as the composition of the two, without
+needing to use the MiniMR cluster, or start a real MapReduce job in Hadoop,
+which are time-consuming processes.
+</p>
+
+<h3>Using MRUnit</h3>
+
+  MRUnit is designed to allow you to write ordinary JUnit test suites.
+
+<ul>
+<li> Include lib/mrunit-0.1.jar and lib/junit-4.4.jar on the classpath when
+    building and testing.</li>
+<li> The test methods for your Mapper implementation should use instances
+    of the <tt>MapDriver</tt>.</li>
+<li> The test methods for your Reducer implementation should use instances
+    of the <tt>ReduceDriver</tt>.</li>
+<li> MapReduce "jobs" consisting of a small number of inputs can be tested
+    with the <tt>MapReduceDriver</tt>. This supports a simple "shuffle" of outputs
+    to maintain the expected input delivery semantics to the reducer.</li>
+</ul>
+
+<p>
+A <tt>MapDriver</tt> or <tt>ReduceDriver</tt> instance is created for each test,
+as well as a fresh instance of your Mapper or Reducer. The driver is configured
+with the input keys and values, and expected output keys and values.
+</p>
+
+<p>
+  The <tt>run()</tt> method will execute the map or reduce, and returns the outputs
+retrieved from the OutputCollector.
+  The <tt>runTest()</tt> method will execute the map or reduce, and compares the
+actual outputs with the expected outputs, and returns true to indicate
+success and false on failure.
+  When expecting multiple outputs, the test drivers enforce that the order of
+the actual outputs is the same as the order in which outputs are configured
+(i.e., the order of calls to <tt>withOutput()</tt> or <tt>addOutput()</tt>).
+</p>
+
+<h3>Example</h3>
+
+<p>
+A brief test of Hadoop's <tt>IdentityMapper</tt> is presented here:
+</p>
+
+<div><tt><pre>
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestExample extends TestCase {
+
+  private Mapper<Text, Text, Text, Text> mapper;
+  private MapDriver<Text, Text, Text, Text> driver;
+
+  &#64;Before
+  public void setUp() {
+    mapper = new IdentityMapper<Text, Text>();
+    driver = new MapDriver<Text, Text, Text, Text>(mapper);
+  }
+
+  &#64;Test
+  public void testIdentityMapper() {
+    driver.withInput(new Text("foo"), new Text("bar"))
+            .withOutput(new Text("foo"), new Text("bar"))
+            .runTest();
+  }
+}
+</pre></tt></div>
+
+<p>
+  This test first instantiates the Mapper and MapDriver. It configures
+an input (key, value) pair consisting of the strings "foo" and "bar",
+and expects these same values as output. It then calls <tt>runTest()</tt> to
+actually invoke the mapper, and compare the actual and expected outputs.
+The <tt>runTest()</tt> method will throw a RuntimeException if the output
+is not what it expects, which causes JUnit to mark the test case as failed.
+</p>
+
+<p>All <tt>with*()</tt> methods in MRUnit return a reference to <tt>this</tt>
+to allow them to be easily chained (e.g.,
+<tt>driver.withInput(a, b).withOutput(c, d).withOutput(d, e)...</tt>). These
+methods are analogous to the more conventional <tt>setInput()</tt>, <tt>addOutput()</tt>,
+etc. methods, which are also included.</p>
+
+<p>
+  Further examples of MRUnit usage can be seen in its own <tt>test/</tt> directory.
+The above example is in <tt>org.apache.hadoop.mrunit.TestExample</tt>. Further
+&quot;tests&quot; of the IdentityMapper are used to test the correctness of MRUnit
+itself; <tt>org.apache.hadoop.mrunit.TestMapDriver</tt> includes several tests of
+correctness for the MapDriver class; the <tt>testRunTest*()</tt> methods show
+how to apply the MapDriver to the IdentityMapper to confirm behavior surrounding
+both correct and incorrect input/output data sets. The <tt>testRunTest*()</tt> methods
+in <tt>org.apache.hadoop.mrunit.TestReduceDriver</tt> show how to apply the ReduceDriver
+test component to the LongSumReducer class.
+</p>
+
+</body>

Added: hadoop/core/trunk/src/contrib/mrunit/ivy.xml
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/ivy.xml?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/ivy.xml (added)
+++ hadoop/core/trunk/src/contrib/mrunit/ivy.xml Mon Apr  6 21:23:26 2009
@@ -0,0 +1,56 @@
+<?xml version="1.0" ?>
+
+<!--
+   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.
+-->
+
+<ivy-module version="1.0">
+  <info organisation="org.apache.hadoop" module="${ant.project.name}">
+    <license name="Apache 2.0"/>
+    <ivyauthor name="Apache Hadoop Team" url="http://hadoop.apache.org"/>
+    <description>
+        Apache Hadoop contrib
+    </description>
+  </info>
+  <configurations defaultconfmapping="default">
+    <!--these match the Maven configurations-->
+    <conf name="default" extends="master,runtime"/>
+    <conf name="master" description="contains the artifact but no dependencies"/>
+    <conf name="runtime" description="runtime but not the artifact" />
+
+    <conf name="common" visibility="private" 
+      description="artifacts needed to compile/test the application"/>
+  </configurations>
+
+  <publications>
+    <!--get the artifact from our module name-->
+    <artifact conf="master"/>
+  </publications>
+  <dependencies>
+    <dependency org="commons-logging"
+      name="commons-logging"
+      rev="${commons-logging.version}"
+      conf="common->master"/>
+    <dependency org="log4j"
+      name="log4j"
+      rev="${log4j.version}"
+      conf="common->master"/>
+   <dependency org="junit"
+      name="junit"
+      rev="${junit.version}"
+      conf="common->master"/>
+  </dependencies>
+</ivy-module>

Added: hadoop/core/trunk/src/contrib/mrunit/ivy/libraries.properties
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/ivy/libraries.properties?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/ivy/libraries.properties (added)
+++ hadoop/core/trunk/src/contrib/mrunit/ivy/libraries.properties Mon Apr  6 21:23:26 2009
@@ -0,0 +1,17 @@
+#   Licensed 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.
+
+#This properties file lists the versions of the various artifacts used by streaming.
+#It drives ivy and the generation of a maven POM
+
+#Please list the dependencies name with version if they are different from the ones 
+#listed in the global libraries.properties file (in alphabetical order)

Added: hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapDriver.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapDriver.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapDriver.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapDriver.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,328 @@
+/**
+ * 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.mrunit;
+
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mrunit.mock.MockOutputCollector;
+import org.apache.hadoop.mrunit.mock.MockReporter;
+import org.apache.hadoop.mrunit.types.Pair;
+
+/**
+ * Harness that allows you to test a Mapper instance. You provide the input
+ * key and value that should be sent to the Mapper, and outputs you expect to
+ * be sent by the Mapper to the collector for those inputs. By calling
+ * runTest(), the harness will deliver the input to the Mapper and will check
+ * its outputs against the expected results. This is designed to handle a
+ * single (k, v) -> (k, v)* case from the Mapper, representing a single unit
+ * test. Multiple input (k, v) pairs should go in separate unit tests.
+ */
+public class MapDriver<K1, V1, K2, V2> extends TestDriver<K1, V1, K2, V2> {
+
+  public static final Log LOG = LogFactory.getLog(MapDriver.class);
+
+  private Mapper<K1, V1, K2, V2> myMapper;
+
+  private K1 inputKey;
+  private V1 inputVal;
+
+  public MapDriver(final Mapper<K1, V1, K2, V2> m) {
+    myMapper = m;
+  }
+
+  public MapDriver() {
+  }
+
+
+  /**
+   * Set the Mapper instance to use with this test driver
+   *
+   * @param m the Mapper instance to use
+   */
+  public void setMapper(Mapper<K1, V1, K2, V2> m) {
+    myMapper = m;
+  }
+
+  /** Sets the Mapper instance to use and returns self for fluent style */
+  public MapDriver<K1, V1, K2, V2> withMapper(Mapper<K1, V1, K2, V2> m) {
+    setMapper(m);
+    return this;
+  }
+
+  /**
+   * @return the Mapper object being used by this test
+   */
+  public Mapper<K1, V1, K2, V2> getMapper() {
+    return myMapper;
+  }
+
+  /**
+   * Sets the input key to send to the mapper
+   *
+   */
+  public void setInputKey(K1 key) {
+    inputKey = key;
+  }
+
+  public K1 getInputKey() {
+    return inputKey;
+  }
+
+  /**
+   * Identical to setInputKey() but with fluent programming style
+   *
+   * @return this
+   */
+  public MapDriver<K1, V1, K2, V2> withInputKey(K1 key) {
+    setInputKey(key);
+    return this;
+  }
+
+  /**
+   * Sets the input value to send to the mapper
+   *
+   * @param val
+   */
+  public void setInputValue(V1 val) {
+    inputVal = val;
+  }
+
+  public V1 getInputValue() {
+    return inputVal;
+  }
+
+  /**
+   * Identical to setInputValue() but with fluent programming style
+   *
+   * @param val
+   * @return this
+   */
+  public MapDriver<K1, V1, K2, V2> withInputValue(V1 val) {
+    setInputValue(val);
+    return this;
+  }
+
+  /**
+   * Sets the input to send to the mapper
+   *
+   */
+  public void setInput(K1 key, V1 val) {
+    setInputKey(key);
+    setInputValue(val);
+  }
+
+  /**
+   * Identical to setInput() but returns self for fluent programming style
+   *
+   * @return this
+   */
+  public MapDriver<K1, V1, K2, V2> withInput(K1 key, V1 val) {
+    setInput(key, val);
+    return this;
+  }
+
+  /**
+   * Sets the input to send to the mapper
+   *
+   * @param inputRecord
+   *          a (key, val) pair
+   */
+  public void setInput(Pair<K1, V1> inputRecord) {
+    if (null != inputRecord) {
+      setInputKey(inputRecord.getFirst());
+      setInputValue(inputRecord.getSecond());
+    } else {
+      throw new IllegalArgumentException("null inputRecord in setInput()");
+    }
+  }
+
+  /**
+   * Identical to setInput() but returns self for fluent programming style
+   *
+   * @param inputRecord
+   * @return this
+   */
+  public MapDriver<K1, V1, K2, V2> withInput(Pair<K1, V1> inputRecord) {
+    setInput(inputRecord);
+    return this;
+  }
+
+  /**
+   * Adds an output (k, v) pair we expect from the Mapper
+   *
+   * @param outputRecord
+   *          The (k, v) pair to add
+   */
+  public void addOutput(Pair<K2, V2> outputRecord) {
+    if (null != outputRecord) {
+      expectedOutputs.add(outputRecord);
+    } else {
+      throw new IllegalArgumentException("Tried to add null outputRecord");
+    }
+  }
+
+  /**
+   * Works like addOutput(), but returns self for fluent style
+   *
+   * @param outputRecord
+   * @return this
+   */
+  public MapDriver<K1, V1, K2, V2> withOutput(Pair<K2, V2> outputRecord) {
+    addOutput(outputRecord);
+    return this;
+  }
+
+  /**
+   * Adds a (k, v) pair we expect as output from the mapper
+   *
+   */
+  public void addOutput(K2 key, V2 val) {
+    addOutput(new Pair<K2, V2>(key, val));
+  }
+
+  /**
+   * Functions like addOutput() but returns self for fluent programming
+   * style
+   *
+   * @return this
+   */
+  public MapDriver<K1, V1, K2, V2> withOutput(K2 key, V2 val) {
+    addOutput(key, val);
+    return this;
+  }
+
+  /**
+   * Expects an input of the form "key \t val" Forces the Mapper input types
+   * to Text.
+   *
+   * @param input
+   *          A string of the form "key \t val".
+   */
+  public void setInputFromString(String input) {
+    if (null == input) {
+      throw new IllegalArgumentException("null input given to setInputFromString");
+    } else {
+      Pair<Text, Text> inputPair = parseTabbedPair(input);
+      if (null != inputPair) {
+        // I know this is not type-safe, but I don't know a better way to do
+        // this.
+        setInputKey((K1) inputPair.getFirst());
+        setInputValue((V1) inputPair.getSecond());
+      } else {
+        throw new IllegalArgumentException(
+            "Could not parse input pair in setInputFromString");
+      }
+    }
+  }
+
+  /**
+   * Identical to setInputFromString, but with a fluent programming style
+   *
+   * @param input
+   *          A string of the form "key \t val". Trims any whitespace.
+   * @return this
+   */
+  public MapDriver<K1, V1, K2, V2> withInputFromString(String input) {
+    setInputFromString(input);
+    return this;
+  }
+
+  /**
+   * Expects an input of the form "key \t val" Forces the Mapper output types
+   * to Text.
+   *
+   * @param output
+   *          A string of the form "key \t val". Trims any whitespace.
+   */
+  public void addOutputFromString(String output) {
+    if (null == output) {
+      throw new IllegalArgumentException("null input given to setOutput");
+    } else {
+      Pair<Text, Text> outputPair = parseTabbedPair(output);
+      if (null != outputPair) {
+        // I know this is not type-safe, but I don't know a better way to do
+        // this.
+        addOutput((Pair<K2, V2>) outputPair);
+      } else {
+        throw new IllegalArgumentException("Could not parse output pair in setOutput");
+      }
+    }
+  }
+
+  /**
+   * Identical to addOutputFromString, but with a fluent programming style
+   *
+   * @param output
+   *          A string of the form "key \t val". Trims any whitespace.
+   * @return this
+   */
+  public MapDriver<K1, V1, K2, V2> withOutputFromString(String output) {
+    addOutputFromString(output);
+    return this;
+  }
+
+  @Override
+  public List<Pair<K2, V2>> run() throws IOException {
+    MockOutputCollector<K2, V2> outputCollector =
+      new MockOutputCollector<K2, V2>();
+    MockReporter reporter = new MockReporter(MockReporter.ReporterType.Mapper);
+
+    myMapper.map(inputKey, inputVal, outputCollector, reporter);
+
+    return outputCollector.getOutputs();
+  }
+
+  @Override
+  public void runTest() throws RuntimeException {
+    String inputKeyStr = "(null)";
+    String inputValStr = "(null)";
+
+    if (null != inputKey) {
+      inputKeyStr = inputKey.toString();
+    }
+
+    if (null != inputVal) {
+      inputValStr = inputVal.toString();
+    }
+
+    LOG.debug("Mapping input (" + inputKeyStr + ", " + inputValStr + ")");
+
+    List<Pair<K2, V2>> outputs = null;
+
+    try {
+      outputs = run();
+      validate(outputs);
+    } catch (IOException ioe) {
+      LOG.error("IOException in mapper: " + ioe.toString());
+      LOG.debug("Setting success to false based on IOException");
+      throw new RuntimeException();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "MapDriver (" + myMapper + ")";
+  }
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapReduceDriver.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapReduceDriver.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapReduceDriver.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/MapReduceDriver.java Mon Apr  6 21:23:26 2009
@@ -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.mrunit;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mrunit.types.Pair;
+
+/**
+ * Harness that allows you to test a Mapper and a Reducer instance together
+ * You provide the input key and value that should be sent to the Mapper, and
+ * outputs you expect to be sent by the Reducer to the collector for those
+ * inputs. By calling runTest(), the harness will deliver the input to the
+ * Mapper, feed the intermediate results to the Reducer (without checking
+ * them), and will check the Reducer's outputs against the expected results.
+ * This is designed to handle a single (k, v)* -> (k, v)* case from the
+ * Mapper/Reducer pair, representing a single unit test.
+ */
+public class MapReduceDriver<K1, V1, K2, V2, K3, V3>
+    extends TestDriver<K1, V1, K3, V3> {
+
+  public static final Log LOG = LogFactory.getLog(MapReduceDriver.class);
+
+  private Mapper<K1, V1, K2, V2> myMapper;
+  private Reducer<K2, V2, K3, V3> myReducer;
+
+  private List<Pair<K1, V1>> inputList;
+
+  public MapReduceDriver(final Mapper<K1, V1, K2, V2> m,
+                         final Reducer<K2, V2, K3, V3> r) {
+    myMapper = m;
+    myReducer = r;
+    inputList = new ArrayList<Pair<K1, V1>>();
+  }
+
+  public MapReduceDriver() {
+    inputList = new ArrayList<Pair<K1, V1>>();
+  }
+
+  /** Set the Mapper instance to use with this test driver
+   * @param m the Mapper instance to use */
+  public void setMapper(Mapper<K1, V1, K2, V2> m) {
+    myMapper = m;
+  }
+
+  /** Sets the Mapper instance to use and returns self for fluent style */
+  public MapReduceDriver<K1, V1, K2, V2, K3, V3> withMapper(
+          Mapper<K1, V1, K2, V2> m) {
+    setMapper(m);
+    return this;
+  }
+
+  /**
+   * @return the Mapper object being used by this test
+   */
+  public Mapper<K1, V1, K2, V2> getMapper() {
+    return myMapper;
+  }
+
+  /**
+   * Sets the reducer object to use for this test
+   * @param r The reducer object to use
+   */
+  public void setReducer(Reducer<K2, V2, K3, V3> r) {
+    myReducer = r;
+  }
+
+  /**
+   * Identical to setReducer(), but with fluent programming style
+   * @param r The Reducer to use
+   * @return this
+   */
+  public MapReduceDriver<K1, V1, K2, V2, K3, V3> withReducer(
+          Reducer<K2, V2, K3, V3> r) {
+    setReducer(r);
+    return this;
+  }
+
+  /**
+   * @return the Reducer object being used for this test
+   */
+  public Reducer<K2, V2, K3, V3> getReducer() {
+    return myReducer;
+  }
+
+  /**
+   * Adds an input to send to the mapper
+   * @param key
+   * @param val
+   */
+  public void addInput(K1 key, V1 val) {
+    inputList.add(new Pair<K1, V1>(key, val));
+  }
+
+  /**
+   * Identical to addInput() but returns self for fluent programming style
+   * @param key
+   * @param val
+   * @return this
+   */
+  public MapReduceDriver<K1, V1, K2, V2, K3, V3> withInput(K1 key, V1 val) {
+    addInput(key, val);
+    return this;
+  }
+
+  /**
+   * Adds an input to send to the Mapper
+   * @param input The (k, v) pair to add to the input list.
+   */
+  public void addInput(Pair<K1, V1> input) {
+    if (null == input) {
+      throw new IllegalArgumentException("Null input in addInput()");
+    }
+
+    inputList.add(input);
+  }
+
+  /**
+   * Identical to addInput() but returns self for fluent programming style
+   * @param input The (k, v) pair to add
+   * @return this
+   */
+  public MapReduceDriver<K1, V1, K2, V2, K3, V3> withInput(
+      Pair<K1, V1> input) {
+    addInput(input);
+    return this;
+  }
+
+  /**
+   * Adds an output (k, v) pair we expect from the Reducer
+   * @param outputRecord The (k, v) pair to add
+   */
+  public void addOutput(Pair<K3, V3> outputRecord) {
+    if (null != outputRecord) {
+      expectedOutputs.add(outputRecord);
+    } else {
+      throw new IllegalArgumentException("Tried to add null outputRecord");
+    }
+  }
+
+  /**
+   * Works like addOutput(), but returns self for fluent style
+   * @param outputRecord
+   * @return this
+   */
+  public MapReduceDriver<K1, V1, K2, V2, K3, V3> withOutput(
+          Pair<K3, V3> outputRecord) {
+    addOutput(outputRecord);
+    return this;
+  }
+
+  /**
+   * Adds a (k, v) pair we expect as output from the Reducer
+   * @param key
+   * @param val
+   */
+  public void addOutput(K3 key, V3 val) {
+    addOutput(new Pair<K3, V3>(key, val));
+  }
+
+  /**
+   * Functions like addOutput() but returns self for fluent programming style
+   * @param key
+   * @param val
+   * @return this
+   */
+  public MapReduceDriver<K1, V1, K2, V2, K3, V3> withOutput(K3 key, V3 val) {
+    addOutput(key, val);
+    return this;
+  }
+
+  /**
+   * Expects an input of the form "key \t val"
+   * Forces the Mapper input types to Text.
+   * @param input A string of the form "key \t val". Trims any whitespace.
+   */
+  public void addInputFromString(String input) {
+    if (null == input) {
+      throw new IllegalArgumentException("null input given to setInput");
+    } else {
+      Pair<Text, Text> inputPair = parseTabbedPair(input);
+      if (null != inputPair) {
+        // I know this is not type-safe, but I don't
+        // know a better way to do this.
+        addInput((Pair<K1, V1>) inputPair);
+      } else {
+        throw new IllegalArgumentException("Could not parse input pair in addInput");
+      }
+    }
+  }
+
+  /**
+   * Identical to addInputFromString, but with a fluent programming style
+   * @param input A string of the form "key \t val". Trims any whitespace.
+   * @return this
+   */
+  public MapReduceDriver<K1, V1, K2, V2, K3, V3> withInputFromString(String input) {
+    addInputFromString(input);
+    return this;
+  }
+
+  /**
+   * Expects an input of the form "key \t val"
+   * Forces the Reducer output types to Text.
+   * @param output A string of the form "key \t val". Trims any whitespace.
+   */
+  public void addOutputFromString(String output) {
+    if (null == output) {
+      throw new IllegalArgumentException("null input given to setOutput");
+    } else {
+      Pair<Text, Text> outputPair = parseTabbedPair(output);
+      if (null != outputPair) {
+        // I know this is not type-safe,
+        // but I don't know a better way to do this.
+        addOutput((Pair<K3, V3>) outputPair);
+      } else {
+        throw new IllegalArgumentException(
+            "Could not parse output pair in setOutput");
+      }
+    }
+  }
+
+  /**
+   * Identical to addOutputFromString, but with a fluent programming style
+   * @param output A string of the form "key \t val". Trims any whitespace.
+   * @return this
+   */
+  public MapReduceDriver<K1, V1, K2, V2, K3, V3> withOutputFromString(String output) {
+    addOutputFromString(output);
+    return this;
+  }
+
+  public List<Pair<K3, V3>> run() throws IOException {
+
+    List<Pair<K2, V2>> mapOutputs = new ArrayList<Pair<K2, V2>>();
+
+    // run map component
+    for (Pair<K1, V1> input : inputList) {
+      LOG.debug("Mapping input " + input.toString() + ")");
+
+      mapOutputs.addAll(new MapDriver<K1, V1, K2, V2>(myMapper).withInput(
+              input).run());
+    }
+
+    List<Pair<K2, List<V2>>> reduceInputs = shuffle(mapOutputs);
+    List<Pair<K3, V3>> reduceOutputs = new ArrayList<Pair<K3, V3>>();
+
+    for (Pair<K2, List<V2>> input : reduceInputs) {
+      K2 inputKey = input.getFirst();
+      List<V2> inputValues = input.getSecond();
+      StringBuilder sb = new StringBuilder();
+      formatValueList(inputValues, sb);
+      LOG.debug("Reducing input (" + inputKey.toString() + ", "
+          + sb.toString() + ")");
+
+      reduceOutputs.addAll(new ReduceDriver<K2, V2, K3, V3>(myReducer)
+              .withInputKey(inputKey).withInputValues(inputValues).run());
+    }
+
+    return reduceOutputs;
+  }
+
+  @Override
+  public void runTest() throws RuntimeException {
+    List<Pair<K3, V3>> reduceOutputs = null;
+    boolean succeeded;
+
+    try {
+      reduceOutputs = run();
+      validate(reduceOutputs);
+    } catch (IOException ioe) {
+      LOG.error("IOException: " + ioe.toString());
+      LOG.debug("Setting success to false based on IOException");
+      throw new RuntimeException();
+    }
+  }
+
+  /** Take the outputs from the Mapper, combine all values for the
+   *  same key, and sort them by key.
+   * @param mapOutputs An unordered list of (key, val) pairs from the mapper
+   * @return the sorted list of (key, list(val))'s to present to the reducer
+   */
+  List<Pair<K2, List<V2>>> shuffle(List<Pair<K2, V2>> mapOutputs) {
+    HashMap<K2, List<V2>> reducerInputs = new HashMap<K2, List<V2>>();
+
+    // step 1: condense all values with the same key into a list.
+    for (Pair<K2, V2> mapOutput : mapOutputs) {
+      List<V2> valuesForKey = reducerInputs.get(mapOutput.getFirst());
+
+      if (null == valuesForKey) {
+        // this is the first (k, v) pair for this key. Add it to the list.
+        valuesForKey = new ArrayList<V2>();
+        valuesForKey.add(mapOutput.getSecond());
+        reducerInputs.put(mapOutput.getFirst(), valuesForKey);
+      } else {
+        // add this value to the existing list for this key
+        valuesForKey.add(mapOutput.getSecond());
+      }
+    }
+
+    // build a list out of these (k, list(v)) pairs
+    List<Pair<K2, List<V2>>> finalInputs = new ArrayList<Pair<K2, List<V2>>>();
+    Set<Map.Entry<K2, List<V2>>> entries = reducerInputs.entrySet();
+    for (Map.Entry<K2, List<V2>> entry : entries) {
+      K2 key = entry.getKey();
+      List<V2> vals = entry.getValue();
+      finalInputs.add(new Pair<K2, List<V2>>(key, vals));
+    }
+
+    // and then sort the output list by key
+    if (finalInputs.size() > 0) {
+      Collections.sort(finalInputs,
+              finalInputs.get(0).new FirstElemComparator());
+    }
+
+    return finalInputs;
+  }
+}

Added: hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/ReduceDriver.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/ReduceDriver.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/ReduceDriver.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/ReduceDriver.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,343 @@
+/**
+ * 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.mrunit;
+
+
+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.io.Text;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mrunit.mock.MockOutputCollector;
+import org.apache.hadoop.mrunit.mock.MockReporter;
+import org.apache.hadoop.mrunit.types.Pair;
+
+/**
+ * Harness that allows you to test a Reducer instance. You provide a key and a
+ * set of intermediate values for that key that represent inputs that should
+ * be sent to the Reducer (as if they came from a Mapper), and outputs you
+ * expect to be sent by the Reducer to the collector. By calling runTest(),
+ * the harness will deliver the input to the Reducer and will check its
+ * outputs against the expected results. This is designed to handle a single
+ * (k, v*) -> (k, v)* case from the Reducer, representing a single unit test.
+ * Multiple input (k, v*) sets should go in separate unit tests.
+ */
+public class ReduceDriver<K1, V1, K2, V2> extends TestDriver<K1, V1, K2, V2> {
+
+  public static final Log LOG = LogFactory.getLog(ReduceDriver.class);
+
+  private Reducer<K1, V1, K2, V2> myReducer;
+
+  private K1 inputKey;
+  private List<V1> inputValues;
+
+  public ReduceDriver(final Reducer<K1, V1, K2, V2> r) {
+    myReducer = r;
+    inputValues = new ArrayList<V1>();
+  }
+
+  public ReduceDriver() {
+    inputValues = new ArrayList<V1>();
+  }
+
+  /**
+   * Sets the reducer object to use for this test
+   *
+   * @param r
+   *          The reducer object to use
+   */
+  public void setReducer(Reducer<K1, V1, K2, V2> r) {
+    myReducer = r;
+  }
+
+  /**
+   * Identical to setReducer(), but with fluent programming style
+   *
+   * @param r
+   *          The Reducer to use
+   * @return this
+   */
+  public ReduceDriver<K1, V1, K2, V2> withReducer(Reducer<K1, V1, K2, V2> r) {
+    setReducer(r);
+    return this;
+  }
+
+  public Reducer<K1, V1, K2, V2> getReducer() {
+    return myReducer;
+  }
+
+  /**
+   * Sets the input key to send to the Reducer
+   *
+   */
+  public void setInputKey(K1 key) {
+    inputKey = key;
+  }
+
+  /**
+   * Identical to setInputKey() but with fluent programming style
+   *
+   * @return this
+   */
+  public ReduceDriver<K1, V1, K2, V2> withInputKey(K1 key) {
+    setInputKey(key);
+    return this;
+  }
+
+  /**
+   * adds an input value to send to the reducer
+   *
+   * @param val
+   */
+  public void addInputValue(V1 val) {
+    inputValues.add(val);
+  }
+
+  /**
+   * Identical to addInputValue() but with fluent programming style
+   *
+   * @param val
+   * @return this
+   */
+  public ReduceDriver<K1, V1, K2, V2> withInputValue(V1 val) {
+    addInputValue(val);
+    return this;
+  }
+
+  /**
+   * Sets the input values to send to the reducer; overwrites existing ones
+   *
+   * @param values
+   */
+  public void setInputValues(List<V1> values) {
+    inputValues.clear();
+    inputValues.addAll(values);
+  }
+
+  /**
+   * Adds a set of input values to send to the reducer
+   *
+   * @param values
+   */
+  public void addInputValues(List<V1> values) {
+    inputValues.addAll(values);
+  }
+
+  /**
+   * Identical to addInputValues() but with fluent programming style
+   *
+   * @param values
+   * @return this
+   */
+  public ReduceDriver<K1, V1, K2, V2> withInputValues(List<V1> values) {
+    addInputValues(values);
+    return this;
+  }
+
+  /**
+   * Sets the input to send to the reducer
+   *
+   * @param values
+   */
+  public void setInput(K1 key, List<V1> values) {
+    setInputKey(key);
+    setInputValues(values);
+  }
+
+  /**
+   * Identical to setInput() but returns self for fluent programming style
+   *
+   * @return this
+   */
+  public ReduceDriver<K1, V1, K2, V2> withInput(K1 key, List<V1> values) {
+    setInput(key, values);
+    return this;
+  }
+
+  /**
+   * Adds an output (k, v) pair we expect from the Reducer
+   *
+   * @param outputRecord
+   *          The (k, v) pair to add
+   */
+  public void addOutput(Pair<K2, V2> outputRecord) {
+    if (null != outputRecord) {
+      expectedOutputs.add(outputRecord);
+    } else {
+      throw new IllegalArgumentException("Tried to add null outputRecord");
+    }
+  }
+
+  /**
+   * Works like addOutput(), but returns self for fluent style
+   *
+   * @param outputRecord
+   * @return this
+   */
+  public ReduceDriver<K1, V1, K2, V2> withOutput(Pair<K2, V2> outputRecord) {
+    addOutput(outputRecord);
+    return this;
+  }
+
+  /**
+   * Adds an output (k, v) pair we expect from the Reducer
+   *
+   * @param key The key part of a (k, v) pair to add
+   * @param val The val part of a (k, v) pair to add
+   */
+  public void addOutput(K2 key, V2 val) {
+    addOutput(new Pair<K2, V2>(key, val));
+  }
+
+  /**
+   * Works like addOutput(), but returns self for fluent style
+   *
+   * @param key The key part of a (k, v) pair to add
+   * @param val The val part of a (k, v) pair to add
+   * @return this
+   */
+  public ReduceDriver<K1, V1, K2, V2> withOutput(K2 key, V2 val) {
+    addOutput(key, val);
+    return this;
+  }
+
+  /**
+   * Expects an input of the form "key \t val, val, val..." Forces the Reducer
+   * input types to Text.
+   *
+   * @param input
+   *          A string of the form "key \t val,val,val". Trims any whitespace.
+   */
+  public void setInputFromString(String input) {
+    if (null == input) {
+      throw new IllegalArgumentException("null input given to setInputFromString");
+    } else {
+      Pair<Text, Text> inputPair = parseTabbedPair(input);
+      if (null != inputPair) {
+        // I know this is not type-safe, but I don't know a better way to do
+        // this.
+        setInputKey((K1) inputPair.getFirst());
+        setInputValues((List<V1>) parseCommaDelimitedList(inputPair.getSecond()
+                .toString()));
+      } else {
+        throw new IllegalArgumentException(
+            "Could not parse input pair in setInputFromString");
+      }
+    }
+  }
+
+  /**
+   * Identical to setInput, but with a fluent programming style
+   *
+   * @param input
+   *          A string of the form "key \t val". Trims any whitespace.
+   * @return this
+   */
+  public ReduceDriver<K1, V1, K2, V2> withInputFromString(String input) {
+    setInputFromString(input);
+    return this;
+  }
+
+  /**
+   * Expects an input of the form "key \t val" Forces the Reducer output types
+   * to Text.
+   *
+   * @param output
+   *          A string of the form "key \t val". Trims any whitespace.
+   */
+  public void addOutputFromString(String output) {
+    if (null == output) {
+      throw new IllegalArgumentException("null input given to setOutput");
+    } else {
+      Pair<Text, Text> outputPair = parseTabbedPair(output);
+      if (null != outputPair) {
+        // I know this is not type-safe, but I don't know a better way to do
+        // this.
+        addOutput((Pair<K2, V2>) outputPair);
+      } else {
+        throw new IllegalArgumentException("Could not parse output pair in setOutput");
+      }
+    }
+  }
+
+  /**
+   * Identical to addOutput, but with a fluent programming style
+   *
+   * @param output
+   *          A string of the form "key \t val". Trims any whitespace.
+   * @return this
+   */
+  public ReduceDriver<K1, V1, K2, V2> withOutputFromString(String output) {
+    addOutputFromString(output);
+    return this;
+  }
+
+  @Override
+  public List<Pair<K2, V2>> run() throws IOException {
+
+    MockOutputCollector<K2, V2> outputCollector =
+      new MockOutputCollector<K2, V2>();
+    MockReporter reporter = new MockReporter(MockReporter.ReporterType.Reducer);
+
+    myReducer.reduce(inputKey, inputValues.iterator(), outputCollector,
+            reporter);
+
+    List<Pair<K2, V2>> outputs = outputCollector.getOutputs();
+    return outputs;
+  }
+
+  @Override
+  public void runTest() throws RuntimeException {
+
+    String inputKeyStr = "(null)";
+
+    if (null != inputKey) {
+      inputKeyStr = inputKey.toString();
+    }
+
+    StringBuilder sb = new StringBuilder();
+    formatValueList(inputValues, sb);
+
+    LOG.debug("Reducing input (" + inputKeyStr + ", " + sb.toString() + ")");
+
+    List<Pair<K2, V2>> outputs = null;
+    try {
+      outputs = run();
+      validate(outputs);
+    } catch (IOException ioe) {
+      LOG.error("IOException in reducer: " + ioe.toString());
+      LOG.debug("Setting success to false based on IOException");
+      throw new RuntimeException();
+    }
+  }
+
+  @Override
+  public String toString() {
+    String reducerStr = "null";
+
+    if (null != myReducer) {
+      reducerStr = myReducer.toString();
+    }
+
+    return "ReduceDriver (" + reducerStr + ")";
+  }
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/TestDriver.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/TestDriver.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/TestDriver.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/TestDriver.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,245 @@
+/**
+ * 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.mrunit;
+
+
+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.io.Text;
+import org.apache.hadoop.mrunit.types.Pair;
+
+public abstract class TestDriver<K1, V1, K2, V2> {
+
+  public static final Log LOG = LogFactory.getLog(TestDriver.class);
+
+  protected List<Pair<K2, V2>> expectedOutputs;
+
+  public TestDriver() {
+    expectedOutputs = new ArrayList<Pair<K2, V2>>();
+  }
+
+  /**
+   * @return the list of (k, v) pairs expected as output from this driver
+   */
+  public List<Pair<K2, V2>> getExpectedOutputs() {
+    return expectedOutputs;
+  }
+
+  /**
+   * Clears the list of outputs expected from this driver
+   */
+  public void resetOutput() {
+    expectedOutputs.clear();
+  }
+
+  /**
+   * Runs the test but returns the result set instead of validating it
+   * (ignores any addOutput(), etc calls made before this)
+   * @return the list of (k, v) pairs returned as output from the test
+   */
+  public abstract List<Pair<K2, V2>> run() throws IOException;
+
+  /**
+   * Runs the test and validates the results
+   * @return void if the tests passed
+   * @throws RuntimeException if they don't
+   * *
+   */
+  public abstract void runTest() throws RuntimeException;
+
+  /**
+   * Split "key \t val" into Pair(Text(key), Text(val))
+   * @param tabSeparatedPair
+   */
+  static Pair<Text, Text> parseTabbedPair(String tabSeparatedPair) {
+
+    String key, val;
+
+    if (null == tabSeparatedPair) {
+      return null;
+    }
+
+    int split = tabSeparatedPair.indexOf('\t');
+    if (-1 == split) {
+      return null;
+    }
+
+    key = tabSeparatedPair.substring(0, split);
+    val = tabSeparatedPair.substring(split + 1);
+
+    return new Pair<Text, Text>(new Text(key), new Text(val));
+  }
+
+  /**
+   * Split "val,val,val,val..." into a List of Text(val) objects.
+   * @param commaDelimList A list of values separated by commas
+   */
+  protected static List<Text> parseCommaDelimitedList(String commaDelimList) {
+    ArrayList<Text> outList = new ArrayList<Text>();
+
+    if (null == commaDelimList) {
+      return null;
+    }
+
+    int len = commaDelimList.length();
+    int curPos = 0;
+    int curComma = commaDelimList.indexOf(',');
+    if (curComma == -1) {
+      curComma = len;
+    }
+
+    while (curPos < len) {
+      outList.add(new Text(
+              commaDelimList.substring(curPos, curComma).trim()));
+      curPos = curComma + 1;
+      curComma = commaDelimList.indexOf(',', curPos);
+      if (curComma == -1) {
+        curComma = len;
+      }
+    }
+
+    return outList;
+  }
+
+  /**
+   * check the outputs against the expected inputs in record
+   * @param outputs The actual output (k, v) pairs from the Mapper
+   * @return void if they all pass
+   * @throws RuntimeException if they don't
+   */
+  protected void validate(List<Pair<K2, V2>> outputs) throws RuntimeException {
+
+    boolean success = true;
+
+    // were we supposed to get output in the first place?
+    // return false if we don't.
+    if (expectedOutputs.size() == 0 && outputs.size() > 0) {
+      LOG.error("Expected no outputs; got " + outputs.size() + " outputs.");
+      success = false;
+    }
+
+    // make sure all actual outputs are in the expected set,
+    // and at the proper position.
+    for (int i = 0; i < outputs.size(); i++) {
+      Pair<K2, V2> actual = outputs.get(i);
+      success = lookupExpectedValue(actual, i) && success;
+    }
+
+    // make sure all expected outputs were accounted for.
+    if (expectedOutputs.size() != outputs.size() || !success) {
+      // something is unaccounted for. Figure out what.
+
+      ArrayList<Pair<K2, V2>> actuals = new ArrayList<Pair<K2, V2>>();
+      actuals.addAll(outputs);
+
+      for (int i = 0; i < expectedOutputs.size(); i++) {
+        Pair<K2, V2> expected = expectedOutputs.get(i);
+
+        boolean found = false;
+        for (int j = 0; j < actuals.size() && !found; j++) {
+          Pair<K2, V2> actual = actuals.get(j);
+
+          if (actual.equals(expected)) {
+            // don't match against this actual output again
+            actuals.remove(j);
+            found = true;
+          }
+        }
+
+        if (!found) {
+          String expectedStr = "(null)";
+          if (null != expected) {
+            expectedStr = expected.toString();
+          }
+
+          LOG.error("Missing expected output " + expectedStr + " at position "
+              + i);
+        }
+      }
+
+      success = false;
+    }
+
+    if (!success) {
+      throw new RuntimeException();
+    }
+  }
+
+  /**
+   * Part of the validation system.
+   * @param actualVal A (k, v) pair we got from the Mapper
+   * @param actualPos The position of this pair in the actual output
+   * @return true if the expected val at 'actualPos' in the expected
+   *              list equals actualVal
+   */
+  private boolean lookupExpectedValue(Pair<K2, V2> actualVal, int actualPos) {
+
+    // first: Do we have the success condition?
+    if (expectedOutputs.size() > actualPos
+            && expectedOutputs.get(actualPos).equals(actualVal)) {
+      LOG.debug("Matched expected output " + actualVal.toString()
+          + " at position " + actualPos);
+      return true;
+    }
+
+    // second: can we find this output somewhere else in
+    // the expected list?
+    boolean foundSomewhere = false;
+
+    for (int i = 0; i < expectedOutputs.size() && !foundSomewhere; i++) {
+      Pair<K2, V2> expected = expectedOutputs.get(i);
+
+      if (expected.equals(actualVal)) {
+        LOG.error("Matched expected output "
+                + actualVal.toString() + " but at incorrect position "
+                + actualPos + " (expected position " + i + ")");
+        foundSomewhere = true;
+      }
+    }
+
+    if (!foundSomewhere) {
+      LOG.error("Received unexpected output " + actualVal.toString());
+    }
+
+    return false;
+  }
+
+  protected static void formatValueList(List values, StringBuilder sb) {
+    sb.append("(");
+
+    if (null != values) {
+      boolean first = true;
+
+      for (Object val : values) {
+        if (!first) {
+          sb.append(", ");
+        }
+
+        first = false;
+        sb.append(val.toString());
+      }
+    }
+
+    sb.append(")");
+  }
+
+}

Added: hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockInputSplit.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockInputSplit.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockInputSplit.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockInputSplit.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mrunit.mock;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+
+/**
+ * A fake InputSplit object to be returned by our MockReporter to the user
+ * class.
+ */
+public class MockInputSplit extends FileSplit implements InputSplit {
+
+  private static final Path MOCK_PATH = new Path("somefile");
+
+  public MockInputSplit() {
+    super(MOCK_PATH, 0, 0, (String []) null);
+  }
+
+  @Override
+  public long getLength() {
+    return 0;
+  }
+
+  @Override
+  public String[] getLocations() throws IOException {
+    return null;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+  }
+
+  public String toString() {
+    return "MockInputSplit";
+  }
+
+
+  /**
+   * Return the path object represented by this as a FileSplit.
+   */
+  public static Path getMockPath() {
+    return MOCK_PATH;
+  }
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockOutputCollector.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockOutputCollector.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockOutputCollector.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockOutputCollector.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,66 @@
+/**
+ * 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.mrunit.mock;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mrunit.types.Pair;
+
+/**
+ * OutputCollector to use in the test framework for Mapper and Reducer
+ * classes. Accepts a set of output (k, v) pairs and returns them to the
+ * framework for validation.
+ *
+ * BUG: Currently, this does not make deep copies of values passed to collect().
+ * So emitting the same Text object (for instance) repeatedly, with different
+ * internal string data each time, is not tested in the same way that Hadoop's
+ * OutputCollector works.
+ *
+ */
+public class MockOutputCollector<K, V> implements OutputCollector<K, V> {
+
+  private ArrayList<Pair<K, V>> collectedOutputs;
+
+  public MockOutputCollector() {
+    collectedOutputs = new ArrayList<Pair<K, V>>();
+  }
+
+  /**
+   * Accepts another (key, value) pair as an output of this mapper/reducer.
+   *
+   * BUG: Currently, this does not make deep copies of values passed to collect().
+   * So emitting the same Text object (for instance) repeatedly, with different
+   * internal string data each time, is not tested in the same way that Hadoop's
+   * OutputCollector works.
+   */
+  public void collect(K key, V value) throws IOException {
+    collectedOutputs.add(new Pair<K, V>(key, value));
+  }
+
+  /**
+   * @return The outputs generated by the mapper/reducer being tested
+   */
+  public List<Pair<K, V>> getOutputs() {
+    return collectedOutputs;
+  }
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockReporter.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockReporter.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockReporter.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mock/MockReporter.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,81 @@
+/**
+ * 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.mrunit.mock;
+
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.Counters.Counter;
+
+public class MockReporter implements Reporter {
+
+  private MockInputSplit inputSplit = new MockInputSplit();
+
+  public enum ReporterType {
+    Mapper,
+    Reducer
+  }
+
+  private ReporterType typ;
+
+  public MockReporter(final ReporterType kind) {
+    this.typ = kind;
+  }
+
+  @Override
+  public InputSplit getInputSplit() {
+    if (typ == ReporterType.Reducer) {
+      throw new UnsupportedOperationException(
+              "Reducer cannot call getInputSplit()");
+    } else {
+      return inputSplit;
+    }
+  }
+
+  @Override
+  public void incrCounter(Enum key, long amount) {
+    // do nothing.
+  }
+
+  @Override
+  public void incrCounter(String group, String counter, long amount) {
+    // do nothing.
+  }
+
+  @Override
+  public void setStatus(String status) {
+    // do nothing.
+  }
+
+  @Override
+  public void progress() {
+    // do nothing.
+  }
+
+  @Override
+  public Counter getCounter(String s1, String s2) {
+    // do nothing
+    return null;
+  }
+
+  @Override
+  public Counter getCounter(Enum key) {
+    // do nothing
+    return null;
+  }
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/testutil/ExtendedAssert.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/testutil/ExtendedAssert.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/testutil/ExtendedAssert.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/testutil/ExtendedAssert.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,73 @@
+/**
+ * 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.mrunit.testutil;
+
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+public final class ExtendedAssert {
+
+  private ExtendedAssert() { }
+
+  /**
+   * Asserts that all the elements of the list are equivalent under equals()
+   * @param expected a list full of expected values
+   * @param actual a list full of actual test values
+   */
+  public static void assertListEquals(List expected, List actual) {
+    if (expected.size() != actual.size()) {
+      fail("Expected list of size " + expected.size() + "; actual size is "
+          + actual.size());
+    }
+
+    for (int i = 0; i < expected.size(); i++) {
+      Object t1 = expected.get(i);
+      Object t2 = actual.get(i);
+
+      if (!t1.equals(t2)) {
+        fail("Expected element " + t1 + " at index " + i
+            + " != actual element " + t2);
+      }
+    }
+  }
+
+  /**
+   * asserts x &gt; y
+   */
+  public static void assertGreater(int x, int y) {
+    assertTrue("Expected " + x + " > " + y, x > y);
+  }
+
+  /** asserts x &gt;= y) */
+  public static void assertGreaterOrEqual(int x, int y) {
+    assertTrue("Expected " + x + " >= " + y, x >= y);
+  }
+
+  /**
+   * asserts x &lt; y
+   */
+  public static void assertLess(int x, int y) {
+    assertTrue("Expected " + x + " < " + y, x < y);
+  }
+
+  /** asserts x &gt;= y) */
+  public static void assertLessOrEqual(int x, int y) {
+    assertTrue("Expected " + x + " <= " + y, x <= y);
+  }
+}

Added: hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/types/Pair.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/types/Pair.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/types/Pair.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/types/Pair.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,127 @@
+/**
+ * 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.mrunit.types;
+
+import java.util.Comparator;
+
+
+/**
+ * A very basic pair type.
+ */
+public class Pair<S, T> implements Comparable<Pair<S, T>> {
+
+  private final S first;
+  private final T second;
+
+  public Pair(final S car, final T cdr) {
+    first = car;
+    second = cdr;
+  }
+
+  public S getFirst() { return first; }
+  public T getSecond() { return second; }
+
+  @Override
+  public boolean equals(Object o) {
+    if (null == o) {
+      return false;
+    } else if (o instanceof Pair) {
+      Pair<S, T> p = (Pair<S, T>) o;
+      if (first == null && second == null) {
+        return p.first == null && p.second == null;
+      } else if (first == null) {
+        return p.first == null && second.equals(p.second);
+      } else if (second == null) {
+        return p.second == null && first.equals(p.first);
+      } else {
+        return first.equals(p.first) && second.equals(p.second);
+      }
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    int code = 0;
+
+    if (null != first) {
+      code += first.hashCode();
+    }
+
+    if (null != second) {
+      code += second.hashCode() << 1;
+    }
+
+    return code;
+  }
+
+  @Override
+  public int compareTo(Pair<S, T> p) {
+    if (null == p) {
+      return 1;
+    }
+
+    Comparable<S> firstCompare = (Comparable<S>) first;
+
+    int firstResult = firstCompare.compareTo(p.first);
+    if (firstResult == 0) {
+      Comparable<T> secondCompare = (Comparable<T>) second;
+      return secondCompare.compareTo(p.second);
+    } else {
+      return firstResult;
+    }
+  }
+
+  // TODO: Can this be made static? Same with SecondElemComparator?
+  public class FirstElemComparator implements Comparator<Pair<S, T>> {
+    public FirstElemComparator() {
+    }
+
+    public int compare(Pair<S, T> p1, Pair<S, T> p2) {
+      Comparable<S> cS = (Comparable<S>) p1.first;
+      return cS.compareTo(p2.first);
+    }
+  }
+
+  public class SecondElemComparator implements Comparator<Pair<S, T>> {
+    public SecondElemComparator() {
+    }
+
+    public int compare(Pair<S, T> p1, Pair<S, T> p2) {
+      Comparable<T> cT = (Comparable<T>) p1.second;
+      return cT.compareTo(p2.second);
+    }
+  }
+
+  @Override
+  public String toString() {
+    String firstString = "null";
+    String secondString = "null";
+
+    if (null != first) {
+      firstString = first.toString();
+    }
+
+    if (null != second) {
+      secondString = second.toString();
+    }
+
+    return "(" + firstString + ", " + secondString + ")";
+  }
+}

Added: hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/AllTests.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/AllTests.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/AllTests.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/AllTests.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,48 @@
+/**
+ * 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.mrunit;
+
+import org.apache.hadoop.mrunit.mock.TestMockReporter;
+
+import junit.framework.Test;
+import junit.framework.TestSuite;
+
+/**
+ * All tests for MRUnit testing device (org.apache.hadoop.mrunit)
+ *
+ */
+public final class AllTests  {
+
+  private AllTests() { }
+
+  public static Test suite() {
+    TestSuite suite = new TestSuite("Test for org.apache.hadoop.mrunit");
+
+    suite.addTestSuite(TestMapDriver.class);
+    suite.addTestSuite(TestMapReduceDriver.class);
+    suite.addTestSuite(TestMockReporter.class);
+    suite.addTestSuite(TestReduceDriver.class);
+    suite.addTestSuite(TestTestDriver.class);
+    suite.addTestSuite(TestExample.class);
+
+    suite.addTest(org.apache.hadoop.mrunit.types.AllTests.suite());
+    return suite;
+  }
+
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestExample.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestExample.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestExample.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestExample.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mrunit;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Example test of the IdentityMapper to demonstrate proper MapDriver
+ * usage in a test case.
+ *
+ * This example is reproduced in the overview for the MRUnit javadoc.
+ */
+public class TestExample extends TestCase {
+
+  private Mapper<Text, Text, Text, Text> mapper;
+  private MapDriver<Text, Text, Text, Text> driver;
+
+  @Before
+  public void setUp() {
+    mapper = new IdentityMapper<Text, Text>();
+    driver = new MapDriver<Text, Text, Text, Text>(mapper);
+  }
+
+  @Test
+  public void testIdentityMapper() {
+    driver.withInput(new Text("foo"), new Text("bar"))
+            .withOutput(new Text("foo"), new Text("bar"))
+            .runTest();
+  }
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapDriver.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapDriver.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapDriver.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapDriver.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,186 @@
+/**
+ * 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.mrunit;
+
+import static org.apache.hadoop.mrunit.testutil.ExtendedAssert.*;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestMapDriver extends TestCase {
+
+  private Mapper<Text, Text, Text, Text> mapper;
+  private MapDriver<Text, Text, Text, Text> driver;
+
+  @Before
+  public void setUp() {
+    mapper = new IdentityMapper<Text, Text>();
+    driver = new MapDriver<Text, Text, Text, Text>(mapper);
+  }
+
+  @Test
+  public void testRun() {
+    List<Pair<Text, Text>> out = null;
+
+    try {
+      out = driver.withInput(new Text("foo"), new Text("bar")).run();
+    } catch (IOException ioe) {
+      fail();
+    }
+
+    List<Pair<Text, Text>> expected = new ArrayList<Pair<Text, Text>>();
+    expected.add(new Pair<Text, Text>(new Text("foo"), new Text("bar")));
+
+    assertListEquals(out, expected);
+  }
+
+  @Test
+  public void TesttestRun1() {
+    driver.withInput(new Text("foo"), new Text("bar"))
+            .withOutput(new Text("foo"), new Text("bar"))
+            .runTest();
+  }
+
+  @Test
+  public void TesttestRun2() {
+    try {
+      driver.withInput(new Text("foo"), new Text("bar"))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void TesttestRun3() {
+    try {
+      driver.withInput(new Text("foo"), new Text("bar"))
+            .withOutput(new Text("foo"), new Text("bar"))
+            .withOutput(new Text("foo"), new Text("bar"))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void TesttestRun4() {
+    try {
+      driver.withInput(new Text("foo"), new Text("bar"))
+            .withOutput(new Text("foo"), new Text("bar"))
+            .withOutput(new Text("bonusfoo"), new Text("bar"))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+
+  }
+  @Test
+  public void TesttestRun5() {
+    try {
+      driver.withInput(new Text("foo"), new Text("bar"))
+            .withOutput(new Text("foo"), new Text("somethingelse"))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void TesttestRun6() {
+    try {
+      driver.withInput(new Text("foo"), new Text("bar"))
+              .withOutput(new Text("someotherkey"), new Text("bar"))
+              .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void TesttestRun7() {
+    try {
+      driver.withInput(new Text("foo"), new Text("bar"))
+            .withOutput(new Text("someotherkey"), new Text("bar"))
+            .withOutput(new Text("foo"), new Text("bar"))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void testSetInput() {
+    try {
+      driver.setInput(new Pair<Text, Text>(new Text("foo"), new Text("bar")));
+    } catch (Exception e) {
+      fail();
+    }
+
+    assertEquals(driver.getInputKey(), new Text("foo"));
+    assertEquals(driver.getInputValue(), new Text("bar"));
+  }
+
+  @Test
+  public void testSetInputNull() {
+    try {
+      driver.setInput((Pair<Text, Text>) null);
+      fail();
+    } catch (Exception e) {
+      // expect this.
+    }
+  }
+
+  @Test
+  public void testEmptyInput() {
+    // MapDriver will forcibly map (null, null) as undefined input;
+    // identity mapper expects (null, null) back.
+    driver.withOutput(null, null).runTest();
+  }
+
+  @Test
+  public void testEmptyInput2() {
+    // it is an error to expect no output because we expect
+    // the mapper to be fed (null, null) as an input if the
+    // user doesn't set any input.
+    try {
+      driver.runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+}
+



Mime
View raw message