mirror of https://github.com/apache/druid.git
Better logging in ServiceClientImpl (#15269)
ServiceClientImpl logs the cause of every retry, even though we are retrying the connection attempt. This leads to slight pollution in the logs because a lot of the time, the reason for retrying is the same. This is seen primarily in MSQ, when the worker task hasn't launched yet however controller attempts to connect to the worker task, which can lead to scary-looking messages (with INFO log level), even though they are normal. This PR changes the logging logic to log every 10 (arbitrary number) retries instead of every retry, to reduce the pollution of the logs. Note: If there are no retries left, the client returns an exception, which would get thrown up by the caller, and therefore this change doesn't hide any important information.
This commit is contained in:
parent
6b6d73b5d4
commit
b82ad59dfe
|
@ -301,7 +301,7 @@ public class IndexerWorkerClient implements WorkerClient
|
|||
final ServiceClient client = clientFactory.makeClient(
|
||||
id,
|
||||
locator,
|
||||
new SpecificTaskRetryPolicy(workerTaskId, StandardRetryPolicy.unlimited())
|
||||
new SpecificTaskRetryPolicy(workerTaskId, StandardRetryPolicy.unlimitedWithoutRetryLogging())
|
||||
);
|
||||
return Pair.of(client, locator);
|
||||
}
|
||||
|
|
|
@ -220,7 +220,20 @@ public class ServiceClientImpl implements ServiceClient
|
|||
if (shouldTry(nextAttemptNumber) && retryPolicy.retryThrowable(t)) {
|
||||
final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber);
|
||||
|
||||
log.noStackTrace().info(t, buildErrorMessage(request, null, backoffMs, nextAttemptNumber));
|
||||
if (retryPolicy.retryLoggable()) {
|
||||
// log as INFO level if the retry is loggable
|
||||
log.noStackTrace().info(t, buildErrorMessage(request, null, backoffMs, nextAttemptNumber));
|
||||
} else if (log.isDebugEnabled()) {
|
||||
// log as DEBUG level if the debug log is enabled
|
||||
log.noStackTrace().debug(t, buildErrorMessage(request, null, backoffMs, nextAttemptNumber));
|
||||
} else {
|
||||
// If none of the above is valid, we log the error message every tenth time we retry. It seems like
|
||||
// a good balance between making the logs not too verbose when the retry is due to the same cause
|
||||
// and enriching logs with useful information, if we keep retrying due to the same reason
|
||||
if (nextAttemptNumber > 0 && nextAttemptNumber % 10 == 0) {
|
||||
log.noStackTrace().info(t, buildErrorMessage(request, null, backoffMs, nextAttemptNumber));
|
||||
}
|
||||
}
|
||||
|
||||
connectExec.schedule(
|
||||
() -> tryRequest(requestBuilder, handler, retVal, nextAttemptNumber, ImmutableSet.of()),
|
||||
|
@ -272,7 +285,15 @@ public class ServiceClientImpl implements ServiceClient
|
|||
// Retryable server response (or null errorHolder, which means null result, which can happen
|
||||
// if the HttpClient encounters an exception in the midst of response processing).
|
||||
final long backoffMs = computeBackoffMs(retryPolicy, attemptNumber);
|
||||
log.info(buildErrorMessage(request, errorHolder, backoffMs, nextAttemptNumber));
|
||||
if (retryPolicy.retryLoggable()) {
|
||||
log.noStackTrace().info(buildErrorMessage(request, errorHolder, backoffMs, nextAttemptNumber));
|
||||
} else if (log.isDebugEnabled()) {
|
||||
log.noStackTrace().debug(buildErrorMessage(request, errorHolder, backoffMs, nextAttemptNumber));
|
||||
} else {
|
||||
if (nextAttemptNumber > 0 && nextAttemptNumber % 10 == 0) {
|
||||
log.noStackTrace().info(buildErrorMessage(request, errorHolder, backoffMs, nextAttemptNumber));
|
||||
}
|
||||
}
|
||||
connectExec.schedule(
|
||||
() -> tryRequest(requestBuilder, handler, retVal, nextAttemptNumber, ImmutableSet.of()),
|
||||
backoffMs,
|
||||
|
|
|
@ -54,6 +54,11 @@ public interface ServiceRetryPolicy
|
|||
*/
|
||||
boolean retryThrowable(Throwable t);
|
||||
|
||||
/**
|
||||
* Returns whether to log the cause of failure before retrying
|
||||
*/
|
||||
boolean retryLoggable();
|
||||
|
||||
/**
|
||||
* Returns whether service-not-available, i.e. empty {@link ServiceLocations#getLocations()}, can be retried.
|
||||
*/
|
||||
|
|
|
@ -44,6 +44,9 @@ public class StandardRetryPolicy implements ServiceRetryPolicy
|
|||
private static final int MAX_ATTEMPTS_ABOUT_AN_HOUR = 125;
|
||||
|
||||
private static final StandardRetryPolicy DEFAULT_UNLIMITED_POLICY = new Builder().maxAttempts(UNLIMITED).build();
|
||||
private static final StandardRetryPolicy DEFAULT_UNLIMITED_POLICY_NO_RETRY_LOG = new Builder().maxAttempts(UNLIMITED)
|
||||
.retryLoggable(false)
|
||||
.build();
|
||||
private static final StandardRetryPolicy DEFAULT_ABOUT_AN_HOUR_POLICY =
|
||||
new Builder().maxAttempts(MAX_ATTEMPTS_ABOUT_AN_HOUR).build();
|
||||
private static final StandardRetryPolicy DEFAULT_NO_RETRIES_POLICY = new Builder().maxAttempts(1).build();
|
||||
|
@ -52,13 +55,21 @@ public class StandardRetryPolicy implements ServiceRetryPolicy
|
|||
private final long minWaitMillis;
|
||||
private final long maxWaitMillis;
|
||||
private final boolean retryNotAvailable;
|
||||
private final boolean retryLoggable;
|
||||
|
||||
private StandardRetryPolicy(long maxAttempts, long minWaitMillis, long maxWaitMillis, boolean retryNotAvailable)
|
||||
private StandardRetryPolicy(
|
||||
long maxAttempts,
|
||||
long minWaitMillis,
|
||||
long maxWaitMillis,
|
||||
boolean retryNotAvailable,
|
||||
boolean retryLoggable
|
||||
)
|
||||
{
|
||||
this.maxAttempts = maxAttempts;
|
||||
this.minWaitMillis = minWaitMillis;
|
||||
this.maxWaitMillis = maxWaitMillis;
|
||||
this.retryNotAvailable = retryNotAvailable;
|
||||
this.retryLoggable = retryLoggable;
|
||||
|
||||
if (maxAttempts == 0) {
|
||||
throw new IAE("maxAttempts must be positive (limited) or negative (unlimited); cannot be zero.");
|
||||
|
@ -79,6 +90,14 @@ public class StandardRetryPolicy implements ServiceRetryPolicy
|
|||
return DEFAULT_UNLIMITED_POLICY;
|
||||
}
|
||||
|
||||
/**
|
||||
* Standard unlimited retry policy along with muted the logging for the retries.
|
||||
*/
|
||||
public static StandardRetryPolicy unlimitedWithoutRetryLogging()
|
||||
{
|
||||
return DEFAULT_UNLIMITED_POLICY_NO_RETRY_LOG;
|
||||
}
|
||||
|
||||
/**
|
||||
* Retry policy that uses up to about an hour of total wait time. Note that this is just the total waiting time
|
||||
* between attempts. It does not include the time that each attempt takes to execute.
|
||||
|
@ -135,6 +154,12 @@ public class StandardRetryPolicy implements ServiceRetryPolicy
|
|||
|| (t.getCause() != null && retryThrowable(t.getCause()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean retryLoggable()
|
||||
{
|
||||
return retryLoggable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean retryNotAvailable()
|
||||
{
|
||||
|
@ -147,6 +172,7 @@ public class StandardRetryPolicy implements ServiceRetryPolicy
|
|||
private long minWaitMillis = DEFAULT_MIN_WAIT_MS;
|
||||
private long maxWaitMillis = DEFAULT_MAX_WAIT_MS;
|
||||
private boolean retryNotAvailable = true;
|
||||
private boolean retryLoggable = true;
|
||||
|
||||
public Builder maxAttempts(final long maxAttempts)
|
||||
{
|
||||
|
@ -172,9 +198,15 @@ public class StandardRetryPolicy implements ServiceRetryPolicy
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder retryLoggable(final boolean retryLoggable)
|
||||
{
|
||||
this.retryLoggable = retryLoggable;
|
||||
return this;
|
||||
}
|
||||
|
||||
public StandardRetryPolicy build()
|
||||
{
|
||||
return new StandardRetryPolicy(maxAttempts, minWaitMillis, maxWaitMillis, retryNotAvailable);
|
||||
return new StandardRetryPolicy(maxAttempts, minWaitMillis, maxWaitMillis, retryNotAvailable, retryLoggable);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -77,6 +77,12 @@ public class SpecificTaskRetryPolicy implements ServiceRetryPolicy
|
|||
return StandardRetryPolicy.unlimited().retryThrowable(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean retryLoggable()
|
||||
{
|
||||
return baseRetryPolicy.retryLoggable();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean retryNotAvailable()
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue