Skip to content

Commit bc9fab8

Browse files
authored
RATIS-2374. Do not print stack trace for known exceptions in OrderedAsync (#1330)
1 parent b73cf97 commit bc9fab8

File tree

2 files changed

+34
-8
lines changed

2 files changed

+34
-8
lines changed

ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java

Lines changed: 22 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,6 @@
5151
import java.util.concurrent.ConcurrentMap;
5252
import java.util.concurrent.Semaphore;
5353
import java.util.concurrent.atomic.AtomicReference;
54-
import java.util.function.Consumer;
5554
import java.util.function.Function;
5655
import java.util.function.LongFunction;
5756

@@ -176,9 +175,9 @@ CompletableFuture<RaftClientReply> send(RaftClientRequest.Type type, Message mes
176175
).whenComplete((r, e) -> {
177176
if (e != null) {
178177
if (e.getCause() instanceof AlreadyClosedException) {
179-
LOG.error("Failed to send request, message=" + message + " due to " + e);
178+
LOG.error("Failed to send request, message={} due to {}", message, e.toString());
180179
} else {
181-
LOG.error("Failed to send request, message=" + message, e);
180+
LOG.error("Failed to send request, message={}", message, e);
182181
}
183182
}
184183
requestSemaphore.release();
@@ -212,13 +211,31 @@ private void sendRequestWithRetry(PendingOrderedRequest pending) {
212211
}).exceptionally(e -> {
213212
final Throwable exception = e;
214213
final String key = client.getId() + "-" + request.getCallId() + "-" + exception;
215-
final Consumer<String> op = suffix -> LOG.error("{} {}: Failed* {}", suffix, client.getId(), request, exception);
216-
BatchLogger.print(BatchLogKey.SEND_REQUEST_EXCEPTION, key, op);
214+
BatchLogger.print(BatchLogKey.SEND_REQUEST_EXCEPTION, key, prefix -> logError(prefix, request, exception));
217215
handleException(pending, request, e);
218216
return null;
219217
});
220218
}
221219

220+
private void logError(String prefix, RaftClientRequest request, Throwable e) {
221+
final Class<?>[] knownExceptionClasses = {AlreadyClosedException.class, NotLeaderException.class};
222+
for(Class<?> known : knownExceptionClasses) {
223+
if (logError(prefix, request, e, known)) {
224+
return;
225+
}
226+
}
227+
LOG.error("{} {}: Failed* {}", prefix, client.getId(), request, e);
228+
}
229+
230+
private boolean logError(String prefix, RaftClientRequest request, Throwable e, Class<?> cause) {
231+
if (JavaUtils.isCausedBy(e, cause)) {
232+
LOG.error("{} {}: Failed* {} due to {} caused by {}",
233+
prefix, client.getId(), request, e, cause.getSimpleName());
234+
return true;
235+
}
236+
return false;
237+
}
238+
222239
private void handleException(PendingOrderedRequest pending, RaftClientRequest request, Throwable e) {
223240
final RetryPolicy retryPolicy = client.getRetryPolicy();
224241
if (client.isClosed()) {

ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java

Lines changed: 12 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -134,7 +134,7 @@ static <T> T doPrivileged(Supplier<T> action, Function<SecurityException, T> exc
134134

135135
static <T> T doPrivileged(Supplier<T> action, Supplier<String> name) {
136136
return doPrivileged(action, e -> {
137-
LOG.warn("Failed to " + name.get(), e);
137+
LOG.warn("Failed to {}", name.get(), e);
138138
return null;
139139
});
140140
}
@@ -228,8 +228,8 @@ static <RETURN, THROWABLE extends Throwable> RETURN attempt(
228228
throw t;
229229
}
230230
if (log != null && log.isWarnEnabled()) {
231-
log.warn("FAILED \"" + name.get() + "\", attempt #" + i + "/" + numAttempts
232-
+ ", sleep " + sleepTime + " and then retry: " + t);
231+
log.warn("FAILED \"{}\", attempt #{}/{}, sleep {} and then retry: {}",
232+
name.get(), i, numAttempts, sleepTime, t.toString());
233233
}
234234
}
235235

@@ -291,6 +291,15 @@ static Throwable unwrapCompletionException(Throwable t) {
291291
return t instanceof CompletionException && t.getCause() != null? t.getCause(): t;
292292
}
293293

294+
static boolean isCausedBy(Throwable t, Class<?> cause) {
295+
for(; t != null; t = t.getCause()) {
296+
if (cause.isInstance(t)) {
297+
return true;
298+
}
299+
}
300+
return false;
301+
}
302+
294303
static <T> CompletableFuture<Void> allOf(Collection<CompletableFuture<T>> futures) {
295304
if (futures == null || futures.isEmpty()) {
296305
return CompletableFuture.completedFuture(null);

0 commit comments

Comments
 (0)