HADOOP-12386. RetryPolicies.RETRY_FOREVER should be able to specify a retry interval. (Sunil G via wangda)
This commit is contained in:
parent
ee4ee6af6a
commit
6b97fa6652
|
@ -1072,6 +1072,9 @@ Release 2.8.0 - UNRELEASED
|
|||
HADOOP-12087. [JDK8] Fix javadoc errors caused by incorrect or illegal
|
||||
tags. (Akira AJISAKA via stevel).
|
||||
|
||||
HADOOP-12386. RetryPolicies.RETRY_FOREVER should be able to specify a
|
||||
retry interval. (Sunil G via wangda)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HADOOP-12051. ProtobufRpcEngine.invoke() should use Exception.toString()
|
||||
|
|
|
@ -63,6 +63,17 @@ public class RetryPolicies {
|
|||
*/
|
||||
public static final RetryPolicy RETRY_FOREVER = new RetryForever();
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Keep trying forever with a fixed time between attempts.
|
||||
* </p>
|
||||
*/
|
||||
public static final RetryPolicy retryForeverWithFixedSleep(long sleepTime,
|
||||
TimeUnit timeUnit) {
|
||||
return new RetryUpToMaximumCountWithFixedSleep(Integer.MAX_VALUE,
|
||||
sleepTime, timeUnit);
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Keep trying a limited number of times, waiting a fixed time between attempts,
|
||||
|
@ -151,7 +162,7 @@ public class RetryPolicies {
|
|||
return new FailoverOnNetworkExceptionRetry(fallbackPolicy, maxFailovers,
|
||||
maxRetries, delayMillis, maxDelayBase);
|
||||
}
|
||||
|
||||
|
||||
static class TryOnceThenFail implements RetryPolicy {
|
||||
@Override
|
||||
public RetryAction shouldRetry(Exception e, int retries, int failovers,
|
||||
|
|
|
@ -25,6 +25,7 @@ import static org.apache.hadoop.io.retry.RetryPolicies.retryByRemoteException;
|
|||
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;
|
||||
import static org.apache.hadoop.io.retry.RetryPolicies.retryForeverWithFixedSleep;
|
||||
import static org.apache.hadoop.io.retry.RetryPolicies.exponentialBackoffRetry;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
|
@ -110,7 +111,17 @@ public class TestRetryProxy {
|
|||
unreliable.failsOnceThenSucceeds();
|
||||
unreliable.failsTenTimesThenSucceeds();
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testRetryForeverWithFixedSleep() throws UnreliableException {
|
||||
UnreliableInterface unreliable = (UnreliableInterface) RetryProxy.create(
|
||||
UnreliableInterface.class, unreliableImpl,
|
||||
retryForeverWithFixedSleep(1, TimeUnit.MILLISECONDS));
|
||||
unreliable.alwaysSucceeds();
|
||||
unreliable.failsOnceThenSucceeds();
|
||||
unreliable.failsTenTimesThenSucceeds();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRetryUpToMaximumCountWithFixedSleep() throws UnreliableException {
|
||||
UnreliableInterface unreliable = (UnreliableInterface)
|
||||
|
|
Loading…
Reference in New Issue