You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mrunit.apache.org by db...@apache.org on 2012/07/31 23:05:15 UTC

svn commit: r1367781 - in /mrunit/trunk/src: main/java/org/apache/hadoop/mrunit/ test/java/org/apache/hadoop/mrunit/

Author: dbeech
Date: Tue Jul 31 21:05:14 2012
New Revision: 1367781

URL: http://svn.apache.org/viewvc?rev=1367781&view=rev
Log:
MRUNIT-126 - Counter tests are ignored if test is executed using run() rather than runTest()

Modified:
    mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapDriverBase.java
    mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapReduceDriverBase.java
    mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/ReduceDriverBase.java
    mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/TestDriver.java
    mrunit/trunk/src/test/java/org/apache/hadoop/mrunit/TestMapDriver.java

Modified: mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapDriverBase.java
URL: http://svn.apache.org/viewvc/mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapDriverBase.java?rev=1367781&r1=1367780&r2=1367781&view=diff
==============================================================================
--- mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapDriverBase.java (original)
+++ mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapDriverBase.java Tue Jul 31 21:05:14 2012
@@ -200,12 +200,10 @@ public abstract class MapDriverBase<K1, 
   public abstract List<Pair<K2, V2>> run() throws IOException;
 
   @Override
-  public void runTest(final boolean orderMatters) throws IOException {
-  	for (Pair<K1, V1> input : inputs) {
-  		LOG.debug("Mapping input (" + input.getFirst() + ", " + input.getSecond() + ")");
-  	}
-    final List<Pair<K2, V2>> outputs = run();
-    validate(outputs, orderMatters);
-    validate(counterWrapper);
+  protected void printPreTestDebugLog() {
+    for (Pair<K1, V1> input : inputs) {
+      LOG.debug("Mapping input (" + input.getFirst() + ", " + input.getSecond() + ")");
+    }
   }
+
 }

Modified: mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapReduceDriverBase.java
URL: http://svn.apache.org/viewvc/mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapReduceDriverBase.java?rev=1367781&r1=1367780&r2=1367781&view=diff
==============================================================================
--- mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapReduceDriverBase.java (original)
+++ mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/MapReduceDriverBase.java Tue Jul 31 21:05:14 2012
@@ -154,13 +154,6 @@ public abstract class MapReduceDriverBas
   @Override
   public abstract List<Pair<K3, V3>> run() throws IOException;
 
-  @Override
-  public void runTest(final boolean orderMatters) throws IOException {
-    final List<Pair<K3, V3>> reduceOutputs = run();
-    validate(reduceOutputs, orderMatters);
-    validate(counterWrapper);
-  }
-
   /**
    * Take the outputs from the Mapper, combine all values for the same key, and
    * sort them by key.
@@ -257,4 +250,5 @@ public abstract class MapReduceDriverBas
     keyValueOrderComparator = ReflectionUtils.newInstance(
         orderComparator.getClass(), getConfiguration());
   }
+
 }

Modified: mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/ReduceDriverBase.java
URL: http://svn.apache.org/viewvc/mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/ReduceDriverBase.java?rev=1367781&r1=1367780&r2=1367781&view=diff
==============================================================================
--- mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/ReduceDriverBase.java (original)
+++ mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/ReduceDriverBase.java Tue Jul 31 21:05:14 2012
@@ -232,20 +232,13 @@ public abstract class ReduceDriverBase<K
   public abstract List<Pair<K2, V2>> run() throws IOException;
 
   @Override
-  public void runTest(final boolean orderMatters) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      final StringBuilder sb = new StringBuilder();
-      for (Pair<K1, List<V1>> input : inputs) {
-        formatValueList(input.getSecond(), sb);
-        LOG.debug("Reducing input (" + input.getFirst() + ", " + sb + ")");
-        sb.delete(0, sb.length());
-      }
+  protected void printPreTestDebugLog() {
+    final StringBuilder sb = new StringBuilder();
+    for (Pair<K1, List<V1>> input : inputs) {
+      formatValueList(input.getSecond(), sb);
+      LOG.debug("Reducing input (" + input.getFirst() + ", " + sb + ")");
+      sb.delete(0, sb.length());
     }
-
-    List<Pair<K2, V2>> outputs = null;
-    outputs = run();
-    validate(outputs, orderMatters);
-    validate(counterWrapper);
   }
 
   protected static class ValueClassInstanceReuseList<T> extends ArrayList<T> {

Modified: mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/TestDriver.java
URL: http://svn.apache.org/viewvc/mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/TestDriver.java?rev=1367781&r1=1367780&r2=1367781&view=diff
==============================================================================
--- mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/TestDriver.java (original)
+++ mrunit/trunk/src/main/java/org/apache/hadoop/mrunit/TestDriver.java Tue Jul 31 21:05:14 2012
@@ -170,7 +170,7 @@ public abstract class TestDriver<K1, V1,
       final Configuration configuration) {
     this.outputCopyingOrInputFormatConf = returnNonNull(configuration);
   }
-  
+
   /**
    * @return the path passed to the mapper InputSplit
    */
