You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/13 18:55:01 UTC

svn commit: r1445770 [2/2] - in /hbase/branches/hbase-7290/hbase-server/src: main/java/org/apache/hadoop/hbase/server/ main/java/org/apache/hadoop/hbase/server/errorhandling/ main/java/org/apache/hadoop/hbase/server/errorhandling/exception/ main/java/o...

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestFaultInjectionPolicies.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestFaultInjectionPolicies.java?rev=1445770&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestFaultInjectionPolicies.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestFaultInjectionPolicies.java Wed Feb 13 17:55:00 2013
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.server.errorhandling.impl;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the fault injection policies and combinations
+ */
+@Category(SmallTests.class)
+public class TestFaultInjectionPolicies {
+
+  @Test
+  public void testAndCombination() {
+    FaultInjectionPolicy alwaysFalse = new FaultInjectionPolicy();
+    assertFalse("Default policy isn't false", alwaysFalse.shouldFault(null));
+
+    FaultInjectionPolicy alwaysTrue = new AlwaysTrue();
+    FaultInjectionPolicy andTrue = new AlwaysTrue().or(alwaysTrue).or(alwaysTrue);
+    assertTrue("And True isn't always returning true", andTrue.shouldFault(null));
+
+    FaultInjectionPolicy andFalse = new FaultInjectionPolicy().and(alwaysTrue);
+    assertFalse("false AND true", andFalse.shouldFault(null));
+    assertFalse("true AND false", alwaysTrue.and(alwaysFalse).shouldFault(null));
+    assertFalse("true AND (false AND true)",
+      new AlwaysTrue().and(new FaultInjectionPolicy().and(new AlwaysTrue())).shouldFault(null));
+    assertFalse("(true AND false AND true)",
+      new AlwaysTrue().and(new FaultInjectionPolicy()).and(new AlwaysTrue()).shouldFault(null));
+  }
+
+  @Test
+  public void testORCombination() {
+    FaultInjectionPolicy alwaysTrue = new AlwaysTrue();
+
+    FaultInjectionPolicy andTrue = new AlwaysTrue().or(alwaysTrue).or(alwaysTrue);
+    assertTrue("OR True isn't always returning true", andTrue.shouldFault(null));
+
+    FaultInjectionPolicy andFalse = new FaultInjectionPolicy().or(alwaysTrue);
+    assertTrue("Combination of true OR false should be true", andFalse.shouldFault(null));
+    assertTrue("Combining multiple ands isn't correct",
+      new FaultInjectionPolicy().or(andTrue).or(andFalse).shouldFault(null));
+  }
+
+  @Test
+  public void testMixedAndOr() {
+    assertTrue("true AND (false OR true)",
+      new AlwaysTrue().and(new FaultInjectionPolicy().or(new AlwaysTrue())).shouldFault(null));
+    assertTrue("(true AND false) OR true",
+      new AlwaysTrue().or(new AlwaysTrue().and(new FaultInjectionPolicy())).shouldFault(null));
+    assertFalse(
+      "(true AND false) OR false",
+      new FaultInjectionPolicy().or(new AlwaysTrue().and(new FaultInjectionPolicy())).shouldFault(
+        null));
+  }
+
+  private static class AlwaysTrue extends FaultInjectionPolicy {
+
+    protected boolean checkForFault(StackTraceElement[] stack) {
+      return true;
+    }
+  }
+
+  public static class SimplePolicyFaultInjector extends PoliciedFaultInjector<Exception> {
+
+    public SimplePolicyFaultInjector(FaultInjectionPolicy policy) {
+      super(policy);
+    }
+
+    @Override
+    protected Pair<Exception, Object[]> getInjectedError(StackTraceElement[] trace) {
+      return new Pair<Exception, Object[]>(new RuntimeException("error"), null);
+    }
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestOperationAttemptTimer.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestOperationAttemptTimer.java?rev=1445770&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestOperationAttemptTimer.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestOperationAttemptTimer.java Wed Feb 13 17:55:00 2013
@@ -0,0 +1,97 @@
+/**
+ * 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.hbase.server.errorhandling.impl;
+
+import static org.junit.Assert.fail;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.server.errorhandling.ExceptionListener;
+import org.apache.hadoop.hbase.server.errorhandling.OperationAttemptTimer;
+import org.apache.hadoop.hbase.server.errorhandling.exception.OperationAttemptTimeoutException;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * Test the {@link OperationAttemptTimer} to ensure we fulfill contracts
+ */
+@Category(SmallTests.class)
+@SuppressWarnings("unchecked")
+public class TestOperationAttemptTimer {
+
+  private static final Log LOG = LogFactory.getLog(TestOperationAttemptTimer.class);
+
+  @Test(timeout = 1000)
+  public void testTimerTrigger() {
+    final long time = 10000000;
+    ExceptionListener<Exception> listener = Mockito.mock(ExceptionListener.class);
+    OperationAttemptTimer timer = new OperationAttemptTimer(listener, time);
+    timer.start();
+    timer.trigger();
+    Mockito.verify(listener, Mockito.times(1)).receiveError(Mockito.anyString(),
+      Mockito.any(OperationAttemptTimeoutException.class));
+  }
+
+  @Test
+  public void testTimerPassesOnErrorInfo() {
+    final long time = 10;
+    ExceptionListener<Exception> listener = Mockito.mock(ExceptionListener.class);
+    final Object[] data = new Object[] { "data" };
+    OperationAttemptTimer timer = new OperationAttemptTimer(listener, time, data);
+    timer.start();
+    timer.trigger();
+    Mockito.verify(listener).receiveError(Mockito.anyString(),
+      Mockito.any(OperationAttemptTimeoutException.class), Mockito.eq(data[0]));
+  }
+
+  @Test(timeout = 1000)
+  public void testStartAfterComplete() throws InterruptedException {
+    final long time = 10;
+    ExceptionListener<Exception> listener = Mockito.mock(ExceptionListener.class);
+    OperationAttemptTimer timer = new OperationAttemptTimer(listener, time);
+    timer.complete();
+    try {
+      timer.start();
+      fail("Timer should fail to start after complete.");
+    } catch (IllegalStateException e) {
+      LOG.debug("Correctly failed timer: " + e.getMessage());
+    }
+    Thread.sleep(time + 1);
+    Mockito.verifyZeroInteractions(listener);
+  }
+
+  @Test(timeout = 1000)
+  public void testStartAfterTrigger() throws InterruptedException {
+    final long time = 10;
+    ExceptionListener<Exception> listener = Mockito.mock(ExceptionListener.class);
+    OperationAttemptTimer timer = new OperationAttemptTimer(listener, time);
+    timer.trigger();
+    try {
+      timer.start();
+      fail("Timer should fail to start after complete.");
+    } catch (IllegalStateException e) {
+      LOG.debug("Correctly failed timer: " + e.getMessage());
+    }
+    Thread.sleep(time * 2);
+    Mockito.verify(listener, Mockito.times(1)).receiveError(Mockito.anyString(),
+      Mockito.any(OperationAttemptTimeoutException.class));
+    Mockito.verifyNoMoreInteractions(listener);
+  }
+}

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestSingleExceptionDispatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestSingleExceptionDispatcher.java?rev=1445770&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestSingleExceptionDispatcher.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/server/errorhandling/impl/TestSingleExceptionDispatcher.java Wed Feb 13 17:55:00 2013
@@ -0,0 +1,113 @@
+/**
+ * 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.hbase.server.errorhandling.impl;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.server.errorhandling.ExceptionListener;
+import org.apache.hadoop.hbase.server.errorhandling.OperationAttemptTimer;
+import org.apache.hadoop.hbase.server.errorhandling.exception.OperationAttemptTimeoutException;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * Test using the single error dispatcher
+ */
+@SuppressWarnings("unchecked")
+@Category(SmallTests.class)
+public class TestSingleExceptionDispatcher {
+
+  private static final Log LOG = LogFactory.getLog(TestSingleExceptionDispatcher.class);
+  @Test
+  public void testErrorPropagation() {
+    ExceptionListener<Exception> listener1 = Mockito.mock(ExceptionListener.class);
+    ExceptionListener<Exception> listener2 = Mockito.mock(ExceptionListener.class);
+
+    ExceptionDispatcher<? extends ExceptionListener<Exception>, Exception> monitor = new ExceptionDispatcher<ExceptionListener<Exception>, Exception>();
+
+    // add the listeners
+    monitor.addErrorListener(monitor.genericVisitor, listener1);
+    monitor.addErrorListener(monitor.genericVisitor, listener2);
+
+    // create an artificial error
+    String message = "Some error";
+    Exception expected = new ExceptionForTesting("error");
+    Object info = "info1";
+    monitor.receiveError(message, expected, info);
+
+    // make sure the listeners got the error
+    Mockito.verify(listener1).receiveError(message, expected, info);
+    Mockito.verify(listener2).receiveError(message, expected, info);
+
+    // make sure that we get an exception
+    try {
+      monitor.failOnError();
+      fail("Monitor should have thrown an exception after getting error.");
+    } catch (Exception e) {
+      assertTrue("Got an unexpected exception:" + e, e instanceof ExceptionForTesting);
+      LOG.debug("Got the testing exception!");
+    }
+    // push another error, but this shouldn't be passed to the listeners
+    monitor.receiveError("another error", new ExceptionForTesting("hello"),
+      "shouldn't be found");
+    // make sure we don't re-propagate the error
+    Mockito.verifyNoMoreInteractions(listener1, listener2);
+  }
+
+  @Test
+  public void testSingleDispatcherWithTimer() {
+    ExceptionListener<Exception> listener1 = Mockito.mock(ExceptionListener.class);
+    ExceptionListener<Exception> listener2 = Mockito.mock(ExceptionListener.class);
+
+    ExceptionDispatcher<? extends ExceptionListener<Exception>, Exception> monitor = new ExceptionDispatcher<ExceptionListener<Exception>, Exception>();
+
+    // add the listeners
+    monitor.addErrorListener(monitor.genericVisitor, listener1);
+    monitor.addErrorListener(monitor.genericVisitor, listener2);
+
+    Object info = "message";
+    OperationAttemptTimer timer = new OperationAttemptTimer(monitor, 1000, info);
+    timer.start();
+    timer.trigger();
+
+    assertTrue("Monitor didn't get timeout", monitor.checkForError());
+
+    // verify that that we propagated the error
+    Mockito.verify(listener1).receiveError(Mockito.anyString(),
+      Mockito.any(OperationAttemptTimeoutException.class), Mockito.eq(info));
+    Mockito.verify(listener2).receiveError(Mockito.anyString(),
+      Mockito.any(OperationAttemptTimeoutException.class), Mockito.eq(info));
+  }
+
+  @Test
+  public void testAddListenerWithoutVisitor() {
+    SimpleErrorListener<Exception> listener = new SimpleErrorListener<Exception>();
+    ExceptionDispatcher<SimpleErrorListener<Exception>, Exception> monitor = new ExceptionDispatcher<SimpleErrorListener<Exception>, Exception>();
+    try {
+      monitor.addErrorListener(listener);
+      fail("Monitor needs t have a visitor for adding generically typed listeners");
+    } catch (UnsupportedOperationException e) {
+      LOG.debug("Correctly failed to add listener without visitor: " + e.getMessage());
+    }
+  }
+}
\ No newline at end of file