YARN-4288. Fixed RMProxy to retry on IOException from local host. Contributed by Junping Du
(cherry picked from commit c41699965e
)
This commit is contained in:
parent
e3fcb711d6
commit
7d109ef5b5
|
@ -139,7 +139,17 @@ public class RetryPolicies {
|
|||
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap) {
|
||||
return new RemoteExceptionDependentRetry(defaultPolicy, exceptionToPolicyMap);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A retry policy for exceptions other than RemoteException.
|
||||
*/
|
||||
public static final RetryPolicy retryOtherThanRemoteException(
|
||||
RetryPolicy defaultPolicy,
|
||||
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap) {
|
||||
return new OtherThanRemoteExceptionDependentRetry(defaultPolicy,
|
||||
exceptionToPolicyMap);
|
||||
}
|
||||
|
||||
public static final RetryPolicy failoverOnNetworkException(int maxFailovers) {
|
||||
return failoverOnNetworkException(TRY_ONCE_THEN_FAIL, maxFailovers);
|
||||
}
|
||||
|
@ -489,7 +499,37 @@ public class RetryPolicies {
|
|||
return policy.shouldRetry(e, retries, failovers, isIdempotentOrAtMostOnce);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static class OtherThanRemoteExceptionDependentRetry implements RetryPolicy {
|
||||
|
||||
private RetryPolicy defaultPolicy;
|
||||
private Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap;
|
||||
|
||||
public OtherThanRemoteExceptionDependentRetry(RetryPolicy defaultPolicy,
|
||||
Map<Class<? extends Exception>,
|
||||
RetryPolicy> exceptionToPolicyMap) {
|
||||
this.defaultPolicy = defaultPolicy;
|
||||
this.exceptionToPolicyMap = exceptionToPolicyMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RetryAction shouldRetry(Exception e, int retries, int failovers,
|
||||
boolean isIdempotentOrAtMostOnce) throws Exception {
|
||||
RetryPolicy policy = null;
|
||||
// ignore Remote Exception
|
||||
if (e instanceof RemoteException) {
|
||||
// do nothing
|
||||
} else {
|
||||
policy = exceptionToPolicyMap.get(e.getClass());
|
||||
}
|
||||
if (policy == null) {
|
||||
policy = defaultPolicy;
|
||||
}
|
||||
return policy.shouldRetry(
|
||||
e, retries, failovers, isIdempotentOrAtMostOnce);
|
||||
}
|
||||
}
|
||||
|
||||
static class ExponentialBackoffRetry extends RetryLimited {
|
||||
|
||||
public ExponentialBackoffRetry(
|
||||
|
|
|
@ -22,6 +22,7 @@ import static org.apache.hadoop.io.retry.RetryPolicies.RETRY_FOREVER;
|
|||
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;
|
||||
import static org.apache.hadoop.io.retry.RetryPolicies.retryOtherThanRemoteException;
|
||||
import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithFixedSleep;
|
||||
import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithProportionalSleep;
|
||||
import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumTimeWithFixedSleep;
|
||||
|
@ -29,6 +30,7 @@ import static org.apache.hadoop.io.retry.RetryPolicies.retryForeverWithFixedSlee
|
|||
import static org.apache.hadoop.io.retry.RetryPolicies.exponentialBackoffRetry;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
|
@ -213,8 +215,29 @@ public class TestRetryProxy {
|
|||
} catch (RemoteException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRetryOtherThanRemoteException() throws Throwable {
|
||||
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
|
||||
Collections.<Class<? extends Exception>, RetryPolicy>singletonMap(
|
||||
IOException.class, RETRY_FOREVER);
|
||||
|
||||
UnreliableInterface unreliable = (UnreliableInterface)
|
||||
RetryProxy.create(UnreliableInterface.class, unreliableImpl,
|
||||
retryOtherThanRemoteException(TRY_ONCE_THEN_FAIL,
|
||||
exceptionToPolicyMap));
|
||||
// should retry with local IOException.
|
||||
unreliable.failsOnceWithIOException();
|
||||
try {
|
||||
// won't get retry on remote exception
|
||||
unreliable.failsOnceWithRemoteException();
|
||||
fail("Should fail");
|
||||
} catch (RemoteException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRetryInterruptible() throws Throwable {
|
||||
final UnreliableInterface unreliable = (UnreliableInterface)
|
||||
|
|
|
@ -26,6 +26,8 @@ class UnreliableImplementation implements UnreliableInterface {
|
|||
|
||||
private int failsOnceInvocationCount,
|
||||
failsOnceWithValueInvocationCount,
|
||||
failsOnceIOExceptionInvocationCount,
|
||||
failsOnceRemoteExceptionInvocationCount,
|
||||
failsTenTimesInvocationCount,
|
||||
succeedsOnceThenFailsCount,
|
||||
succeedsOnceThenFailsIdempotentCount,
|
||||
|
@ -89,6 +91,21 @@ class UnreliableImplementation implements UnreliableInterface {
|
|||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failsOnceWithIOException() throws IOException {
|
||||
if (failsOnceIOExceptionInvocationCount++ == 0) {
|
||||
throw new IOException("test exception for failsOnceWithIOException");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failsOnceWithRemoteException() throws RemoteException {
|
||||
if (failsOnceRemoteExceptionInvocationCount++ == 0) {
|
||||
throw new RemoteException(IOException.class.getName(),
|
||||
"test exception for failsOnceWithRemoteException");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failsTenTimesThenSucceeds() throws UnreliableException {
|
||||
if (failsTenTimesInvocationCount++ < 10) {
|
||||
|
|
|
@ -54,6 +54,9 @@ public interface UnreliableInterface {
|
|||
void alwaysFailsWithFatalException() throws FatalException;
|
||||
void alwaysFailsWithRemoteFatalException() throws RemoteException;
|
||||
|
||||
void failsOnceWithIOException() throws IOException;
|
||||
void failsOnceWithRemoteException() throws RemoteException;
|
||||
|
||||
void failsOnceThenSucceeds() throws UnreliableException;
|
||||
boolean failsOnceThenSucceedsWithReturnValue() throws UnreliableException;
|
||||
|
||||
|
|
|
@ -982,6 +982,9 @@ Release 2.8.0 - UNRELEASED
|
|||
YARN-4130. Duplicate declaration of ApplicationId in RMAppManager#submitApplication method.
|
||||
(Kai Sasaki via rohithsharmaks)
|
||||
|
||||
YARN-4288. Fixed RMProxy to retry on IOException from local host.
|
||||
(Junping Du via jianhe)
|
||||
|
||||
Release 2.7.2 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -250,8 +250,10 @@ public class RMProxy<T> {
|
|||
exceptionToPolicyMap.put(ConnectTimeoutException.class, retryPolicy);
|
||||
exceptionToPolicyMap.put(RetriableException.class, retryPolicy);
|
||||
exceptionToPolicyMap.put(SocketException.class, retryPolicy);
|
||||
|
||||
return RetryPolicies.retryByException(
|
||||
// YARN-4288: local IOException is also possible.
|
||||
exceptionToPolicyMap.put(IOException.class, retryPolicy);
|
||||
// Not retry on remote IO exception.
|
||||
return RetryPolicies.retryOtherThanRemoteException(
|
||||
RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue