only print failure to reconnect to node from other nodes after several retries

This commit is contained in:
Shay Banon 2012-08-31 17:14:04 +02:00
parent dea2de3304
commit 888b7cc48f
1 changed files with 24 additions and 1 deletions

View File

@ -375,6 +375,9 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
private class ReconnectToNodes implements Runnable {
private ConcurrentMap<DiscoveryNode, Integer> failureCount = ConcurrentCollections.newConcurrentMap();
@Override
public void run() {
// master node will check against all nodes if its alive with certain discoveries implementations,
@ -395,12 +398,32 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
return;
}
if (clusterState.nodes().nodeExists(node.id())) { // double check here as well, maybe its gone?
logger.warn("failed to reconnect to node {}", e, node);
Integer nodeFailureCount = failureCount.get(node);
if (nodeFailureCount == null) {
nodeFailureCount = 1;
} else {
nodeFailureCount = nodeFailureCount + 1;
}
// log every 6th failure
if ((nodeFailureCount % 6) == 0) {
// reset the failure count...
nodeFailureCount = 0;
logger.warn("failed to reconnect to node {}", e, node);
}
failureCount.put(node, nodeFailureCount);
}
}
}
}
}
// go over and remove failed nodes that have been removed
DiscoveryNodes nodes = clusterState.nodes();
for (Iterator<DiscoveryNode> failedNodesIt = failureCount.keySet().iterator(); failedNodesIt.hasNext(); ) {
DiscoveryNode failedNode = failedNodesIt.next();
if (!nodes.nodeExists(failedNode.id())) {
failedNodesIt.remove();
}
}
if (lifecycle.started()) {
reconnectToNodes = threadPool.schedule(reconnectInterval, ThreadPool.Names.GENERIC, this);
}