diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index 1e21b171b3..fbeb4b992a 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -51,7 +51,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongFunction; @@ -176,9 +175,9 @@ CompletableFuture send(RaftClientRequest.Type type, Message mes ).whenComplete((r, e) -> { if (e != null) { if (e.getCause() instanceof AlreadyClosedException) { - LOG.error("Failed to send request, message=" + message + " due to " + e); + LOG.error("Failed to send request, message={} due to {}", message, e.toString()); } else { - LOG.error("Failed to send request, message=" + message, e); + LOG.error("Failed to send request, message={}", message, e); } } requestSemaphore.release(); @@ -212,13 +211,31 @@ private void sendRequestWithRetry(PendingOrderedRequest pending) { }).exceptionally(e -> { final Throwable exception = e; final String key = client.getId() + "-" + request.getCallId() + "-" + exception; - final Consumer op = suffix -> LOG.error("{} {}: Failed* {}", suffix, client.getId(), request, exception); - BatchLogger.print(BatchLogKey.SEND_REQUEST_EXCEPTION, key, op); + BatchLogger.print(BatchLogKey.SEND_REQUEST_EXCEPTION, key, prefix -> logError(prefix, request, exception)); handleException(pending, request, e); return null; }); } + private void logError(String prefix, RaftClientRequest request, Throwable e) { + final Class[] knownExceptionClasses = {AlreadyClosedException.class, NotLeaderException.class}; + for(Class known : knownExceptionClasses) { + if (logError(prefix, request, e, known)) { + return; + } + } + LOG.error("{} {}: Failed* {}", prefix, client.getId(), request, e); + } + + private boolean logError(String prefix, RaftClientRequest request, Throwable e, Class cause) { + if (JavaUtils.isCausedBy(e, cause)) { + LOG.error("{} {}: Failed* {} due to {} caused by {}", + prefix, client.getId(), request, e, cause.getSimpleName()); + return true; + } + return false; + } + private void handleException(PendingOrderedRequest pending, RaftClientRequest request, Throwable e) { final RetryPolicy retryPolicy = client.getRetryPolicy(); if (client.isClosed()) { diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java index 958e88cee5..c7f8f76298 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java @@ -134,7 +134,7 @@ static T doPrivileged(Supplier action, Function exc static T doPrivileged(Supplier action, Supplier name) { return doPrivileged(action, e -> { - LOG.warn("Failed to " + name.get(), e); + LOG.warn("Failed to {}", name.get(), e); return null; }); } @@ -228,8 +228,8 @@ static RETURN attempt( throw t; } if (log != null && log.isWarnEnabled()) { - log.warn("FAILED \"" + name.get() + "\", attempt #" + i + "/" + numAttempts - + ", sleep " + sleepTime + " and then retry: " + t); + log.warn("FAILED \"{}\", attempt #{}/{}, sleep {} and then retry: {}", + name.get(), i, numAttempts, sleepTime, t.toString()); } } @@ -291,6 +291,15 @@ static Throwable unwrapCompletionException(Throwable t) { return t instanceof CompletionException && t.getCause() != null? t.getCause(): t; } + static boolean isCausedBy(Throwable t, Class cause) { + for(; t != null; t = t.getCause()) { + if (cause.isInstance(t)) { + return true; + } + } + return false; + } + static CompletableFuture allOf(Collection> futures) { if (futures == null || futures.isEmpty()) { return CompletableFuture.completedFuture(null);