mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-07 03:19:11 +00:00
Transport Client: Improve remote node freeze handling by adding another timeout layer, closes #1653.
This commit is contained in:
parent
9194d36a64
commit
eb4f6709d9
@ -55,6 +55,8 @@ public class TransportClientNodesService extends AbstractComponent {
|
|||||||
|
|
||||||
private final TimeValue nodesSamplerInterval;
|
private final TimeValue nodesSamplerInterval;
|
||||||
|
|
||||||
|
private final long pingTimeout;
|
||||||
|
|
||||||
private final ClusterName clusterName;
|
private final ClusterName clusterName;
|
||||||
|
|
||||||
private final TransportService transportService;
|
private final TransportService transportService;
|
||||||
@ -87,6 +89,7 @@ public class TransportClientNodesService extends AbstractComponent {
|
|||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
|
|
||||||
this.nodesSamplerInterval = componentSettings.getAsTime("nodes_sampler_interval", timeValueSeconds(5));
|
this.nodesSamplerInterval = componentSettings.getAsTime("nodes_sampler_interval", timeValueSeconds(5));
|
||||||
|
this.pingTimeout = componentSettings.getAsTime("ping_timeout", timeValueSeconds(5)).millis();
|
||||||
|
|
||||||
if (logger.isDebugEnabled()) {
|
if (logger.isDebugEnabled()) {
|
||||||
logger.debug("node_sampler_interval[" + nodesSamplerInterval + "]");
|
logger.debug("node_sampler_interval[" + nodesSamplerInterval + "]");
|
||||||
@ -242,10 +245,14 @@ public class TransportClientNodesService extends AbstractComponent {
|
|||||||
class ScheduledNodeSampler implements Runnable {
|
class ScheduledNodeSampler implements Runnable {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
try {
|
||||||
nodesSampler.sample();
|
nodesSampler.sample();
|
||||||
if (!closed) {
|
if (!closed) {
|
||||||
nodesSamplerFuture = threadPool.schedule(nodesSamplerInterval, ThreadPool.Names.CACHED, this);
|
nodesSamplerFuture = threadPool.schedule(nodesSamplerInterval, ThreadPool.Names.CACHED, this);
|
||||||
}
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
logger.warn("failed to sample", e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -262,24 +269,28 @@ public class TransportClientNodesService extends AbstractComponent {
|
|||||||
try {
|
try {
|
||||||
transportService.connectToNode(node);
|
transportService.connectToNode(node);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.debug("Failed to connect to node " + node + ", removed from nodes list", e);
|
logger.debug("failed to connect to node [{}], removed from nodes list", e, node);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
NodesInfoResponse nodeInfo = transportService.submitRequest(node, NodesInfoAction.NAME, Requests.nodesInfoRequest("_local"), new FutureTransportResponseHandler<NodesInfoResponse>() {
|
NodesInfoResponse nodeInfo = transportService.submitRequest(node, NodesInfoAction.NAME,
|
||||||
|
Requests.nodesInfoRequest("_local"),
|
||||||
|
TransportRequestOptions.options().withTimeout(pingTimeout),
|
||||||
|
new FutureTransportResponseHandler<NodesInfoResponse>() {
|
||||||
@Override
|
@Override
|
||||||
public NodesInfoResponse newInstance() {
|
public NodesInfoResponse newInstance() {
|
||||||
return new NodesInfoResponse();
|
return new NodesInfoResponse();
|
||||||
}
|
}
|
||||||
}).txGet();
|
}).txGet();
|
||||||
if (!clusterName.equals(nodeInfo.clusterName())) {
|
if (!clusterName.equals(nodeInfo.clusterName())) {
|
||||||
logger.warn("Node {} not part of the cluster {}, ignoring...", node, clusterName);
|
logger.warn("node {} not part of the cluster {}, ignoring...", node, clusterName);
|
||||||
} else {
|
} else {
|
||||||
newNodes.add(node);
|
newNodes.add(node);
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.warn("failed to get node info for {}", e, node);
|
logger.info("failed to get node info for {}, disconnecting...", e, node);
|
||||||
|
transportService.disconnectFromNode(node);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
nodes = new ImmutableList.Builder<DiscoveryNode>().addAll(newNodes).build();
|
nodes = new ImmutableList.Builder<DiscoveryNode>().addAll(newNodes).build();
|
||||||
@ -311,8 +322,18 @@ public class TransportClientNodesService extends AbstractComponent {
|
|||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
try {
|
try {
|
||||||
transportService.connectToNode(listedNode); // make sure we are connected to it
|
if (!transportService.nodeConnected(listedNode)) {
|
||||||
transportService.sendRequest(listedNode, NodesInfoAction.NAME, Requests.nodesInfoRequest("_all"), new BaseTransportResponseHandler<NodesInfoResponse>() {
|
try {
|
||||||
|
transportService.connectToNode(listedNode);
|
||||||
|
} catch (Exception e) {
|
||||||
|
logger.debug("failed to connect to node [{}], removed from nodes list", e, listedNode);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
transportService.sendRequest(listedNode, NodesInfoAction.NAME,
|
||||||
|
Requests.nodesInfoRequest("_all"),
|
||||||
|
TransportRequestOptions.options().withTimeout(pingTimeout),
|
||||||
|
new BaseTransportResponseHandler<NodesInfoResponse>() {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public NodesInfoResponse newInstance() {
|
public NodesInfoResponse newInstance() {
|
||||||
@ -331,13 +352,15 @@ public class TransportClientNodesService extends AbstractComponent {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void handleException(TransportException exp) {
|
public void handleException(TransportException e) {
|
||||||
logger.debug("Failed to get node info from " + listedNode + ", removed from nodes list", exp);
|
logger.info("failed to get node info for {}, disconnecting...", e, listedNode);
|
||||||
|
transportService.disconnectFromNode(listedNode);
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.debug("Failed to get node info from " + listedNode + ", removed from nodes list", e);
|
logger.info("failed to get node info for {}, disconnecting...", e, listedNode);
|
||||||
|
transportService.disconnectFromNode(listedNode);
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -354,7 +377,7 @@ public class TransportClientNodesService extends AbstractComponent {
|
|||||||
for (NodesInfoResponse nodesInfoResponse : nodesInfoResponses) {
|
for (NodesInfoResponse nodesInfoResponse : nodesInfoResponses) {
|
||||||
for (NodeInfo nodeInfo : nodesInfoResponse) {
|
for (NodeInfo nodeInfo : nodesInfoResponse) {
|
||||||
if (!clusterName.equals(nodesInfoResponse.clusterName())) {
|
if (!clusterName.equals(nodesInfoResponse.clusterName())) {
|
||||||
logger.warn("Node {} not part of the cluster {}, ignoring...", nodeInfo.node(), clusterName);
|
logger.warn("node {} not part of the cluster {}, ignoring...", nodeInfo.node(), clusterName);
|
||||||
} else {
|
} else {
|
||||||
if (nodeInfo.node().dataNode()) { // only add data nodes to connect to
|
if (nodeInfo.node().dataNode()) { // only add data nodes to connect to
|
||||||
newNodes.add(nodeInfo.node());
|
newNodes.add(nodeInfo.node());
|
||||||
@ -369,7 +392,7 @@ public class TransportClientNodesService extends AbstractComponent {
|
|||||||
transportService.connectToNode(node);
|
transportService.connectToNode(node);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
it.remove();
|
it.remove();
|
||||||
logger.debug("Failed to connect to discovered node [" + node + "]", e);
|
logger.debug("failed to connect to discovered node [" + node + "]", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
nodes = new ImmutableList.Builder<DiscoveryNode>().addAll(newNodes).build();
|
nodes = new ImmutableList.Builder<DiscoveryNode>().addAll(newNodes).build();
|
||||||
|
Loading…
x
Reference in New Issue
Block a user