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

(cherry picked from commit e62d8f8412)
This commit is contained in:
Hanisha Koneru 2020-07-13 12:55:34 -07:00 committed by Takanobu Asanuma
parent 26705bbc60
commit 9da7d80c4e

View File

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