You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2009/04/06 23:23:27 UTC

svn commit: r762509 [2/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...

Added: hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapReduceDriver.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapReduceDriver.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapReduceDriver.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestMapReduceDriver.java Mon Apr  6 21:23:26 2009
@@ -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.mrunit;
+
+import static org.apache.hadoop.mrunit.testutil.ExtendedAssert.assertListEquals;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mapred.lib.LongSumReducer;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestMapReduceDriver extends TestCase {
+
+  private static final int FOO_IN_A = 42;
+  private static final int FOO_IN_B = 10;
+  private static final int BAR_IN = 12;
+  private static final int FOO_OUT = 52;
+
+  private Mapper<Text, LongWritable, Text, LongWritable> mapper;
+  private Reducer<Text, LongWritable, Text, LongWritable> reducer;
+  private MapReduceDriver<Text, LongWritable,
+                  Text, LongWritable,
+                  Text, LongWritable> driver;
+
+  private MapReduceDriver<Text, Text, Text, Text, Text, Text> driver2;
+
+  @Before
+  public void setUp() throws Exception {
+    mapper = new IdentityMapper<Text, LongWritable>();
+    reducer = new LongSumReducer<Text>();
+    driver = new MapReduceDriver<Text, LongWritable,
+                                 Text, LongWritable,
+                                 Text, LongWritable>(
+                        mapper, reducer);
+    // for shuffle tests
+    driver2 = new MapReduceDriver<Text, Text, Text, Text, Text, Text>();
+  }
+
+  @Test
+  public void testRun() {
+    List<Pair<Text, LongWritable>> out = null;
+    try {
+      out = driver
+              .withInput(new Text("foo"), new LongWritable(FOO_IN_A))
+              .withInput(new Text("foo"), new LongWritable(FOO_IN_B))
+              .withInput(new Text("bar"), new LongWritable(BAR_IN))
+              .run();
+    } catch (IOException ioe) {
+      fail();
+    }
+
+    List<Pair<Text, LongWritable>> expected =
+      new ArrayList<Pair<Text, LongWritable>>();
+    expected.add(new Pair<Text, LongWritable>(new Text("bar"),
+            new LongWritable(BAR_IN)));
+    expected.add(new Pair<Text, LongWritable>(new Text("foo"),
+            new LongWritable(FOO_OUT)));
+
+    assertListEquals(out, expected);
+  }
+
+  @Test
+  public void TesttestRun1() {
+    driver
+            .withInput(new Text("foo"), new LongWritable(FOO_IN_A))
+            .withInput(new Text("foo"), new LongWritable(FOO_IN_B))
+            .withInput(new Text("bar"), new LongWritable(BAR_IN))
+            .withOutput(new Text("bar"), new LongWritable(BAR_IN))
+            .withOutput(new Text("foo"), new LongWritable(FOO_OUT))
+            .runTest();
+  }
+
+  @Test
+  public void TesttestRun2() {
+    driver
+            .withInput(new Text("foo"), new LongWritable(FOO_IN_A))
+            .withInput(new Text("bar"), new LongWritable(BAR_IN))
+            .withInput(new Text("foo"), new LongWritable(FOO_IN_B))
+            .withOutput(new Text("bar"), new LongWritable(BAR_IN))
+            .withOutput(new Text("foo"), new LongWritable(FOO_OUT))
+            .runTest();
+  }
+
+  @Test
+  public void TesttestRun3() {
+    try {
+      driver
+            .withInput(new Text("foo"), new LongWritable(FOO_IN_A))
+            .withInput(new Text("bar"), new LongWritable(BAR_IN))
+            .withInput(new Text("foo"), new LongWritable(FOO_IN_B))
+            .withOutput(new Text("foo"), new LongWritable(FOO_OUT))
+            .withOutput(new Text("bar"), new LongWritable(BAR_IN))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testEmptyInput() {
+    driver.runTest();
+  }
+
+  @Test
+  public void testEmptyInputWithOutputFails() {
+    try {
+      driver
+              .withOutput(new Text("foo"), new LongWritable(FOO_OUT))
+              .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void testEmptyShuffle() {
+    List<Pair<Text, Text>> inputs = new ArrayList<Pair<Text, Text>>();
+    List<Pair<Text, List<Text>>> outputs = driver2.shuffle(inputs);
+    assertEquals(0, outputs.size());
+  }
+
+  // just shuffle a single (k, v) pair
+  @Test
+  public void testSingleShuffle() {
+    List<Pair<Text, Text>> inputs = new ArrayList<Pair<Text, Text>>();
+    inputs.add(new Pair<Text, Text>(new Text("a"), new Text("b")));
+
+    List<Pair<Text, List<Text>>> outputs = driver2.shuffle(inputs);
+
+    List<Pair<Text, List<Text>>> expected = new ArrayList<Pair<Text, List<Text>>>();
+    List<Text> sublist = new ArrayList<Text>();
+    sublist.add(new Text("b"));
+    expected.add(new Pair<Text, List<Text>>(new Text("a"), sublist));
+
+    assertListEquals(expected, outputs);
+  }
+
+  // shuffle multiple values from the same key.
+  @Test
+  public void testShuffleOneKey() {
+    List<Pair<Text, Text>> inputs = new ArrayList<Pair<Text, Text>>();
+    inputs.add(new Pair<Text, Text>(new Text("a"), new Text("b")));
+    inputs.add(new Pair<Text, Text>(new Text("a"), new Text("c")));
+
+    List<Pair<Text, List<Text>>> outputs = driver2.shuffle(inputs);
+
+    List<Pair<Text, List<Text>>> expected = new ArrayList<Pair<Text, List<Text>>>();
+    List<Text> sublist = new ArrayList<Text>();
+    sublist.add(new Text("b"));
+    sublist.add(new Text("c"));
+    expected.add(new Pair<Text, List<Text>>(new Text("a"), sublist));
+
+    assertListEquals(expected, outputs);
+  }
+
+  // shuffle multiple keys
+  @Test
+  public void testMultiShuffle1() {
+    List<Pair<Text, Text>> inputs = new ArrayList<Pair<Text, Text>>();
+    inputs.add(new Pair<Text, Text>(new Text("a"), new Text("x")));
+    inputs.add(new Pair<Text, Text>(new Text("b"), new Text("z")));
+    inputs.add(new Pair<Text, Text>(new Text("b"), new Text("w")));
+    inputs.add(new Pair<Text, Text>(new Text("a"), new Text("y")));
+
+    List<Pair<Text, List<Text>>> outputs = driver2.shuffle(inputs);
+
+    List<Pair<Text, List<Text>>> expected = new ArrayList<Pair<Text, List<Text>>>();
+    List<Text> sublist1 = new ArrayList<Text>();
+    sublist1.add(new Text("x"));
+    sublist1.add(new Text("y"));
+    expected.add(new Pair<Text, List<Text>>(new Text("a"), sublist1));
+
+    List<Text> sublist2 = new ArrayList<Text>();
+    sublist2.add(new Text("z"));
+    sublist2.add(new Text("w"));
+    expected.add(new Pair<Text, List<Text>>(new Text("b"), sublist2));
+
+    assertListEquals(expected, outputs);
+  }
+
+
+  // shuffle multiple keys that are out-of-order to start.
+  @Test
+  public void testMultiShuffle2() {
+    List<Pair<Text, Text>> inputs = new ArrayList<Pair<Text, Text>>();
+    inputs.add(new Pair<Text, Text>(new Text("b"), new Text("z")));
+    inputs.add(new Pair<Text, Text>(new Text("a"), new Text("x")));
+    inputs.add(new Pair<Text, Text>(new Text("b"), new Text("w")));
+    inputs.add(new Pair<Text, Text>(new Text("a"), new Text("y")));
+
+    List<Pair<Text, List<Text>>> outputs = driver2.shuffle(inputs);
+
+    List<Pair<Text, List<Text>>> expected = new ArrayList<Pair<Text, List<Text>>>();
+    List<Text> sublist1 = new ArrayList<Text>();
+    sublist1.add(new Text("x"));
+    sublist1.add(new Text("y"));
+    expected.add(new Pair<Text, List<Text>>(new Text("a"), sublist1));
+
+    List<Text> sublist2 = new ArrayList<Text>();
+    sublist2.add(new Text("z"));
+    sublist2.add(new Text("w"));
+    expected.add(new Pair<Text, List<Text>>(new Text("b"), sublist2));
+
+    assertListEquals(expected, outputs);
+  }
+
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestReduceDriver.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestReduceDriver.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestReduceDriver.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestReduceDriver.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,226 @@
+/**
+ * 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.assertListEquals;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.lib.LongSumReducer;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestReduceDriver extends TestCase {
+
+  private static final int IN_A = 4;
+  private static final int IN_B = 6;
+  private static final int OUT_VAL = 10;
+  private static final int INCORRECT_OUT = 12;
+  private static final int OUT_EMPTY = 0;
+
+  private Reducer<Text, LongWritable, Text, LongWritable> reducer;
+  private ReduceDriver<Text, LongWritable, Text, LongWritable> driver;
+
+  @Before
+  public void setUp() throws Exception {
+    reducer = new LongSumReducer<Text>();
+    driver = new ReduceDriver<Text, LongWritable, Text, LongWritable>(
+                   reducer);
+  }
+
+  @Test
+  public void testRun() {
+    List<Pair<Text, LongWritable>> out = null;
+
+    try {
+      out = driver.withInputKey(new Text("foo"))
+                  .withInputValue(new LongWritable(IN_A))
+                  .withInputValue(new LongWritable(IN_B))
+                  .run();
+    } catch (IOException ioe) {
+      fail();
+    }
+
+    List<Pair<Text, LongWritable>> expected =
+        new ArrayList<Pair<Text, LongWritable>>();
+    expected.add(new Pair<Text, LongWritable>(new Text("foo"),
+            new LongWritable(OUT_VAL)));
+
+    assertListEquals(out, expected);
+
+  }
+
+  @Test
+  public void TesttestRun1() {
+    driver
+            .withInputKey(new Text("foo"))
+            .withOutput(new Text("foo"), new LongWritable(0))
+            .runTest();
+  }
+
+  @Test
+  public void TesttestRun2() {
+    driver
+            .withInputKey(new Text("foo"))
+            .withInputValue(new LongWritable(IN_A))
+            .withInputValue(new LongWritable(IN_B))
+            .withOutput(new Text("foo"), new LongWritable(OUT_VAL))
+            .runTest();
+  }
+
+  @Test
+  public void TesttestRun3() {
+    try {
+      driver
+            .withInputKey(new Text("foo"))
+            .withInputValue(new LongWritable(IN_A))
+            .withInputValue(new LongWritable(IN_B))
+            .withOutput(new Text("bar"), new LongWritable(OUT_VAL))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+
+  }
+
+  @Test
+  public void TesttestRun4() {
+    try {
+      driver
+            .withInputKey(new Text("foo"))
+            .withInputValue(new LongWritable(IN_A))
+            .withInputValue(new LongWritable(IN_B))
+            .withOutput(new Text("foo"), new LongWritable(INCORRECT_OUT))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void TesttestRun5() {
+    try {
+      driver
+            .withInputKey(new Text("foo"))
+            .withInputValue(new LongWritable(IN_A))
+            .withInputValue(new LongWritable(IN_B))
+            .withOutput(new Text("foo"), new LongWritable(IN_A))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void TesttestRun6() {
+    try {
+      driver
+            .withInputKey(new Text("foo"))
+            .withInputValue(new LongWritable(IN_A))
+            .withInputValue(new LongWritable(IN_B))
+            .withOutput(new Text("foo"), new LongWritable(IN_A))
+            .withOutput(new Text("foo"), new LongWritable(IN_B))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void TesttestRun7() {
+    try {
+      driver
+              .withInputKey(new Text("foo"))
+              .withInputValue(new LongWritable(IN_A))
+              .withInputValue(new LongWritable(IN_B))
+              .withOutput(new Text("foo"), new LongWritable(OUT_VAL))
+              .withOutput(new Text("foo"), new LongWritable(OUT_VAL))
+              .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void TesttestRun8() {
+    try {
+      driver
+            .withInputKey(new Text("foo"))
+            .withInputValue(new LongWritable(IN_A))
+            .withInputValue(new LongWritable(IN_B))
+            .withOutput(new Text("bar"), new LongWritable(OUT_VAL))
+            .withOutput(new Text("foo"), new LongWritable(OUT_VAL))
+            .runTest();
+            fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void TesttestRun9() {
+    try {
+      driver
+            .withInputKey(new Text("foo"))
+            .withInputValue(new LongWritable(IN_A))
+            .withInputValue(new LongWritable(IN_B))
+            .withOutput(new Text("foo"), new LongWritable(OUT_VAL))
+            .withOutput(new Text("bar"), new LongWritable(OUT_VAL))
+            .runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+
+  @Test
+  public void testEmptyInput() {
+    // (null, <empty>) will be forcibly fed as input
+    // since we use LongSumReducer, expect (null, 0) out.
+    driver
+            .withOutput(null, new LongWritable(OUT_EMPTY))
+            .runTest();
+  }
+
+  @Test
+  public void testEmptyInput2() {
+    // because a null key with zero inputs will be fed as input
+    // to this reducer, do not accept no outputs.
+    try {
+      driver.runTest();
+      fail();
+    } catch (RuntimeException re) {
+      // expected.
+    }
+  }
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestTestDriver.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestTestDriver.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestTestDriver.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/TestTestDriver.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,204 @@
+/**
+ * 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.assertListEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Test;
+
+public class TestTestDriver extends TestCase {
+
+  /**
+   * Test method for
+   * {@link org.apache.hadoop.mrunit.TestDriver#parseTabbedPair(java.lang.String)}.
+   */
+  @Test
+  public void testParseTabbedPair1() {
+    Pair<Text, Text> pr = TestDriver.parseTabbedPair("foo\tbar");
+    assertEquals(pr.getFirst().toString(), "foo");
+    assertEquals(pr.getSecond().toString(), "bar");
+  }
+
+  @Test
+  public void testParseTabbedPair2() {
+    Pair<Text, Text> pr = TestDriver.parseTabbedPair("   foo\tbar");
+    assertEquals(pr.getFirst().toString(), "   foo");
+    assertEquals(pr.getSecond().toString(), "bar");
+  }
+
+  @Test
+  public void testParseTabbedPair3() {
+    Pair<Text, Text> pr = TestDriver.parseTabbedPair("foo\tbar   ");
+    assertEquals(pr.getFirst().toString(), "foo");
+    assertEquals(pr.getSecond().toString(), "bar   ");
+  }
+
+  @Test
+  public void testParseTabbedPair4() {
+    Pair<Text, Text> pr = TestDriver.parseTabbedPair("foo    \tbar");
+    assertEquals(pr.getFirst().toString(), "foo    ");
+    assertEquals(pr.getSecond().toString(), "bar");
+  }
+
+  @Test
+  public void testParseTabbedPair5() {
+    Pair<Text, Text> pr = TestDriver.parseTabbedPair("foo\t  bar");
+    assertEquals(pr.getFirst().toString(), "foo");
+    assertEquals(pr.getSecond().toString(), "  bar");
+  }
+
+  @Test
+  public void testParseTabbedPair6() {
+    Pair<Text, Text> pr = TestDriver.parseTabbedPair("foo\t\tbar");
+    assertEquals(pr.getFirst().toString(), "foo");
+    assertEquals(pr.getSecond().toString(), "\tbar");
+  }
+
+  @Test
+  public void testParseTabbedPair7() {
+    Pair<Text, Text> pr = TestDriver.parseTabbedPair("foo\tbar\n");
+    assertEquals(pr.getFirst().toString(), "foo");
+    assertEquals(pr.getSecond().toString(), "bar\n");
+  }
+
+  @Test
+  public void testParseTabbedPair8() {
+    Pair<Text, Text> pr = TestDriver.parseTabbedPair("foo\t  bar\tbaz");
+    assertEquals(pr.getFirst().toString(), "foo");
+    assertEquals(pr.getSecond().toString(), "  bar\tbaz");
+  }
+
+  /**
+   * Test method for
+   * {@link
+   * org.apache.hadoop.mrunit.TestDriver#parseCommaDelimitedList(java.lang.String)}.
+   */
+  @Test
+  public void testParseCommaDelimList1() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("foo");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    assertListEquals(out, verify);
+  }
+
+  @Test
+  public void testParseCommaDelimList2() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("foo,bar");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    verify.add(new Text("bar"));
+    assertListEquals(out, verify);
+  }
+
+  @Test
+  public void testParseCommaDelimList3() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("foo   ,bar");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    verify.add(new Text("bar"));
+    assertListEquals(out, verify);
+  }
+
+  @Test
+  public void testParseCommaDelimList4() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("foo,   bar");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    verify.add(new Text("bar"));
+    assertListEquals(out, verify);
+  }
+
+  @Test
+  public void testParseCommaDelimList5() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("   foo,bar");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    verify.add(new Text("bar"));
+    assertListEquals(out, verify);
+  }
+
+  @Test
+  public void testParseCommaDelimList6() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("foo,bar   ");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    verify.add(new Text("bar"));
+    assertListEquals(out, verify);
+  }
+
+  @Test
+  public void testParseCommaDelimList7() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("foo,bar, baz");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    verify.add(new Text("bar"));
+    verify.add(new Text("baz"));
+    assertListEquals(out, verify);
+  }
+
+  // note: we decide that correct behavior is that this does *not*
+  // add a tailing empty element by itself.
+  @Test
+  public void testParseCommaDelimList8() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("foo,bar,");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    verify.add(new Text("bar"));
+    assertListEquals(out, verify);
+  }
+
+  // but this one does.
+  @Test
+  public void testParseCommaDelimList8a() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("foo,bar,,");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    verify.add(new Text("bar"));
+    verify.add(new Text(""));
+    assertListEquals(out, verify);
+  }
+
+  @Test
+  public void testParseCommaDelimList9() {
+    List<Text> out = TestDriver.parseCommaDelimitedList("foo,,bar");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text("foo"));
+    verify.add(new Text(""));
+    verify.add(new Text("bar"));
+    assertListEquals(out, verify);
+  }
+
+  @Test
+  public void testParseCommaDelimList10() {
+    List<Text> out = TestDriver.parseCommaDelimitedList(",foo,bar");
+    ArrayList<Text> verify = new ArrayList<Text>();
+    verify.add(new Text(""));
+    verify.add(new Text("foo"));
+    verify.add(new Text("bar"));
+    assertListEquals(out, verify);
+  }
+
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mock/TestMockReporter.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mock/TestMockReporter.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mock/TestMockReporter.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mock/TestMockReporter.java Mon Apr  6 21:23:26 2009
@@ -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.mrunit.mock;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.mapred.InputSplit;
+import org.junit.Test;
+
+
+public class TestMockReporter extends TestCase {
+
+  @Test
+  public void testGetInputSplitForMapper() {
+    InputSplit split = new MockReporter(MockReporter.ReporterType.Mapper).getInputSplit();
+    assertTrue(null != split);
+  }
+
+  // reporter is contractually obligated to throw an exception
+  // if the reducer tries to grab the input split.
+  @Test
+  public void testGetInputSplitForReducer() {
+    try {
+      new MockReporter(MockReporter.ReporterType.Reducer).getInputSplit();
+      fail(); // shouldn't get here
+    } catch (UnsupportedOperationException uoe) {
+      // expected this.
+    }
+  }
+}
+

