Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -176,9 +175,9 @@ CompletableFuture<RaftClientReply> 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();
Expand Down Expand Up @@ -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<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()) {
Expand Down
15 changes: 12 additions & 3 deletions ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ static <T> T doPrivileged(Supplier<T> action, Function<SecurityException, T> exc

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;
});
}
Expand Down Expand Up @@ -228,8 +228,8 @@ static <RETURN, THROWABLE extends Throwable> 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());
}
}

Expand Down Expand Up @@ -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 <T> CompletableFuture<Void> allOf(Collection<CompletableFuture<T>> futures) {
if (futures == null || futures.isEmpty()) {
return CompletableFuture.completedFuture(null);
Expand Down
Loading