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 19:33:43 UTC
svn commit: r1445811 - in /hbase/branches/hbase-7290/hbase-server/src:
main/java/org/apache/hadoop/hbase/errorhandling/
test/java/org/apache/hadoop/hbase/errorhandling/
Author: jmhsieh
Date: Wed Feb 13 18:33:43 2013
New Revision: 1445811
URL: http://svn.apache.org/r1445811
Log:
HBASE-7452 Change ForeignExceptionListener#receive(String, FE) to only be #receive(FE)
Modified:
hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionDispatcher.java
hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionListener.java
hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutExceptionInjector.java
hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java
hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java
Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionDispatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionDispatcher.java?rev=1445811&r1=1445810&r2=1445811&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionDispatcher.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionDispatcher.java Wed Feb 13 18:33:43 2013
@@ -59,29 +59,21 @@ public class ForeignExceptionDispatcher
return name;
}
- public synchronized void receive(String message) {
- receive(new ForeignException(name, message));
- }
-
- public synchronized void receive(ForeignException e) {
- receive(e);
- }
-
@Override
- public synchronized void receive(String message, ForeignException e) {
+ public synchronized void receive(ForeignException e) {
// if we already have an exception, then ignore it
if (exception != null) return;
- LOG.debug(name + " accepting received error:" + message);
+ LOG.debug(name + " accepting received exception" , e);
// mark that we got the error
if (e != null) {
exception = e;
} else {
- exception = new ForeignException(name, message);
+ exception = new ForeignException(name, e);
}
// notify all the listeners
- dispatch(message, e);
+ dispatch(e);
}
@Override
@@ -108,11 +100,11 @@ public class ForeignExceptionDispatcher
* @param message human readable message passed to the listener
* @param e {@link ForeignException} containing the cause. Can be null.
*/
- private void dispatch(String message, ForeignException e) {
+ private void dispatch(ForeignException e) {
// update all the listeners with the passed error
LOG.debug(name + " Recieved error, notifying listeners...");
for (ForeignExceptionListener l: listeners) {
- l.receive(message, e);
+ l.receive(e);
}
}
Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionListener.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionListener.java?rev=1445811&r1=1445810&r2=1445811&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionListener.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignExceptionListener.java Wed Feb 13 18:33:43 2013
@@ -37,5 +37,5 @@ public interface ForeignExceptionListene
* @param message reason for the error
* @param e exception causing the error. Implementations must accept and handle null here.
*/
- public void receive(String message, ForeignException e);
+ public void receive(ForeignException e);
}
\ No newline at end of file
Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutExceptionInjector.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutExceptionInjector.java?rev=1445811&r1=1445810&r2=1445811&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutExceptionInjector.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutExceptionInjector.java Wed Feb 13 18:33:43 2013
@@ -69,7 +69,7 @@ public class TimeoutExceptionInjector {
TimeoutException tee = new TimeoutException(
"Timeout caused Foreign Exception", start, end, maxTime);
String source = "timer-" + timer;
- listener.receive("Timeout elapsed!", new ForeignException(source, tee));
+ listener.receive(new ForeignException(source, tee));
}
};
}
Modified: 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=1445811&r1=1445810&r2=1445811&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestForeignExceptionDispatcher.java Wed Feb 13 18:33:43 2013
@@ -56,12 +56,11 @@ public class TestForeignExceptionDispatc
dispatcher.addListener(listener2);
// create an artificial error
- String message = "Some error";
- dispatcher.receive(message, EXTEXN);
+ dispatcher.receive(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);
+ Mockito.verify(listener1, Mockito.times(1)).receive(EXTEXN);
+ Mockito.verify(listener2, Mockito.times(1)).receive(EXTEXN);
// make sure that we get an exception
try {
@@ -73,10 +72,9 @@ public class TestForeignExceptionDispatc
}
// 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);
+ dispatcher.receive(EXTEXN2);
+ Mockito.verify(listener1, Mockito.never()).receive(EXTEXN2);
+ Mockito.verify(listener2, Mockito.never()).receive(EXTEXN2);
}
@Test
@@ -97,8 +95,8 @@ public class TestForeignExceptionDispatc
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));
+ Mockito.verify(listener1).receive(Mockito.any(ForeignException.class));
+ Mockito.verify(listener2).receive(Mockito.any(ForeignException.class));
}
/**
@@ -119,9 +117,7 @@ public class TestForeignExceptionDispatc
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));
+ Mockito.verify(listener1, Mockito.times(1)).receive(Mockito.any(ForeignException.class));
+ Mockito.verify(listener2, Mockito.times(1)).receive(Mockito.any(ForeignException.class));
}
}
\ No newline at end of file
Modified: 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=1445811&r1=1445810&r2=1445811&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/errorhandling/TestTimeoutExceptionInjector.java Wed Feb 13 18:33:43 2013
@@ -44,8 +44,7 @@ public class TestTimeoutExceptionInjecto
TimeoutExceptionInjector timer = new TimeoutExceptionInjector(listener, time);
timer.start();
timer.trigger();
- Mockito.verify(listener, Mockito.times(1)).receive(Mockito.anyString(),
- Mockito.any(ForeignException.class));
+ Mockito.verify(listener, Mockito.times(1)).receive(Mockito.any(ForeignException.class));
}
/**
@@ -58,7 +57,7 @@ public class TestTimeoutExceptionInjecto
TimeoutExceptionInjector timer = new TimeoutExceptionInjector(listener, time);
timer.start();
timer.trigger();
- Mockito.verify(listener).receive(Mockito.anyString(), Mockito.any(ForeignException.class));
+ Mockito.verify(listener).receive(Mockito.any(ForeignException.class));
}
/**
@@ -98,8 +97,7 @@ public class TestTimeoutExceptionInjecto
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.verify(listener, Mockito.times(1)).receive(Mockito.any(ForeignException.class));
Mockito.verifyNoMoreInteractions(listener);
}
}