Skip to content

Commit

Permalink
IGNITE-23395 Remove excessive raft subsystem log spam with network ex…
Browse files Browse the repository at this point in the history
…ceptions (apache#4761)
  • Loading branch information
denis-chudov authored Nov 21, 2024
1 parent dcaad51 commit ee004ae
Show file tree
Hide file tree
Showing 2 changed files with 16 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -57,6 +57,8 @@ public abstract class AbstractClientService implements ClientService, TopologyEv
protected ExecutorService rpcExecutor;
protected RpcOptions rpcOptions;

private Set<PeerId> deadPeers = ConcurrentHashMap.newKeySet();

/**
* The set of pinged consistent IDs.
*/
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -153,6 +159,8 @@ public CompletableFuture<Boolean> connectAsync(PeerId peerId) {
if (resp != null && resp.errorCode() == 0) {
readyConsistentIds.add(peerId.getConsistentId());

deadPeers.remove(peerId);

return true;
} else {
return false;
Expand Down

0 comments on commit ee004ae

Please sign in to comment.