You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by to...@apache.org on 2009/08/21 16:51:40 UTC

svn commit: r806577 [2/2] - in /hadoop/mapreduce/trunk: ./ src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/ src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mapreduce/ src/contrib/mrunit/src/java/org/apache/hadoop/mrunit/mapreduce/mock/ src/co...

Added: hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestMapDriver.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestMapDriver.java?rev=806577&view=auto
==============================================================================
--- hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestMapDriver.java (added)
+++ hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestMapDriver.java Fri Aug 21 14:51:39 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.mapreduce;
+
+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.mapreduce.Mapper;
+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 Mapper<Text, Text, Text, Text>(); // default action is identity mapper.
+    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.
+    }
+  }
+
+}
+

Added: hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestMapReduceDriver.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestMapReduceDriver.java?rev=806577&view=auto
==============================================================================
--- hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestMapReduceDriver.java (added)
+++ hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestMapReduceDriver.java Fri Aug 21 14:51:39 2009
@@ -0,0 +1,235 @@
+/**
+ * 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.mapreduce;
+
+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.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.reduce.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 Mapper<Text, LongWritable, Text, LongWritable>(); // This is the IdentityMapper
+    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/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestReduceDriver.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestReduceDriver.java?rev=806577&view=auto
==============================================================================
--- hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestReduceDriver.java (added)
+++ hadoop/mapreduce/trunk/src/contrib/mrunit/src/test/org/apache/hadoop/mrunit/mapreduce/TestReduceDriver.java Fri Aug 21 14:51:39 2009
@@ -0,0 +1,227 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mrunit.mapreduce;
+
+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.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.reduce.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.
+    }
+  }
+}
+