|
@@ -17,14 +17,21 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.io.retry;
|
|
package org.apache.hadoop.io.retry;
|
|
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
|
+import java.net.ConnectException;
|
|
|
|
+import java.net.NoRouteToHostException;
|
|
|
|
+import java.net.SocketException;
|
|
|
|
+import java.net.UnknownHostException;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.Random;
|
|
import java.util.Random;
|
|
-import java.util.Set;
|
|
|
|
import java.util.Map.Entry;
|
|
import java.util.Map.Entry;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
|
|
+import org.apache.hadoop.ipc.StandbyException;
|
|
|
|
|
|
/**
|
|
/**
|
|
* <p>
|
|
* <p>
|
|
@@ -33,6 +40,8 @@ import org.apache.hadoop.ipc.RemoteException;
|
|
*/
|
|
*/
|
|
public class RetryPolicies {
|
|
public class RetryPolicies {
|
|
|
|
|
|
|
|
+ public static final Log LOG = LogFactory.getLog(RetryPolicies.class);
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* <p>
|
|
* <p>
|
|
* Try once, and fail by re-throwing the exception.
|
|
* Try once, and fail by re-throwing the exception.
|
|
@@ -122,20 +131,32 @@ public class RetryPolicies {
|
|
return new RemoteExceptionDependentRetry(defaultPolicy, exceptionToPolicyMap);
|
|
return new RemoteExceptionDependentRetry(defaultPolicy, exceptionToPolicyMap);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public static final RetryPolicy failoverOnNetworkException(int maxFailovers) {
|
|
|
|
+ return failoverOnNetworkException(TRY_ONCE_THEN_FAIL, maxFailovers);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static final RetryPolicy failoverOnNetworkException(
|
|
|
|
+ RetryPolicy fallbackPolicy, int maxFailovers) {
|
|
|
|
+ return new FailoverOnNetworkExceptionRetry(fallbackPolicy, maxFailovers);
|
|
|
|
+ }
|
|
|
|
+
|
|
static class TryOnceThenFail implements RetryPolicy {
|
|
static class TryOnceThenFail implements RetryPolicy {
|
|
- public boolean shouldRetry(Exception e, int retries) throws Exception {
|
|
|
|
|
|
+ public RetryAction shouldRetry(Exception e, int retries, int failovers,
|
|
|
|
+ boolean isMethodIdempotent) throws Exception {
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
static class TryOnceDontFail implements RetryPolicy {
|
|
static class TryOnceDontFail implements RetryPolicy {
|
|
- public boolean shouldRetry(Exception e, int retries) throws Exception {
|
|
|
|
- return false;
|
|
|
|
|
|
+ public RetryAction shouldRetry(Exception e, int retries, int failovers,
|
|
|
|
+ boolean isMethodIdempotent) throws Exception {
|
|
|
|
+ return RetryAction.FAIL;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
static class RetryForever implements RetryPolicy {
|
|
static class RetryForever implements RetryPolicy {
|
|
- public boolean shouldRetry(Exception e, int retries) throws Exception {
|
|
|
|
- return true;
|
|
|
|
|
|
+ public RetryAction shouldRetry(Exception e, int retries, int failovers,
|
|
|
|
+ boolean isMethodIdempotent) throws Exception {
|
|
|
|
+ return RetryAction.RETRY;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -150,7 +171,8 @@ public class RetryPolicies {
|
|
this.timeUnit = timeUnit;
|
|
this.timeUnit = timeUnit;
|
|
}
|
|
}
|
|
|
|
|
|
- public boolean shouldRetry(Exception e, int retries) throws Exception {
|
|
|
|
|
|
+ public RetryAction shouldRetry(Exception e, int retries, int failovers,
|
|
|
|
+ boolean isMethodIdempotent) throws Exception {
|
|
if (retries >= maxRetries) {
|
|
if (retries >= maxRetries) {
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
@@ -159,7 +181,7 @@ public class RetryPolicies {
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
// retry
|
|
// retry
|
|
}
|
|
}
|
|
- return true;
|
|
|
|
|
|
+ return RetryAction.RETRY;
|
|
}
|
|
}
|
|
|
|
|
|
protected abstract long calculateSleepTime(int retries);
|
|
protected abstract long calculateSleepTime(int retries);
|
|
@@ -204,12 +226,13 @@ public class RetryPolicies {
|
|
this.exceptionToPolicyMap = exceptionToPolicyMap;
|
|
this.exceptionToPolicyMap = exceptionToPolicyMap;
|
|
}
|
|
}
|
|
|
|
|
|
- public boolean shouldRetry(Exception e, int retries) throws Exception {
|
|
|
|
|
|
+ public RetryAction shouldRetry(Exception e, int retries, int failovers,
|
|
|
|
+ boolean isMethodIdempotent) throws Exception {
|
|
RetryPolicy policy = exceptionToPolicyMap.get(e.getClass());
|
|
RetryPolicy policy = exceptionToPolicyMap.get(e.getClass());
|
|
if (policy == null) {
|
|
if (policy == null) {
|
|
policy = defaultPolicy;
|
|
policy = defaultPolicy;
|
|
}
|
|
}
|
|
- return policy.shouldRetry(e, retries);
|
|
|
|
|
|
+ return policy.shouldRetry(e, retries, failovers, isMethodIdempotent);
|
|
}
|
|
}
|
|
|
|
|
|
}
|
|
}
|
|
@@ -230,7 +253,8 @@ public class RetryPolicies {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- public boolean shouldRetry(Exception e, int retries) throws Exception {
|
|
|
|
|
|
+ public RetryAction shouldRetry(Exception e, int retries, int failovers,
|
|
|
|
+ boolean isMethodIdempotent) throws Exception {
|
|
RetryPolicy policy = null;
|
|
RetryPolicy policy = null;
|
|
if (e instanceof RemoteException) {
|
|
if (e instanceof RemoteException) {
|
|
policy = exceptionNameToPolicyMap.get(
|
|
policy = exceptionNameToPolicyMap.get(
|
|
@@ -239,7 +263,7 @@ public class RetryPolicies {
|
|
if (policy == null) {
|
|
if (policy == null) {
|
|
policy = defaultPolicy;
|
|
policy = defaultPolicy;
|
|
}
|
|
}
|
|
- return policy.shouldRetry(e, retries);
|
|
|
|
|
|
+ return policy.shouldRetry(e, retries, failovers, isMethodIdempotent);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -255,4 +279,55 @@ public class RetryPolicies {
|
|
return sleepTime*r.nextInt(1<<(retries+1));
|
|
return sleepTime*r.nextInt(1<<(retries+1));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /*
|
|
|
|
+ * Fail over and retry in the case of:
|
|
|
|
+ * Remote StandbyException (server is up, but is not the active server)
|
|
|
|
+ * Immediate socket exceptions (e.g. no route to host, econnrefused)
|
|
|
|
+ * Socket exceptions after initial connection when operation is idempotent
|
|
|
|
+ *
|
|
|
|
+ * Fail immediately in the case of:
|
|
|
|
+ * Socket exceptions after initial connection when operation is not idempotent
|
|
|
|
+ *
|
|
|
|
+ * Fall back on underlying retry policy otherwise.
|
|
|
|
+ */
|
|
|
|
+ static class FailoverOnNetworkExceptionRetry implements RetryPolicy {
|
|
|
|
+
|
|
|
|
+ private RetryPolicy fallbackPolicy;
|
|
|
|
+ private int maxFailovers;
|
|
|
|
+
|
|
|
|
+ public FailoverOnNetworkExceptionRetry(RetryPolicy fallbackPolicy,
|
|
|
|
+ int maxFailovers) {
|
|
|
|
+ this.fallbackPolicy = fallbackPolicy;
|
|
|
|
+ this.maxFailovers = maxFailovers;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public RetryAction shouldRetry(Exception e, int retries,
|
|
|
|
+ int failovers, boolean isMethodIdempotent) throws Exception {
|
|
|
|
+ if (failovers >= maxFailovers) {
|
|
|
|
+ LOG.info("Failovers (" + failovers + ") exceeded maximum allowed ("
|
|
|
|
+ + maxFailovers + ")");
|
|
|
|
+ return RetryAction.FAIL;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (e instanceof ConnectException ||
|
|
|
|
+ e instanceof NoRouteToHostException ||
|
|
|
|
+ e instanceof UnknownHostException ||
|
|
|
|
+ e instanceof StandbyException) {
|
|
|
|
+ return RetryAction.FAILOVER_AND_RETRY;
|
|
|
|
+ } else if (e instanceof SocketException ||
|
|
|
|
+ e instanceof IOException) {
|
|
|
|
+ if (isMethodIdempotent) {
|
|
|
|
+ return RetryAction.FAILOVER_AND_RETRY;
|
|
|
|
+ } else {
|
|
|
|
+ return RetryAction.FAIL;
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ return fallbackPolicy.shouldRetry(e, retries, failovers,
|
|
|
|
+ isMethodIdempotent);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ }
|
|
}
|
|
}
|