From 8d24aba731bc1f08d5a29992aa205d03da94f8bb Mon Sep 17 00:00:00 2001 From: Hanisha Koneru Date: Mon, 13 Jul 2020 12:55:34 -0700 Subject: [PATCH] HADOOP-17116. Skip Retry INFO logging on first failover from a proxy (cherry picked from commit e62d8f841275ee47a0ba911415aac9e39af291c6) --- .../io/retry/RetryInvocationHandler.java | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java index 64824a15cd8..6db00d724aa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java @@ -35,6 +35,7 @@ import java.lang.reflect.Method; 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 class RetryInvocationHandler implements RpcInvocationHandler { private volatile boolean hasSuccessfulCall = false; + private HashSet failedAtLeastOnce = new HashSet<>(); + private final RetryPolicy defaultPolicy; private final Map methodNameToPolicyMap; @@ -390,12 +393,18 @@ public class RetryInvocationHandler implements RpcInvocationHandler { 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()