Added: hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/AllTests.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/AllTests.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/AllTests.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/AllTests.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,37 @@
+/**
+ * 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 junit.framework.Test;
+import junit.framework.TestSuite;
+
+/**
+ * Test suite for all type objects in the common library
+ *
+ */
+public final class AllTests {
+
+  private AllTests() { }
+
+  public static Test suite() {
+    TestSuite suite = new TestSuite("Test for org.apache.hadoop.mrunit.types");
+
+    suite.addTestSuite(TestPair.class);
+    return suite;
+  }
+}

Added: hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/TestPair.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/TestPair.java?rev=762509&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/TestPair.java (added)
+++ hadoop/core/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/types/TestPair.java Mon Apr  6 21:23:26 2009
@@ -0,0 +1,208 @@
+/**
+ * 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 junit.framework.TestCase;
+
+import org.junit.Test;
+
+public class TestPair extends TestCase {
+
+  public static final int VAL_A = 3;
+  public static final int VAL_B = 4;
+  public static final int VAL_C = 65;
+  public static final int VAL_D = 98;
+
+  /**
+   * Test method for
+   * {@link
+   * org.apache.hadoop.mrunit.types.Pair#equals(org.apache.hadoop.mrunit.types.Pair)}.
+   */
+  @Test
+  public void testEqualsTrulyEqual() {
+    assertTrue(
+      new Pair<Integer, Integer>(Integer.valueOf(VAL_A), Integer.valueOf(VAL_B))
+      .equals(new Pair<Integer, Integer>(
+              Integer.valueOf(VAL_A), Integer.valueOf(VAL_B))));
+  }
+
+  @Test
+  public void testEqualsInequalFirst() {
+    assertFalse(new Pair<Integer, Integer>(
+                Integer.valueOf(VAL_A), Integer.valueOf(VAL_B))
+            .equals(new Pair<Integer, Integer>(Integer.valueOf(VAL_C),
+                    Integer.valueOf(VAL_B))));
+  }
+
+  @Test
+  public void testEqualsInequalSecond() {
+    assertFalse(new Pair<Integer, Integer>(
+                Integer.valueOf(VAL_A), Integer.valueOf(VAL_B))
+            .equals(new Pair<Integer, Integer>(Integer.valueOf(VAL_A),
+                    Integer.valueOf(VAL_D))));
+  }
+
+  @Test
+  public void testEqualsNullNull() {
+    assertTrue(new Pair<Integer, Integer>(null, null)
+            .equals(new Pair<Integer, Integer>(null, null)));
+  }
+
+  @Test
+  public void testEqualsNullIntFirst0() {
+    assertTrue(new Pair<Integer, Integer>(Integer.valueOf(VAL_A), null)
+            .equals(new Pair<Integer, Integer>(Integer.valueOf(VAL_A), null)));
+  }
+
+  @Test
+  public void testEqualsNullIntFirst1() {
+    assertFalse(new Pair<Integer, Integer>(null, null)
+            .equals(new Pair<Integer, Integer>(Integer.valueOf(VAL_A), null)));
+  }
+
+  @Test
+  public void testEqualsNullIntFirst2() {
+    assertFalse(new Pair<Integer, Integer>(Integer.valueOf(VAL_A), null)
+            .equals(new Pair<Integer, Integer>(null, null)));
+  }
+
+  @Test
+  public void testEqualsNullIntSecond0() {
+    assertTrue(new Pair<Integer, Integer>(null, Integer.valueOf(VAL_A))
+            .equals(new Pair<Integer, Integer>(null, Integer.valueOf(VAL_A))));
+  }
+
+  @Test
+  public void testEqualsNullIntSecond1() {
+    assertFalse(new Pair<Integer, Integer>(null, null)
+            .equals(new Pair<Integer, Integer>(null, Integer.valueOf(VAL_A))));
+  }
+
+  @Test
+  public void testEqualsNullIntSecond2() {
+    assertFalse(new Pair<Integer, Integer>(null, Integer.valueOf(VAL_A))
+            .equals(new Pair<Integer, Integer>(null, null)));
+  }
+
+  /**
+   * Test method for
+   * {@link
+   * org.apache.hadoop.mrunit.types.Pair#compareTo(org.apache.hadoop.mrunit.types.Pair)}.
+   */
+  @Test
+  public void testCompareToTrulyEqual() {
+    assertTrue(0 == new Pair<Integer, Integer>(Integer.valueOf(VAL_A),
+                                               Integer.valueOf(VAL_B))
+            .compareTo(new Pair<Integer, Integer>(Integer.valueOf(VAL_A),
+                    Integer.valueOf(VAL_B))));
+  }
+
+  @Test
+  public void testCompareToInequalFirst() {
+    assertFalse(0 == new Pair<Integer, Integer>(Integer.valueOf(VAL_A),
+            Integer.valueOf(VAL_B)).compareTo(
+                new Pair<Integer, Integer>(Integer.valueOf(
+                                           VAL_C), Integer.valueOf(VAL_B))));
+  }
+
+  @Test
+  public void testCompareToInequalSecond() {
+    assertFalse(0 == new Pair<Integer, Integer>(Integer.valueOf(VAL_A),
+            Integer.valueOf(VAL_B)).compareTo(
+            new Pair<Integer, Integer>(Integer.valueOf(VAL_A),
+                                       Integer.valueOf(VAL_D))));
+  }
+
+  @Test
+  public void testCompareToNullNull() {
+    try {
+      new Pair<Integer, Integer>(null, null)
+              .compareTo(new Pair<Integer, Integer>(null, null));
+      fail();
+    } catch (NullPointerException npe) {
+      // expected this; ok.
+    }
+  }
+
+  @Test
+  public void testCompareToNullIntFirst0() {
+    try {
+      new Pair<Integer, Integer>(Integer.valueOf(VAL_A), null)
+          .compareTo(new Pair<Integer, Integer>(Integer.valueOf(VAL_A), null));
+      fail();
+    } catch (NullPointerException npe) {
+      // expected this; ok.
+    }
+  }
+
+  @Test
+  public void testCompareToNullIntFirst1() {
+    try {
+      new Pair<Integer, Integer>(null, null)
+          .compareTo(new Pair<Integer, Integer>(Integer.valueOf(VAL_A), null));
+      fail();
+    } catch (NullPointerException npe) {
+      // expected this; ok.
+    }
+  }
+
+  @Test
+  public void testCompareToNullIntFirst2() {
+    try {
+      new Pair<Integer, Integer>(Integer.valueOf(VAL_A), null)
+          .compareTo(new Pair<Integer, Integer>(null, null));
+      fail();
+    } catch (NullPointerException npe) {
+      // expected this; ok.
+    }
+  }
+
+  @Test
+  public void testCompareToNullIntSecond0() {
+    try {
+      new Pair<Integer, Integer>(null, Integer.valueOf(VAL_A))
+          .compareTo(new Pair<Integer, Integer>(null, Integer.valueOf(VAL_A)));
+      fail();
+    } catch (NullPointerException npe) {
+      // expected this; ok.
+    }
+  }
+
+  @Test
+  public void testCompareToNullIntSecond1() {
+    try {
+      new Pair<Integer, Integer>(null, null)
+          .compareTo(new Pair<Integer, Integer>(null, Integer.valueOf(VAL_A)));
+      fail();
+    } catch (NullPointerException npe) {
+      // expected this; ok.
+    }
+  }
+
+  @Test
+  public void testCompareToNullIntSecond2() {
+    try {
+      new Pair<Integer, Integer>(null, Integer.valueOf(VAL_A))
+          .compareTo(new Pair<Integer, Integer>(null, null));
+      fail();
+    } catch (NullPointerException npe) {
+      // expected this; ok.
+    }
+  }
+
+}