This is an automated email from the ASF dual-hosted git repository.
szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git
The following commit(s) were added to refs/heads/master by this push:
new bc9fab890 RATIS-2374. Do not print stack trace for known exceptions in
OrderedAsync (#1330)
bc9fab890 is described below
commit bc9fab890816a01cf5fa66a8ac231d08bb845939
Author: Tsz-Wo Nicholas Sze <[email protected]>
AuthorDate: Wed Dec 24 05:41:41 2025 -0800
RATIS-2374. Do not print stack trace for known exceptions in OrderedAsync
(#1330)
---
.../org/apache/ratis/client/impl/OrderedAsync.java | 27 ++++++++++++++++++----
.../main/java/org/apache/ratis/util/JavaUtils.java | 15 +++++++++---
2 files changed, 34 insertions(+), 8 deletions(-)
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 1e21b171b..fbeb4b992 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.ConcurrentHashMap;
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 @@ public final class OrderedAsync {
).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 @@ public final class OrderedAsync {
}).exceptionally(e -> {
final Throwable exception = e;
final String key = client.getId() + "-" + request.getCallId() + "-" +
exception;
- final Consumer<String> 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 958e88cee..c7f8f7629 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 @@ public interface JavaUtils {
static <T> T doPrivileged(Supplier<T> action, Supplier<String> 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 @@ public interface JavaUtils {
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 @@ public interface JavaUtils {
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 <T> CompletableFuture<Void> allOf(Collection<CompletableFuture<T>>
futures) {
if (futures == null || futures.isEmpty()) {
return CompletableFuture.completedFuture(null);