HADOOP-8068. void methods can swallow exceptions when going through failover path. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1244628 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2012-02-15 18:20:11 +00:00
parent 011611c765
commit cdc8146bd4
8 changed files with 57 additions and 39 deletions

View File

@ -47,3 +47,5 @@ HADOOP-8038. Add 'ipc.client.connect.max.retries.on.timeouts' entry in
core-default.xml file. (Uma Maheswara Rao G via atm)
HADOOP-8041. Log a warning when a failover is first attempted (todd)
HADOOP-8068. void methods can swallow exceptions when going through failover path (todd)

View File

@ -90,12 +90,12 @@ class RetryInvocationHandler implements RpcInvocationHandler {
RetryAction action = policy.shouldRetry(e, retries++, invocationFailoverCount,
isMethodIdempotent);
if (action.action == RetryAction.RetryDecision.FAIL) {
LOG.warn("Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass() + ". Not retrying.", e);
if (!method.getReturnType().equals(Void.TYPE)) {
throw e; // non-void methods can't fail without an exception
if (action.reason != null) {
LOG.warn("Exception while invoking " +
currentProxy.getClass() + "." + method.getName() +
". Not retrying because " + action.reason, e);
}
return null;
throw e;
} else { // retry or failover
// avoid logging the failover if this is the first call on this
// proxy object, and we successfully achieve the failover without

View File

@ -54,14 +54,6 @@ public class RetryPolicies {
*/
public static final RetryPolicy TRY_ONCE_THEN_FAIL = new TryOnceThenFail();
/**
* <p>
* Try once, and fail silently for <code>void</code> methods, or by
* re-throwing the exception for non-<code>void</code> methods.
* </p>
*/
public static final RetryPolicy TRY_ONCE_DONT_FAIL = new TryOnceDontFail();
/**
* <p>
* Keep trying forever.
@ -152,12 +144,6 @@ public class RetryPolicies {
}
static class TryOnceThenFail implements RetryPolicy {
public RetryAction shouldRetry(Exception e, int retries, int failovers,
boolean isMethodIdempotent) throws Exception {
throw e;
}
}
static class TryOnceDontFail implements RetryPolicy {
public RetryAction shouldRetry(Exception e, int retries, int failovers,
boolean isMethodIdempotent) throws Exception {
return RetryAction.FAIL;
@ -185,7 +171,7 @@ public class RetryPolicies {
public RetryAction shouldRetry(Exception e, int retries, int failovers,
boolean isMethodIdempotent) throws Exception {
if (retries >= maxRetries) {
throw e;
return RetryAction.FAIL;
}
return new RetryAction(RetryAction.RetryDecision.RETRY,
timeUnit.toMillis(calculateSleepTime(retries)));
@ -325,9 +311,9 @@ public class RetryPolicies {
public RetryAction shouldRetry(Exception e, int retries,
int failovers, boolean isMethodIdempotent) throws Exception {
if (failovers >= maxFailovers) {
LOG.info("Failovers (" + failovers + ") exceeded maximum allowed ("
return new RetryAction(RetryAction.RetryDecision.FAIL, 0,
"failovers (" + failovers + ") exceeded maximum allowed ("
+ maxFailovers + ")");
return RetryAction.FAIL;
}
if (e instanceof ConnectException ||
@ -345,7 +331,9 @@ public class RetryPolicies {
if (isMethodIdempotent) {
return RetryAction.FAILOVER_AND_RETRY;
} else {
return RetryAction.FAIL;
return new RetryAction(RetryAction.RetryDecision.FAIL, 0,
"the invoked method is not idempotent, and unable to determine " +
"whether it was invoked");
}
} else {
return fallbackPolicy.shouldRetry(e, retries, failovers,

View File

@ -44,14 +44,20 @@ public interface RetryPolicy {
public final RetryDecision action;
public final long delayMillis;
public final String reason;
public RetryAction(RetryDecision action) {
this(action, 0);
this(action, 0, null);
}
public RetryAction(RetryDecision action, long delayTime) {
this(action, delayTime, null);
}
public RetryAction(RetryDecision action, long delayTime, String reason) {
this.action = action;
this.delayMillis = delayTime;
this.reason = reason;
}
public enum RetryDecision {

View File

@ -128,7 +128,7 @@ public class TestFailoverProxy {
new FlipFlopProxyProvider(UnreliableInterface.class,
new UnreliableImplementation("impl1"),
new UnreliableImplementation("impl2")),
RetryPolicies.TRY_ONCE_DONT_FAIL);
RetryPolicies.TRY_ONCE_THEN_FAIL);
unreliable.succeedsOnceThenFailsReturningString();
try {
@ -196,6 +196,27 @@ public class TestFailoverProxy {
assertEquals("impl2", unreliable.succeedsOnceThenFailsReturningStringIdempotent());
}
/**
* Test that if a non-idempotent void function is called, and there is an exception,
* the exception is properly propagated
*/
@Test
public void testExceptionPropagatedForNonIdempotentVoid() throws Exception {
UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
.create(UnreliableInterface.class,
new FlipFlopProxyProvider(UnreliableInterface.class,
new UnreliableImplementation("impl1", TypeOfExceptionToFailWith.IO_EXCEPTION),
new UnreliableImplementation("impl2", TypeOfExceptionToFailWith.UNRELIABLE_EXCEPTION)),
RetryPolicies.failoverOnNetworkException(1));
try {
unreliable.nonIdempotentVoidFailsIfIdentifierDoesntMatch("impl2");
fail("did not throw an exception");
} catch (Exception e) {
}
}
private static class SynchronizedUnreliableImplementation extends UnreliableImplementation {
private CountDownLatch methodLatch;

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.io.retry;
import static org.apache.hadoop.io.retry.RetryPolicies.RETRY_FOREVER;
import static org.apache.hadoop.io.retry.RetryPolicies.TRY_ONCE_DONT_FAIL;
import static org.apache.hadoop.io.retry.RetryPolicies.TRY_ONCE_THEN_FAIL;
import static org.apache.hadoop.io.retry.RetryPolicies.retryByException;
import static org.apache.hadoop.io.retry.RetryPolicies.retryByRemoteException;
@ -59,19 +58,6 @@ public class TestRetryProxy extends TestCase {
}
}
public void testTryOnceDontFail() throws UnreliableException {
UnreliableInterface unreliable = (UnreliableInterface)
RetryProxy.create(UnreliableInterface.class, unreliableImpl, TRY_ONCE_DONT_FAIL);
unreliable.alwaysSucceeds();
unreliable.failsOnceThenSucceeds();
try {
unreliable.failsOnceThenSucceedsWithReturnValue();
fail("Should fail");
} catch (UnreliableException e) {
// expected
}
}
public void testRetryForever() throws UnreliableException {
UnreliableInterface unreliable = (UnreliableInterface)
RetryProxy.create(UnreliableInterface.class, unreliableImpl, RETRY_FOREVER);

View File

@ -137,6 +137,18 @@ public class UnreliableImplementation implements UnreliableInterface {
}
}
@Override
public void nonIdempotentVoidFailsIfIdentifierDoesntMatch(String identifier)
throws UnreliableException, StandbyException, IOException {
if (this.identifier.equals(identifier)) {
return;
} else {
String message = "expected '" + this.identifier + "' but received '" +
identifier + "'";
throwAppropriateException(exceptionToFailWith, message);
}
}
private static void throwAppropriateException(TypeOfExceptionToFailWith eType,
String message) throws UnreliableException, StandbyException, IOException {
switch (eType) {

View File

@ -67,4 +67,7 @@ public interface UnreliableInterface {
@Idempotent
public String failsIfIdentifierDoesntMatch(String identifier)
throws UnreliableException, StandbyException, IOException;
void nonIdempotentVoidFailsIfIdentifierDoesntMatch(String identifier)
throws UnreliableException, StandbyException, IOException;
}