HADOOP-8041. Log a warning when a failover is first attempted. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1242441 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2012-02-09 18:25:44 +00:00
parent 5e26de982b
commit 1b4c990b61
2 changed files with 21 additions and 8 deletions

View File

@ -45,3 +45,5 @@ ready before failing over. (eli)
HADOOP-8038. Add 'ipc.client.connect.max.retries.on.timeouts' entry in
core-default.xml file. (Uma Maheswara Rao G via atm)
HADOOP-8041. Log a warning when a failover is first attempted (todd)

View File

@ -39,6 +39,7 @@ class RetryInvocationHandler implements RpcInvocationHandler {
* The number of times the associated proxyProvider has ever been failed over.
*/
private long proxyProviderFailoverCount = 0;
private volatile boolean hasMadeASuccessfulCall = false;
private RetryPolicy defaultPolicy;
private Map<String,RetryPolicy> methodNameToPolicyMap;
@ -79,7 +80,9 @@ public Object invoke(Object proxy, Method method, Object[] args)
invocationAttemptFailoverCount = proxyProviderFailoverCount;
}
try {
return invokeMethod(method, args);
Object ret = invokeMethod(method, args);
hasMadeASuccessfulCall = true;
return ret;
} catch (Exception e) {
boolean isMethodIdempotent = proxyProvider.getInterface()
.getMethod(method.getName(), method.getParameterTypes())
@ -94,12 +97,20 @@ public Object invoke(Object proxy, Method method, Object[] args)
}
return null;
} else { // retry or failover
if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY) {
// avoid logging the failover if this is the first call on this
// proxy object, and we successfully achieve the failover without
// any flip-flopping
boolean worthLogging =
!(invocationFailoverCount == 0 && !hasMadeASuccessfulCall);
worthLogging |= LOG.isDebugEnabled();
if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY &&
worthLogging) {
String msg = "Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass()
+ " after " + invocationFailoverCount + " fail over attempts."
+ " Trying to fail over " + formatSleepMessage(action.delayMillis);
+ " of class " + currentProxy.getClass().getSimpleName();
if (invocationFailoverCount > 0) {
msg += " after " + invocationFailoverCount + " fail over attempts";
}
msg += ". Trying to fail over " + formatSleepMessage(action.delayMillis);
if (LOG.isDebugEnabled()) {
LOG.debug(msg, e);
} else {
@ -108,8 +119,8 @@ public Object invoke(Object proxy, Method method, Object[] args)
} else {
if(LOG.isDebugEnabled()) {
LOG.debug("Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass() + ". Retrying " +
formatSleepMessage(action.delayMillis), e);
+ " of class " + currentProxy.getClass().getSimpleName() +
". Retrying " + formatSleepMessage(action.delayMillis), e);
}
}