|
51 | 51 | import java.util.concurrent.ConcurrentMap; |
52 | 52 | import java.util.concurrent.Semaphore; |
53 | 53 | import java.util.concurrent.atomic.AtomicReference; |
54 | | -import java.util.function.Consumer; |
55 | 54 | import java.util.function.Function; |
56 | 55 | import java.util.function.LongFunction; |
57 | 56 |
|
@@ -176,9 +175,9 @@ CompletableFuture<RaftClientReply> send(RaftClientRequest.Type type, Message mes |
176 | 175 | ).whenComplete((r, e) -> { |
177 | 176 | if (e != null) { |
178 | 177 | 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()); |
180 | 179 | } else { |
181 | | - LOG.error("Failed to send request, message=" + message, e); |
| 180 | + LOG.error("Failed to send request, message={}", message, e); |
182 | 181 | } |
183 | 182 | } |
184 | 183 | requestSemaphore.release(); |
@@ -212,13 +211,31 @@ private void sendRequestWithRetry(PendingOrderedRequest pending) { |
212 | 211 | }).exceptionally(e -> { |
213 | 212 | final Throwable exception = e; |
214 | 213 | 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)); |
217 | 215 | handleException(pending, request, e); |
218 | 216 | return null; |
219 | 217 | }); |
220 | 218 | } |
221 | 219 |
|
| 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 | + |
222 | 239 | private void handleException(PendingOrderedRequest pending, RaftClientRequest request, Throwable e) { |
223 | 240 | final RetryPolicy retryPolicy = client.getRetryPolicy(); |
224 | 241 | if (client.isClosed()) { |
|
0 commit comments