HADOOP-7922. Improve some logging for client IPC failovers and StandbyExceptions. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1214082 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2011-12-14 07:51:30 +00:00
parent 9818091a66
commit 9cf3e0805f
3 changed files with 34 additions and 9 deletions

View File

@ -10,3 +10,6 @@ HADOOP-7774. HA: Administrative CLI to control HA daemons. (todd)
HADOOP-7896. HA: if both NNs are in Standby mode, client needs to try failing HADOOP-7896. HA: if both NNs are in Standby mode, client needs to try failing
back and forth several times with sleeps. (atm) back and forth several times with sleeps. (atm)
HADOOP-7922. Improve some logging for client IPC failovers and
StandbyExceptions (todd)

View File

@ -93,16 +93,30 @@ public Object invoke(Object proxy, Method method, Object[] args)
} }
return null; return null;
} else { // retry or failover } else { // retry or failover
if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY) {
String msg = "Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass()
+ " after " + invocationFailoverCount + " fail over attempts."
+ " Trying to fail over " + formatSleepMessage(action.delayMillis);
if (LOG.isDebugEnabled()) {
LOG.debug(msg, e);
} else {
LOG.warn(msg);
}
} else {
if(LOG.isDebugEnabled()) {
LOG.debug("Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass() + ". Retrying " +
formatSleepMessage(action.delayMillis), e);
}
}
if (action.delayMillis > 0) { if (action.delayMillis > 0) {
ThreadUtil.sleepAtLeastIgnoreInterrupts(action.delayMillis); ThreadUtil.sleepAtLeastIgnoreInterrupts(action.delayMillis);
} }
if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY) { if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY) {
LOG.warn("Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass()
+ " after " + invocationFailoverCount + " fail over attempts."
+ " Trying to fail over.", e);
// Make sure that concurrent failed method invocations only cause a // Make sure that concurrent failed method invocations only cause a
// single actual fail over. // single actual fail over.
synchronized (proxyProvider) { synchronized (proxyProvider) {
@ -118,14 +132,18 @@ public Object invoke(Object proxy, Method method, Object[] args)
invocationFailoverCount++; invocationFailoverCount++;
} }
} }
if(LOG.isDebugEnabled()) {
LOG.debug("Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass() + ". Retrying.", e);
}
} }
} }
} }
private static String formatSleepMessage(long millis) {
if (millis > 0) {
return "after sleeping for " + millis + "ms.";
} else {
return "immediately.";
}
}
private Object invokeMethod(Method method, Object[] args) throws Throwable { private Object invokeMethod(Method method, Object[] args) throws Throwable {
try { try {
if (!method.isAccessible()) { if (!method.isAccessible()) {

View File

@ -1616,6 +1616,10 @@ public Writable run() throws Exception {
// on the server side, as opposed to just a normal exceptional // on the server side, as opposed to just a normal exceptional
// result. // result.
LOG.warn(logMsg, e); LOG.warn(logMsg, e);
} else if (e instanceof StandbyException) {
// Don't log the whole stack trace of these exceptions.
// Way too noisy!
LOG.info(logMsg);
} else { } else {
LOG.info(logMsg, e); LOG.info(logMsg, e);
} }