hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jmhs...@apache.org
Subject svn commit: r1445798 [3/3] - in /hbase/branches/hbase-7290/hbase-server/src: main/java/org/apache/hadoop/hbase/errorhandling/ main/java/org/apache/hadoop/hbase/protobuf/generated/ main/protobuf/ test/java/org/apache/hadoop/hbase/errorhandling/ test/jav...
Date Wed, 13 Feb 2013 18:21:27 GMT
Added: hbase/branches/hbase-7290/hbase-server/src/main/protobuf/ErrorHandling.proto
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/protobuf/ErrorHandling.proto?rev=1445798&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/protobuf/ErrorHandling.proto (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/protobuf/ErrorHandling.proto Wed Feb 13
18:21:26 2013
@@ -0,0 +1,58 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that used to error handling
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ErrorHandlingProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+/**
+ * Protobuf version of a java.lang.StackTraceElement
+ * so we can serialize exceptions.
+ */
+message StackTraceElementMessage {
+  optional string declaringClass = 1;
+  optional string methodName = 2;
+  optional string fileName = 3;
+  optional int32 lineNumber = 4;
+}
+
+/**
+ * Cause of a remote failure for a generic exception. Contains
+ * all the information for a generic exception as well as
+ * optional info about the error for generic info passing
+ * (which should be another protobuffed class).
+ */
+message GenericExceptionMessage {
+  optional string className = 1;
+  optional string message = 2;
+  optional bytes errorInfo = 3;
+  repeated StackTraceElementMessage trace = 4;
+}
+
+/**
+ * Exception sent across the wire when a remote task needs
+ * to notify other tasks that it failed and why
+ */
+message ForeignExceptionMessage {
+  optional string source = 1;
+  optional GenericExceptionMessage genericException = 2;
+
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java?rev=1445798&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java
(added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java
Wed Feb 13 18:21:26 2013
@@ -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.hbase.errorhandling;
+
+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.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * Test that we propagate errors through an dispatcher exactly once via different failure
+ * injection mechanisms.
+ */
+@Category(SmallTests.class)
+public class TestForeignExceptionDispatcher {
+  private static final Log LOG = LogFactory.getLog(TestForeignExceptionDispatcher.class);
+
+  /**
+   * Exception thrown from the test
+   */
+  final ForeignException EXTEXN = new ForeignException("FORTEST", new IllegalArgumentException("FORTEST"));
+  final ForeignException EXTEXN2 = new ForeignException("FORTEST2", new IllegalArgumentException("FORTEST2"));
+
+  /**
+   * Tests that a dispatcher only dispatches only the first exception, and does not propagate
+   * subsequent exceptions.
+   */
+  @Test
+  public void testErrorPropagation() {
+    ForeignExceptionListener listener1 = Mockito.mock(ForeignExceptionListener.class);
+    ForeignExceptionListener listener2 = Mockito.mock(ForeignExceptionListener.class);
+    ForeignExceptionDispatcher dispatcher = new ForeignExceptionDispatcher();
+
+    // add the listeners
+    dispatcher.addListener(listener1);
+    dispatcher.addListener(listener2);
+
+    // create an artificial error
+    String message = "Some error";
+    dispatcher.receive(message, EXTEXN);
+
+    // make sure the listeners got the error
+    Mockito.verify(listener1, Mockito.times(1)).receive(message, EXTEXN);
+    Mockito.verify(listener2, Mockito.times(1)).receive(message, EXTEXN);
+
+    // make sure that we get an exception
+    try {
+      dispatcher.rethrowException();
+      fail("Monitor should have thrown an exception after getting error.");
+    } catch (ForeignException ex) {
+      assertTrue("Got an unexpected exception:" + ex, ex == EXTEXN);
+      LOG.debug("Got the testing exception!");
+    }
+
+    // push another error, which should be not be passed to listeners
+    message = "another error";
+    dispatcher.receive(message, EXTEXN2);
+    Mockito.verify(listener1, Mockito.never()).receive(message, EXTEXN2);
+    Mockito.verify(listener2, Mockito.never()).receive(message, EXTEXN2);
+  }
+
+  @Test
+  public void testSingleDispatcherWithTimer() {
+    ForeignExceptionListener listener1 = Mockito.mock(ForeignExceptionListener.class);
+    ForeignExceptionListener listener2 = Mockito.mock(ForeignExceptionListener.class);
+
+    ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher();
+
+    // add the listeners
+    monitor.addListener(listener1);
+    monitor.addListener(listener2);
+
+    TimeoutExceptionInjector timer = new TimeoutExceptionInjector(monitor, 1000);
+    timer.start();
+    timer.trigger();
+
+    assertTrue("Monitor didn't get timeout", monitor.hasException());
+
+    // verify that that we propagated the error
+    Mockito.verify(listener1).receive(Mockito.anyString(), Mockito.any(ForeignException.class));
+    Mockito.verify(listener2).receive(Mockito.anyString(), Mockito.any(ForeignException.class));
+  }
+
+  /**
+   * Test that the dispatcher can receive an error via the timer mechanism.
+   */
+  @Test
+  public void testAttemptTimer() {
+    ForeignExceptionListener listener1 = Mockito.mock(ForeignExceptionListener.class);
+    ForeignExceptionListener listener2 = Mockito.mock(ForeignExceptionListener.class);
+    ForeignExceptionDispatcher orchestrator = new ForeignExceptionDispatcher();
+
+    // add the listeners
+    orchestrator.addListener(listener1);
+    orchestrator.addListener(listener2);
+
+    // now create a timer and check for that error
+    TimeoutExceptionInjector timer = new TimeoutExceptionInjector(orchestrator, 1000);
+    timer.start();
+    timer.trigger();
+    // make sure that we got the timer error
+    Mockito.verify(listener1, Mockito.times(1)).receive(Mockito.anyString(),
+      Mockito.any(ForeignException.class));
+    Mockito.verify(listener2, Mockito.times(1)).receive(Mockito.anyString(),
+      Mockito.any(ForeignException.class));
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionSerialization.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionSerialization.java?rev=1445798&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionSerialization.java
(added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionSerialization.java
Wed Feb 13 18:21:26 2013
@@ -0,0 +1,82 @@
+/**
+ * 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.errorhandling;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+/**
+ * Test that we correctly serialize exceptions from a remote source
+ */
+@Category(SmallTests.class)
+public class TestForeignExceptionSerialization {
+  private static final String srcName = "someNode";
+
+  /**
+   * Verify that we get back similar stack trace information before an after serialization.
+   * @throws InvalidProtocolBufferException
+   */
+  @Test
+  public void testSimpleException() throws InvalidProtocolBufferException {
+    String data = "some bytes";
+    ForeignException in = new ForeignException("SRC", new IllegalArgumentException(data));
+    // check that we get the data back out
+    ForeignException e = ForeignException.deserialize(ForeignException.serialize(srcName,
in));
+    assertNotNull(e);
+
+    // now check that we get the right stack trace
+    StackTraceElement elem = new StackTraceElement(this.getClass().toString(), "method",
"file", 1);
+    in.setStackTrace(new StackTraceElement[] { elem });
+    e = ForeignException.deserialize(ForeignException.serialize(srcName, in));
+
+    assertNotNull(e);
+    assertEquals("Stack trace got corrupted", elem, e.getCause().getStackTrace()[0]);
+    assertEquals("Got an unexpectedly long stack trace", 1, e.getCause().getStackTrace().length);
+  }
+
+  /**
+   * Compare that a generic exception's stack trace has the same stack trace elements after
+   * serialization and deserialization
+   * @throws InvalidProtocolBufferException
+   */
+  @Test
+  public void testRemoteFromLocal() throws InvalidProtocolBufferException {
+    String errorMsg = "some message";
+    Exception generic = new Exception(errorMsg);
+    generic.printStackTrace();
+    assertTrue(generic.getMessage().contains(errorMsg));
+
+    ForeignException e = ForeignException.deserialize(ForeignException.serialize(srcName,
generic));
+    assertArrayEquals("Local stack trace got corrupted", generic.getStackTrace(), e.getCause().getStackTrace());
+
+    e.printStackTrace(); // should have ForeignException and source node in it.
+    assertTrue(e.getCause().getCause() == null);
+
+    // verify that original error message is present in Foreign exception message
+    assertTrue(e.getCause().getMessage().contains(errorMsg));
+  }
+
+}

Added: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java?rev=1445798&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java
(added)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java
Wed Feb 13 18:21:26 2013
@@ -0,0 +1,105 @@
+/**
+ * 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.errorhandling;
+
+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.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * Test the {@link TimeoutExceptionInjector} to ensure we fulfill contracts
+ */
+@Category(SmallTests.class)
+public class TestTimeoutExceptionInjector {
+
+  private static final Log LOG = LogFactory.getLog(TestTimeoutExceptionInjector.class);
+
+  /**
+   * Test that a manually triggered timer fires an exception.
+   */
+  @Test(timeout = 1000)
+  public void testTimerTrigger() {
+    final long time = 10000000; // pick a value that is very far in the future
+    ForeignExceptionListener listener = Mockito.mock(ForeignExceptionListener.class);
+    TimeoutExceptionInjector timer = new TimeoutExceptionInjector(listener, time);
+    timer.start();
+    timer.trigger();
+    Mockito.verify(listener, Mockito.times(1)).receive(Mockito.anyString(),
+      Mockito.any(ForeignException.class));
+  }
+
+  /**
+   * Test that a manually triggered exception with data fires with the data in receiveError.
+   */
+  @Test
+  public void testTimerPassesOnErrorInfo() {
+    final long time = 1000000;
+    ForeignExceptionListener listener = Mockito.mock(ForeignExceptionListener.class);
+    TimeoutExceptionInjector timer = new TimeoutExceptionInjector(listener, time);
+    timer.start();
+    timer.trigger();
+    Mockito.verify(listener).receive(Mockito.anyString(), Mockito.any(ForeignException.class));
+  }
+
+  /**
+   * Demonstrate TimeoutExceptionInjector semantics -- completion means no more exceptions
passed to
+   * error listener.
+   */
+  @Test(timeout = 1000)
+  public void testStartAfterComplete() throws InterruptedException {
+    final long time = 10;
+    ForeignExceptionListener listener = Mockito.mock(ForeignExceptionListener.class);
+    TimeoutExceptionInjector timer = new TimeoutExceptionInjector(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);
+  }
+
+  /**
+   * Demonstrate TimeoutExceptionInjector semantics -- triggering fires exception and completes
+   * the timer.
+   */
+  @Test(timeout = 1000)
+  public void testStartAfterTrigger() throws InterruptedException {
+    final long time = 10;
+    ForeignExceptionListener listener = Mockito.mock(ForeignExceptionListener.class);
+    TimeoutExceptionInjector timer = new TimeoutExceptionInjector(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)).receive(Mockito.anyString(),
+      Mockito.any(ForeignException.class));
+    Mockito.verifyNoMoreInteractions(listener);
+  }
+}



Mime
View raw message