|
@@ -35,8 +35,6 @@ import org.elasticsearch.cluster.ClusterName;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
|
import org.elasticsearch.common.Randomness;
|
|
|
import org.elasticsearch.common.component.AbstractComponent;
|
|
|
-import org.elasticsearch.common.settings.Setting;
|
|
|
-import org.elasticsearch.common.settings.Setting.Property;
|
|
|
import org.elasticsearch.common.settings.Settings;
|
|
|
import org.elasticsearch.common.transport.TransportAddress;
|
|
|
import org.elasticsearch.common.unit.TimeValue;
|
|
@@ -45,6 +43,8 @@ import org.elasticsearch.common.util.concurrent.FutureUtils;
|
|
|
import org.elasticsearch.threadpool.ThreadPool;
|
|
|
import org.elasticsearch.transport.ConnectTransportException;
|
|
|
import org.elasticsearch.transport.FutureTransportResponseHandler;
|
|
|
+import org.elasticsearch.transport.NodeDisconnectedException;
|
|
|
+import org.elasticsearch.transport.NodeNotConnectedException;
|
|
|
import org.elasticsearch.transport.TransportException;
|
|
|
import org.elasticsearch.transport.TransportRequestOptions;
|
|
|
import org.elasticsearch.transport.TransportResponseHandler;
|
|
@@ -64,9 +64,7 @@ import java.util.concurrent.CountDownLatch;
|
|
|
import java.util.concurrent.ScheduledFuture;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
-import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
|
|
|
-
|
|
|
-public class TransportClientNodesService extends AbstractComponent implements Closeable {
|
|
|
+final class TransportClientNodesService extends AbstractComponent implements Closeable {
|
|
|
|
|
|
private final TimeValue nodesSamplerInterval;
|
|
|
|
|
@@ -100,37 +98,30 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
|
|
|
private volatile boolean closed;
|
|
|
|
|
|
+ private final TransportClient.HostFailureListener hostFailureListener;
|
|
|
|
|
|
- public static final Setting<TimeValue> CLIENT_TRANSPORT_NODES_SAMPLER_INTERVAL =
|
|
|
- Setting.positiveTimeSetting("client.transport.nodes_sampler_interval", timeValueSeconds(5), Property.NodeScope);
|
|
|
- public static final Setting<TimeValue> CLIENT_TRANSPORT_PING_TIMEOUT =
|
|
|
- Setting.positiveTimeSetting("client.transport.ping_timeout", timeValueSeconds(5), Property.NodeScope);
|
|
|
- public static final Setting<Boolean> CLIENT_TRANSPORT_IGNORE_CLUSTER_NAME =
|
|
|
- Setting.boolSetting("client.transport.ignore_cluster_name", false, Property.NodeScope);
|
|
|
- public static final Setting<Boolean> CLIENT_TRANSPORT_SNIFF =
|
|
|
- Setting.boolSetting("client.transport.sniff", false, Property.NodeScope);
|
|
|
-
|
|
|
- public TransportClientNodesService(Settings settings,TransportService transportService,
|
|
|
- ThreadPool threadPool) {
|
|
|
+ TransportClientNodesService(Settings settings, TransportService transportService,
|
|
|
+ ThreadPool threadPool, TransportClient.HostFailureListener hostFailureListener) {
|
|
|
super(settings);
|
|
|
this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings);
|
|
|
this.transportService = transportService;
|
|
|
this.threadPool = threadPool;
|
|
|
this.minCompatibilityVersion = Version.CURRENT.minimumCompatibilityVersion();
|
|
|
|
|
|
- this.nodesSamplerInterval = CLIENT_TRANSPORT_NODES_SAMPLER_INTERVAL.get(this.settings);
|
|
|
- this.pingTimeout = CLIENT_TRANSPORT_PING_TIMEOUT.get(this.settings).millis();
|
|
|
- this.ignoreClusterName = CLIENT_TRANSPORT_IGNORE_CLUSTER_NAME.get(this.settings);
|
|
|
+ this.nodesSamplerInterval = TransportClient.CLIENT_TRANSPORT_NODES_SAMPLER_INTERVAL.get(this.settings);
|
|
|
+ this.pingTimeout = TransportClient.CLIENT_TRANSPORT_PING_TIMEOUT.get(this.settings).millis();
|
|
|
+ this.ignoreClusterName = TransportClient.CLIENT_TRANSPORT_IGNORE_CLUSTER_NAME.get(this.settings);
|
|
|
|
|
|
if (logger.isDebugEnabled()) {
|
|
|
logger.debug("node_sampler_interval[{}]", nodesSamplerInterval);
|
|
|
}
|
|
|
|
|
|
- if (CLIENT_TRANSPORT_SNIFF.get(this.settings)) {
|
|
|
+ if (TransportClient.CLIENT_TRANSPORT_SNIFF.get(this.settings)) {
|
|
|
this.nodesSampler = new SniffNodesSampler();
|
|
|
} else {
|
|
|
this.nodesSampler = new SimpleNodeSampler();
|
|
|
}
|
|
|
+ this.hostFailureListener = hostFailureListener;
|
|
|
this.nodesSamplerFuture = threadPool.schedule(nodesSamplerInterval, ThreadPool.Names.GENERIC, new ScheduledNodeSampler());
|
|
|
}
|
|
|
|
|
@@ -224,13 +215,17 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
}
|
|
|
ensureNodesAreAvailable(nodes);
|
|
|
int index = getNodeNumber();
|
|
|
- RetryListener<Response> retryListener = new RetryListener<>(callback, listener, nodes, index);
|
|
|
- DiscoveryNode node = nodes.get((index) % nodes.size());
|
|
|
+ RetryListener<Response> retryListener = new RetryListener<>(callback, listener, nodes, index, hostFailureListener);
|
|
|
+ DiscoveryNode node = retryListener.getNode(0);
|
|
|
try {
|
|
|
callback.doWithNode(node, retryListener);
|
|
|
} catch (Exception e) {
|
|
|
- //this exception can't come from the TransportService as it doesn't throw exception at all
|
|
|
- listener.onFailure(e);
|
|
|
+ try {
|
|
|
+ //this exception can't come from the TransportService as it doesn't throw exception at all
|
|
|
+ listener.onFailure(e);
|
|
|
+ } finally {
|
|
|
+ retryListener.maybeNodeFailed(node, e);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -239,15 +234,17 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
private final ActionListener<Response> listener;
|
|
|
private final List<DiscoveryNode> nodes;
|
|
|
private final int index;
|
|
|
+ private final TransportClient.HostFailureListener hostFailureListener;
|
|
|
|
|
|
private volatile int i;
|
|
|
|
|
|
public RetryListener(NodeListenerCallback<Response> callback, ActionListener<Response> listener,
|
|
|
- List<DiscoveryNode> nodes, int index) {
|
|
|
+ List<DiscoveryNode> nodes, int index, TransportClient.HostFailureListener hostFailureListener) {
|
|
|
this.callback = callback;
|
|
|
this.listener = listener;
|
|
|
this.nodes = nodes;
|
|
|
this.index = index;
|
|
|
+ this.hostFailureListener = hostFailureListener;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -257,13 +254,15 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
|
|
|
@Override
|
|
|
public void onFailure(Exception e) {
|
|
|
- if (ExceptionsHelper.unwrapCause(e) instanceof ConnectTransportException) {
|
|
|
+ Throwable throwable = ExceptionsHelper.unwrapCause(e);
|
|
|
+ if (throwable instanceof ConnectTransportException) {
|
|
|
+ maybeNodeFailed(getNode(this.i), (ConnectTransportException) throwable);
|
|
|
int i = ++this.i;
|
|
|
if (i >= nodes.size()) {
|
|
|
listener.onFailure(new NoNodeAvailableException("None of the configured nodes were available: " + nodes, e));
|
|
|
} else {
|
|
|
try {
|
|
|
- callback.doWithNode(nodes.get((index + i) % nodes.size()), this);
|
|
|
+ callback.doWithNode(getNode(i), this);
|
|
|
} catch(final Exception inner) {
|
|
|
inner.addSuppressed(e);
|
|
|
// this exception can't come from the TransportService as it doesn't throw exceptions at all
|
|
@@ -275,7 +274,15 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ final DiscoveryNode getNode(int i) {
|
|
|
+ return nodes.get((index + i) % nodes.size());
|
|
|
+ }
|
|
|
|
|
|
+ final void maybeNodeFailed(DiscoveryNode node, Exception ex) {
|
|
|
+ if (ex instanceof NodeDisconnectedException || ex instanceof NodeNotConnectedException) {
|
|
|
+ hostFailureListener.onNodeDisconnected(node, ex);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -377,6 +384,7 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
logger.debug(
|
|
|
(Supplier<?>)
|
|
|
() -> new ParameterizedMessage("failed to connect to node [{}], removed from nodes list", listedNode), e);
|
|
|
+ hostFailureListener.onNodeDisconnected(listedNode, e);
|
|
|
newFilteredNodes.add(listedNode);
|
|
|
continue;
|
|
|
}
|
|
@@ -411,6 +419,7 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
logger.info(
|
|
|
(Supplier<?>) () -> new ParameterizedMessage("failed to get node info for {}, disconnecting...", listedNode), e);
|
|
|
transportService.disconnectFromNode(listedNode);
|
|
|
+ hostFailureListener.onNodeDisconnected(listedNode, e);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -489,6 +498,7 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
"failed to get local cluster state for {}, disconnecting...", listedNode), e);
|
|
|
transportService.disconnectFromNode(listedNode);
|
|
|
latch.countDown();
|
|
|
+ hostFailureListener.onNodeDisconnected(listedNode, e);
|
|
|
}
|
|
|
});
|
|
|
} catch (Exception e) {
|
|
@@ -497,6 +507,7 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
"failed to get local cluster state info for {}, disconnecting...", listedNode), e);
|
|
|
transportService.disconnectFromNode(listedNode);
|
|
|
latch.countDown();
|
|
|
+ hostFailureListener.onNodeDisconnected(listedNode, e);
|
|
|
}
|
|
|
}
|
|
|
});
|
|
@@ -531,4 +542,9 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
|
|
|
|
|
|
void doWithNode(DiscoveryNode node, ActionListener<Response> listener);
|
|
|
}
|
|
|
+
|
|
|
+ // pkg private for testing
|
|
|
+ void doSample() {
|
|
|
+ nodesSampler.doSample();
|
|
|
+ }
|
|
|
}
|