From ee004ae4a5836dad129efefc48683fb9a2d97ec0 Mon Sep 17 00:00:00 2001 From: Denis Chudov Date: Thu, 21 Nov 2024 16:15:14 +0300 Subject: [PATCH] IGNITE-23395 Remove excessive raft subsystem log spam with network exceptions (#4761) --- .../ignite/raft/jraft/core/ReplicatorGroupImpl.java | 7 ++++++- .../raft/jraft/rpc/impl/AbstractClientService.java | 12 ++++++++++-- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/ReplicatorGroupImpl.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/ReplicatorGroupImpl.java index 57bedd34758..9c88c261080 100644 --- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/ReplicatorGroupImpl.java +++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/ReplicatorGroupImpl.java @@ -119,9 +119,14 @@ public boolean addReplicator(final PeerId peer, final ReplicatorType replicatorT assert client != null; if (!client.connect(peer)) { - LOG.error("Fail to check replicator connection to peer={}, replicatorType={}.", peer, replicatorType); + if (!failureReplicators.containsKey(peer)) { + LOG.error("Fail to check replicator connection to peer={}, replicatorType={}.", peer, replicatorType); + } + this.failureReplicators.put(peer, replicatorType); return false; + } else { + failureReplicators.remove(peer); } // if (!sync) { diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/AbstractClientService.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/AbstractClientService.java index 07f4d4e1ef8..d7bc85fdad7 100644 --- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/AbstractClientService.java +++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/AbstractClientService.java @@ -21,7 +21,7 @@ import java.net.ConnectException; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.ConcurrentHashMap;import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import org.apache.ignite.internal.logger.IgniteLogger; @@ -57,6 +57,8 @@ public abstract class AbstractClientService implements ClientService, TopologyEv protected ExecutorService rpcExecutor; protected RpcOptions rpcOptions; + private Set deadPeers = ConcurrentHashMap.newKeySet(); + /** * The set of pinged consistent IDs. */ @@ -121,7 +123,11 @@ public boolean connect(final PeerId peerId) { LOG.error("Interrupted while connecting to {}, exception: {}.", peerId, e.getMessage()); } catch (ExecutionException e) { - LOG.error("Fail to connect {}, exception: {}.", peerId, e.getMessage()); + if (!deadPeers.contains(peerId)) { + deadPeers.add(peerId); + + LOG.error("Fail to connect {}, exception: {}.", peerId, e.getMessage()); + } } return false; @@ -153,6 +159,8 @@ public CompletableFuture connectAsync(PeerId peerId) { if (resp != null && resp.errorCode() == 0) { readyConsistentIds.add(peerId.getConsistentId()); + deadPeers.remove(peerId); + return true; } else { return false;