HADOOP-17116. Skip Retry INFO logging on first failover from a proxy

This commit is contained in:
Hanisha Koneru 2020-07-13 12:55:34 -07:00
parent 0427100b75
commit e62d8f8412
1 changed files with 15 additions and 6 deletions

View File

@ -35,6 +35,7 @@ import java.lang.reflect.Method;
import java.lang.reflect.Proxy; import java.lang.reflect.Proxy;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet;
import java.util.Map; import java.util.Map;
/** /**
@ -312,6 +313,8 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
private volatile boolean hasSuccessfulCall = false; private volatile boolean hasSuccessfulCall = false;
private HashSet<String> failedAtLeastOnce = new HashSet<>();
private final RetryPolicy defaultPolicy; private final RetryPolicy defaultPolicy;
private final Map<String,RetryPolicy> methodNameToPolicyMap; private final Map<String,RetryPolicy> methodNameToPolicyMap;
@ -390,12 +393,18 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
private void log(final Method method, final boolean isFailover, private void log(final Method method, final boolean isFailover,
final int failovers, final long delay, final Exception ex) { final int failovers, final long delay, final Exception ex) {
// log info if this has made some successful calls or boolean info = true;
// this is not the first failover // If this is the first failover to this proxy, skip logging at INFO level
final boolean info = hasSuccessfulCall || failovers != 0 if (!failedAtLeastOnce.contains(proxyDescriptor.getProxyInfo().toString()))
|| asyncCallHandler.hasSuccessfulCall(); {
if (!info && !LOG.isDebugEnabled()) { failedAtLeastOnce.add(proxyDescriptor.getProxyInfo().toString());
return;
// If successful calls were made to this proxy, log info even for first
// failover
info = hasSuccessfulCall || asyncCallHandler.hasSuccessfulCall();
if (!info && !LOG.isDebugEnabled()) {
return;
}
} }
final StringBuilder b = new StringBuilder() final StringBuilder b = new StringBuilder()