YARN-6145. Improve log message on fail over. Contributed by Jian He.
This commit is contained in:
parent
2007e0cf2a
commit
eec52e158b
|
@ -130,7 +130,9 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
|
||||||
Thread.sleep(retryInfo.delay);
|
Thread.sleep(retryInfo.delay);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
LOG.warn("Interrupted while waiting to retry", e);
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Interrupted while waiting to retry", e);
|
||||||
|
}
|
||||||
InterruptedIOException intIOE = new InterruptedIOException(
|
InterruptedIOException intIOE = new InterruptedIOException(
|
||||||
"Retry interrupted");
|
"Retry interrupted");
|
||||||
intIOE.initCause(e);
|
intIOE.initCause(e);
|
||||||
|
@ -375,7 +377,7 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
|
||||||
}
|
}
|
||||||
|
|
||||||
final StringBuilder b = new StringBuilder()
|
final StringBuilder b = new StringBuilder()
|
||||||
.append("Exception while invoking ")
|
.append(ex + ", while invoking ")
|
||||||
.append(proxyDescriptor.getProxyInfo().getString(method.getName()));
|
.append(proxyDescriptor.getProxyInfo().getString(method.getName()));
|
||||||
if (failovers > 0) {
|
if (failovers > 0) {
|
||||||
b.append(" after ").append(failovers).append(" failover attempts");
|
b.append(" after ").append(failovers).append(" failover attempts");
|
||||||
|
@ -384,7 +386,7 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
|
||||||
b.append(delay > 0? "after sleeping for " + delay + "ms.": "immediately.");
|
b.append(delay > 0? "after sleeping for " + delay + "ms.": "immediately.");
|
||||||
|
|
||||||
if (info) {
|
if (info) {
|
||||||
LOG.info(b.toString(), ex);
|
LOG.info(b.toString());
|
||||||
} else {
|
} else {
|
||||||
LOG.debug(b.toString(), ex);
|
LOG.debug(b.toString(), ex);
|
||||||
}
|
}
|
||||||
|
|
|
@ -910,8 +910,10 @@ public class Client implements AutoCloseable {
|
||||||
}
|
}
|
||||||
if (action.action == RetryAction.RetryDecision.FAIL) {
|
if (action.action == RetryAction.RetryDecision.FAIL) {
|
||||||
if (action.reason != null) {
|
if (action.reason != null) {
|
||||||
LOG.warn("Failed to connect to server: " + server + ": "
|
if (LOG.isDebugEnabled()) {
|
||||||
+ action.reason, ioe);
|
LOG.debug("Failed to connect to server: " + server + ": "
|
||||||
|
+ action.reason, ioe);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
throw ioe;
|
throw ioe;
|
||||||
}
|
}
|
||||||
|
|
|
@ -144,6 +144,8 @@ public class RequestHedgingRMFailoverProxyProvider<T>
|
||||||
args);
|
args);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
LOG.info("Looking for the active RM in " + Arrays.toString(rmServiceIds)
|
||||||
|
+ "...");
|
||||||
ExecutorService executor = null;
|
ExecutorService executor = null;
|
||||||
CompletionService<Object> completionService;
|
CompletionService<Object> completionService;
|
||||||
try {
|
try {
|
||||||
|
@ -166,7 +168,7 @@ public class RequestHedgingRMFailoverProxyProvider<T>
|
||||||
Object retVal;
|
Object retVal;
|
||||||
try {
|
try {
|
||||||
retVal = callResultFuture.get();
|
retVal = callResultFuture.get();
|
||||||
LOG.info("Invocation successful on [" + pInfo + "]");
|
LOG.info("Found active RM [" + pInfo + "]");
|
||||||
return retVal;
|
return retVal;
|
||||||
} catch (Exception ex) {
|
} catch (Exception ex) {
|
||||||
// Throw exception from first responding RM so that clients can handle
|
// Throw exception from first responding RM so that clients can handle
|
||||||
|
@ -192,7 +194,7 @@ public class RequestHedgingRMFailoverProxyProvider<T>
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void performFailover(T currentProxy) {
|
public void performFailover(T currentProxy) {
|
||||||
LOG.info("Connection lost, trying to fail over.");
|
LOG.info("Connection lost with " + successfulProxy + ", trying to fail over.");
|
||||||
successfulProxy = null;
|
successfulProxy = null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue