|
@@ -17,10 +17,15 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.io.retry;
|
|
|
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.Map;
|
|
|
import java.util.Random;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.Map.Entry;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
+import org.apache.hadoop.ipc.RemoteException;
|
|
|
+
|
|
|
/**
|
|
|
* <p>
|
|
|
* A collection of useful implementations of {@link RetryPolicy}.
|
|
@@ -83,10 +88,19 @@ public class RetryPolicies {
|
|
|
return new RetryUpToMaximumCountWithProportionalSleep(maxRetries, sleepTime, timeUnit);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * <p>
|
|
|
+ * Keep trying a limited number of times, waiting a growing amount of time between attempts,
|
|
|
+ * and then fail by re-throwing the exception.
|
|
|
+ * The time between attempts is <code>sleepTime</code> mutliplied by a random
|
|
|
+ * number in the range of [0, 2 to the number of retries)
|
|
|
+ * </p>
|
|
|
+ */
|
|
|
public static final RetryPolicy exponentialBackoffRetry(
|
|
|
int maxRetries, long sleepTime, TimeUnit timeUnit) {
|
|
|
return new ExponentialBackoffRetry(maxRetries, sleepTime, timeUnit);
|
|
|
}
|
|
|
+
|
|
|
/**
|
|
|
* <p>
|
|
|
* Set a default policy with some explicit handlers for specific exceptions.
|
|
@@ -97,6 +111,18 @@ public class RetryPolicies {
|
|
|
return new ExceptionDependentRetry(defaultPolicy, exceptionToPolicyMap);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * <p>
|
|
|
+ * A retry policy for RemoteException
|
|
|
+ * Set a default policy with some explicit handlers for specific exceptions.
|
|
|
+ * </p>
|
|
|
+ */
|
|
|
+ public static final RetryPolicy retryByRemoteException(
|
|
|
+ RetryPolicy defaultPolicy,
|
|
|
+ Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap) {
|
|
|
+ return new RemoteExceptionDependentRetry(defaultPolicy, exceptionToPolicyMap);
|
|
|
+ }
|
|
|
+
|
|
|
static class TryOnceThenFail implements RetryPolicy {
|
|
|
public boolean shouldRetry(Exception e, int retries) throws Exception {
|
|
|
throw e;
|
|
@@ -189,6 +215,35 @@ public class RetryPolicies {
|
|
|
|
|
|
}
|
|
|
|
|
|
+ static class RemoteExceptionDependentRetry implements RetryPolicy {
|
|
|
+
|
|
|
+ RetryPolicy defaultPolicy;
|
|
|
+ Map<String, RetryPolicy> exceptionNameToPolicyMap;
|
|
|
+
|
|
|
+ public RemoteExceptionDependentRetry(RetryPolicy defaultPolicy,
|
|
|
+ Map<Class<? extends Exception>,
|
|
|
+ RetryPolicy> exceptionToPolicyMap) {
|
|
|
+ this.defaultPolicy = defaultPolicy;
|
|
|
+ this.exceptionNameToPolicyMap = new HashMap<String, RetryPolicy>();
|
|
|
+ for (Entry<Class<? extends Exception>, RetryPolicy> e :
|
|
|
+ exceptionToPolicyMap.entrySet()) {
|
|
|
+ exceptionNameToPolicyMap.put(e.getKey().getName(), e.getValue());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public boolean shouldRetry(Exception e, int retries) throws Exception {
|
|
|
+ RetryPolicy policy = null;
|
|
|
+ if (e instanceof RemoteException) {
|
|
|
+ policy = exceptionNameToPolicyMap.get(
|
|
|
+ ((RemoteException) e).getClassName());
|
|
|
+ }
|
|
|
+ if (policy == null) {
|
|
|
+ policy = defaultPolicy;
|
|
|
+ }
|
|
|
+ return policy.shouldRetry(e, retries);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
static class ExponentialBackoffRetry extends RetryLimited {
|
|
|
private Random r = new Random();
|
|
|
public ExponentialBackoffRetry(
|