@@ -182,7 +182,24 @@ public abstract class TestDriver<K1, V1,
    */
   public void setMapInputPath(Path mapInputPath) {
     this.mapInputPath = mapInputPath;
-  }  
+  }
+
+  /**
+   * Runs the test but returns the result set instead of validating it (ignores
+   * any addOutput(), etc calls made before this). 
+   * 
+   * Also optionally performs counter validation. 
+   * 
+   * @param validateCounters whether to run automatic counter validation
+   * @return the list of (k, v) pairs returned as output from the test
+   */
+  public List<Pair<K2, V2>> run(boolean validateCounters) throws IOException {
+    final List<Pair<K2, V2>> outputs = run();
+    if (validateCounters) {
+      validate(counterWrapper);
+    }
+    return outputs;
+  }
 
   /**
    * Runs the test but returns the result set instead of validating it (ignores
@@ -205,7 +222,21 @@ public abstract class TestDriver<K1, V1,
    * @param orderMatters
    *          Whether or not output ordering is important
    */
-  public abstract void runTest(boolean orderMatters) throws IOException;
+  public void runTest(final boolean orderMatters) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      printPreTestDebugLog();
+    }
+    final List<Pair<K2, V2>> outputs = run();
+    validate(outputs, orderMatters);
+    validate(counterWrapper);
+  }
+
+  /**
+   * Overridable hook for printing pre-test debug information
+   */
+  protected void printPreTestDebugLog() {
+    //
+  }
 
   /**
    * Split "key \t val" into Pair(Text(key), Text(val))

Modified: mrunit/trunk/src/test/java/org/apache/hadoop/mrunit/TestMapDriver.java
URL: http://svn.apache.org/viewvc/mrunit/trunk/src/test/java/org/apache/hadoop/mrunit/TestMapDriver.java?rev=1367781&r1=1367780&r2=1367781&view=diff
==============================================================================
--- mrunit/trunk/src/test/java/org/apache/hadoop/mrunit/TestMapDriver.java (original)
+++ mrunit/trunk/src/test/java/org/apache/hadoop/mrunit/TestMapDriver.java Tue Jul 31 21:05:14 2012
@@ -281,7 +281,40 @@ public class TestMapDriver {
         .withCounter(MapperWithCounters.Counters.X, 1)
         .withCounter("category", "name", 1).runTest();
   }
-  
+
+  @Test
+  public void testWithCounterUsingRunMethodPerformingCounterChecking() throws IOException {
+    MapDriver<Text, Text, Text, Text> driver = MapDriver.newMapDriver();
+
+    driver.withMapper(new MapperWithCounters<Text, Text, Text, Text>())
+        .withInput(new Text("hie"), new Text("Hi"))
+        .withOutput(new Text("hie"), new Text("Hi"))
+        .withCounter(MapperWithCounters.Counters.X, 1)
+        .withCounter("category", "name", 1).run(true);
+  }
+
+  @Test
+  public void testWithCounterUsingRunMethodExplicitIgnoreCounterChecking() throws IOException {
+    MapDriver<Text, Text, Text, Text> driver = MapDriver.newMapDriver();
+
+    driver.withMapper(new MapperWithCounters<Text, Text, Text, Text>())
+        .withInput(new Text("hie"), new Text("Hi"))
+        .withOutput(new Text("hie"), new Text("Hi"))
+        .withCounter(MapperWithCounters.Counters.X, 999)
+        .withCounter("INVALIDCOUNTER", "NOTSET", 999).run(false);
+  }
+
+  @Test
+  public void testWithCounterUsingRunMethodImplicitIgnoreCounterChecking() throws IOException {
+    MapDriver<Text, Text, Text, Text> driver = MapDriver.newMapDriver();
+
+    driver.withMapper(new MapperWithCounters<Text, Text, Text, Text>())
+        .withInput(new Text("hie"), new Text("Hi"))
+        .withOutput(new Text("hie"), new Text("Hi"))
+        .withCounter(MapperWithCounters.Counters.X, 999)
+        .withCounter("INVALIDCOUNTER", "NOTSET", 999).run();
+  }
+
   @Test
   public void testWithCounterAndNoneMissing() throws IOException {
     MapDriver<Text, Text, Text, Text> driver = MapDriver.newMapDriver();