HADOOP-10585. Retry polices ignore interrupted exceptions (Daryn Sharp via jeagles)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1594270 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a7e29582c3
commit
8379c0c944
|
@ -135,6 +135,9 @@ Release 2.5.0 - UNRELEASED
|
||||||
because groups stored in Set and ArrayList are compared.
|
because groups stored in Set and ArrayList are compared.
|
||||||
(Mit Desai via kihwal)
|
(Mit Desai via kihwal)
|
||||||
|
|
||||||
|
HADOOP-10585. Retry polices ignore interrupted exceptions (Daryn Sharp via
|
||||||
|
jeagles)
|
||||||
|
|
||||||
Release 2.4.1 - UNRELEASED
|
Release 2.4.1 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -151,7 +151,7 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (action.delayMillis > 0) {
|
if (action.delayMillis > 0) {
|
||||||
ThreadUtil.sleepAtLeastIgnoreInterrupts(action.delayMillis);
|
Thread.sleep(action.delayMillis);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY) {
|
if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY) {
|
||||||
|
|
|
@ -26,27 +26,37 @@ import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWith
|
||||||
import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithProportionalSleep;
|
import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithProportionalSleep;
|
||||||
import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumTimeWithFixedSleep;
|
import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumTimeWithFixedSleep;
|
||||||
import static org.apache.hadoop.io.retry.RetryPolicies.exponentialBackoffRetry;
|
import static org.apache.hadoop.io.retry.RetryPolicies.exponentialBackoffRetry;
|
||||||
|
import static org.junit.Assert.*;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import junit.framework.TestCase;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.retry.UnreliableInterface.FatalException;
|
import org.apache.hadoop.io.retry.UnreliableInterface.FatalException;
|
||||||
import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException;
|
import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException;
|
||||||
import org.apache.hadoop.ipc.ProtocolTranslator;
|
import org.apache.hadoop.ipc.ProtocolTranslator;
|
||||||
import org.apache.hadoop.ipc.RemoteException;
|
import org.apache.hadoop.ipc.RemoteException;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
public class TestRetryProxy extends TestCase {
|
import java.lang.reflect.UndeclaredThrowableException;
|
||||||
|
|
||||||
|
public class TestRetryProxy {
|
||||||
|
|
||||||
private UnreliableImplementation unreliableImpl;
|
private UnreliableImplementation unreliableImpl;
|
||||||
|
|
||||||
@Override
|
@Before
|
||||||
protected void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
unreliableImpl = new UnreliableImplementation();
|
unreliableImpl = new UnreliableImplementation();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
public void testTryOnceThenFail() throws UnreliableException {
|
public void testTryOnceThenFail() throws UnreliableException {
|
||||||
UnreliableInterface unreliable = (UnreliableInterface)
|
UnreliableInterface unreliable = (UnreliableInterface)
|
||||||
RetryProxy.create(UnreliableInterface.class, unreliableImpl, TRY_ONCE_THEN_FAIL);
|
RetryProxy.create(UnreliableInterface.class, unreliableImpl, TRY_ONCE_THEN_FAIL);
|
||||||
|
@ -62,6 +72,7 @@ public class TestRetryProxy extends TestCase {
|
||||||
/**
|
/**
|
||||||
* Test for {@link RetryInvocationHandler#isRpcInvocation(Object)}
|
* Test for {@link RetryInvocationHandler#isRpcInvocation(Object)}
|
||||||
*/
|
*/
|
||||||
|
@Test
|
||||||
public void testRpcInvocation() throws Exception {
|
public void testRpcInvocation() throws Exception {
|
||||||
// For a proxy method should return true
|
// For a proxy method should return true
|
||||||
final UnreliableInterface unreliable = (UnreliableInterface)
|
final UnreliableInterface unreliable = (UnreliableInterface)
|
||||||
|
@ -91,6 +102,7 @@ public class TestRetryProxy extends TestCase {
|
||||||
assertFalse(RetryInvocationHandler.isRpcInvocation(new Object()));
|
assertFalse(RetryInvocationHandler.isRpcInvocation(new Object()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
public void testRetryForever() throws UnreliableException {
|
public void testRetryForever() throws UnreliableException {
|
||||||
UnreliableInterface unreliable = (UnreliableInterface)
|
UnreliableInterface unreliable = (UnreliableInterface)
|
||||||
RetryProxy.create(UnreliableInterface.class, unreliableImpl, RETRY_FOREVER);
|
RetryProxy.create(UnreliableInterface.class, unreliableImpl, RETRY_FOREVER);
|
||||||
|
@ -99,6 +111,7 @@ public class TestRetryProxy extends TestCase {
|
||||||
unreliable.failsTenTimesThenSucceeds();
|
unreliable.failsTenTimesThenSucceeds();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
public void testRetryUpToMaximumCountWithFixedSleep() throws UnreliableException {
|
public void testRetryUpToMaximumCountWithFixedSleep() throws UnreliableException {
|
||||||
UnreliableInterface unreliable = (UnreliableInterface)
|
UnreliableInterface unreliable = (UnreliableInterface)
|
||||||
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
||||||
|
@ -113,6 +126,7 @@ public class TestRetryProxy extends TestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
public void testRetryUpToMaximumTimeWithFixedSleep() throws UnreliableException {
|
public void testRetryUpToMaximumTimeWithFixedSleep() throws UnreliableException {
|
||||||
UnreliableInterface unreliable = (UnreliableInterface)
|
UnreliableInterface unreliable = (UnreliableInterface)
|
||||||
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
||||||
|
@ -127,6 +141,7 @@ public class TestRetryProxy extends TestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
public void testRetryUpToMaximumCountWithProportionalSleep() throws UnreliableException {
|
public void testRetryUpToMaximumCountWithProportionalSleep() throws UnreliableException {
|
||||||
UnreliableInterface unreliable = (UnreliableInterface)
|
UnreliableInterface unreliable = (UnreliableInterface)
|
||||||
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
||||||
|
@ -141,6 +156,7 @@ public class TestRetryProxy extends TestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
public void testExponentialRetry() throws UnreliableException {
|
public void testExponentialRetry() throws UnreliableException {
|
||||||
UnreliableInterface unreliable = (UnreliableInterface)
|
UnreliableInterface unreliable = (UnreliableInterface)
|
||||||
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
||||||
|
@ -155,6 +171,7 @@ public class TestRetryProxy extends TestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
public void testRetryByException() throws UnreliableException {
|
public void testRetryByException() throws UnreliableException {
|
||||||
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
|
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
|
||||||
Collections.<Class<? extends Exception>, RetryPolicy>singletonMap(FatalException.class, TRY_ONCE_THEN_FAIL);
|
Collections.<Class<? extends Exception>, RetryPolicy>singletonMap(FatalException.class, TRY_ONCE_THEN_FAIL);
|
||||||
|
@ -171,6 +188,7 @@ public class TestRetryProxy extends TestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
public void testRetryByRemoteException() {
|
public void testRetryByRemoteException() {
|
||||||
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
|
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
|
||||||
Collections.<Class<? extends Exception>, RetryPolicy>singletonMap(FatalException.class, TRY_ONCE_THEN_FAIL);
|
Collections.<Class<? extends Exception>, RetryPolicy>singletonMap(FatalException.class, TRY_ONCE_THEN_FAIL);
|
||||||
|
@ -186,4 +204,35 @@ public class TestRetryProxy extends TestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRetryInterruptible() throws Throwable {
|
||||||
|
final UnreliableInterface unreliable = (UnreliableInterface)
|
||||||
|
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
||||||
|
retryUpToMaximumTimeWithFixedSleep(10, 10, TimeUnit.SECONDS));
|
||||||
|
|
||||||
|
final CountDownLatch latch = new CountDownLatch(1);
|
||||||
|
final AtomicReference<Thread> futureThread = new AtomicReference<Thread>();
|
||||||
|
ExecutorService exec = Executors.newSingleThreadExecutor();
|
||||||
|
Future<Throwable> future = exec.submit(new Callable<Throwable>(){
|
||||||
|
@Override
|
||||||
|
public Throwable call() throws Exception {
|
||||||
|
futureThread.set(Thread.currentThread());
|
||||||
|
latch.countDown();
|
||||||
|
try {
|
||||||
|
unreliable.alwaysFailsWithFatalException();
|
||||||
|
} catch (UndeclaredThrowableException ute) {
|
||||||
|
return ute.getCause();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
latch.await();
|
||||||
|
Thread.sleep(1000); // time to fail and sleep
|
||||||
|
assertTrue(futureThread.get().isAlive());
|
||||||
|
futureThread.get().interrupt();
|
||||||
|
Throwable e = future.get(1, TimeUnit.SECONDS); // should return immediately
|
||||||
|
assertNotNull(e);
|
||||||
|
assertEquals(InterruptedException.class, e.getClass());
|
||||||
|
assertEquals("sleep interrupted", e.getMessage());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue